Monitoring is only available for Enterprise changefeeds.
Changefeeds work as jobs in CockroachDB, which allows for monitoring and debugging through the DB Console Jobs page and SHOW JOBS SQL statements using the job ID.
By default, changefeeds treat errors as retryable except for some specific terminal errors that are non-retryable.
- retryable: The changefeed will automatically retry whatever caused the error. (You may need to intervene so that the changefeed can resume.)
- non-retryable: The changefeed has encountered a terminal error and fails.
The following define the categories of non-retryable errors:
- When the changefeed cannot verify the target table's schema. For example, the table is offline or there are types within the table that the changefeed cannot handle.
- The changefeed cannot convert the data to the specified output format. For example, there are Avro types that changefeeds do not support, or a CDC query is using an unsupported or malformed expression.
- The terminal error happens as part of established changefeed behavior. For example, you have specified the schema_change_policy=stopoption and a schema change happens.
We recommend monitoring changefeeds with Prometheus to avoid accumulation of garbage after a changefeed encounters an error. See Garbage collection and changefeeds for more detail on how changefeeds interact with protected timestamps and garbage collection. In addition, see the Recommended changefeed metrics to track section for the essential metrics to track on a changefeed.
Monitor a changefeed
Changefeed progress is exposed as a high-water timestamp that advances as the changefeed progresses. This is a guarantee that all changes before or at the timestamp have been emitted. You can monitor a changefeed:
- On the Changefeeds dashboard of the DB Console.
- On the Jobs page of the DB Console. Hover over the high-water timestamp to view the system time.
- Using - SHOW CHANGEFEED JOB <job_id>:- SHOW CHANGEFEED JOB 383870400694353921;- job_id | job_type | description | ... | high_water_timestamp | ... | +--------------------+------------+-----------------------------------------------------------------------+ ... +--------------------------------+ ... + 383870400694353921 | CHANGEFEED | CREATE CHANGEFEED FOR TABLE office_dogs INTO 'kafka://localhost:9092' | ... | 1537279405671006870.0000000000 | ... | (1 row)
- Using Prometheus and Alertmanager to track and alert on changefeed metrics. See the Monitor CockroachDB with Prometheus tutorial for steps to set up Prometheus. See the Recommended changefeed metrics to track section for the essential metrics to alert you when a changefeed encounters a retryable error, or enters a failed state. 
You can use the high-water timestamp to start a new changefeed where another ended.
Recommended changefeed metrics to track
By default, changefeeds will retry errors with some exceptions. We recommend setting up monitoring for the following metrics to track retryable errors to avoid an over-accumulation of garbage, and non-retryable errors to alert on changefeeds in a failed state:
- changefeed.max_behind_nanos: When a changefeed's high-water mark timestamp is at risk of falling behind the cluster's garbage collection window.
- changefeed.error_retries: The total number of retryable errors encountered by all changefeeds.
- changefeed.failures: The total number of changefeed jobs that have failed.
If you are running more than 10 changefeeds, we recommend monitoring the CPU usage on your cluster. You can use the Overload Dashboard in the DB Console to track the performance of your cluster relating to CPU usage. For recommendations around how many tables a changefeed should target, refer to System resources and running changefeeds.
Protected timestamp and garbage collection monitoring
Protected timestamps will protect changefeed data from garbage collection in particular scenarios, but if a changefeed lags too far behind, the protected changes could cause data storage issues. Refer to Protect Changefeed Data from Garbage Collection for detail on when changefeed data is protected from garbage collection.
You can monitor changefeed jobs for protected timestamp usage. We recommend setting up monitoring for the following metrics:
- jobs.changefeed.protected_age_sec: Tracks the age of the oldest protected timestamp record protected by changefeed jobs. We recommend monitoring if- protected_age_secis greater than- gc.ttlseconds. As- protected_age_secincreases, garbage accumulation increases. Garbage collection will not progress on a table, database, or cluster if the protected timestamp record is present.
- jobs.changefeed.currently_paused: Tracks the number of changefeed jobs currently considered paused. Since paused changefeed jobs can accumulate garbage, it is important to monitor the number of paused changefeeds.
- jobs.changefeed.expired_pts_records: Tracks the number of expired protected timestamp records owned by changefeed jobs. You can monitor this metric in conjunction with the- gc_protect_expires_afteroption.
- jobs.changefeed.protected_record_count: Tracks the number of protected timestamp records held by changefeed jobs.
Schema registry metrics
If you are running a changefeed with the confluent_schema_registry option, set up monitoring for the following metrics:
- changefeed.schema_registry.retry_count: The number of retries encountered when sending requests to the schema registry. A non-zero value could indicate incorrect configuration of the schema registry or changefeed parameters.
- changefeed.schema_registry.registrations: The number of registration attempts with the schema registry.
Using changefeed metrics labels
An Enterprise license is required to use metrics labels in changefeeds.
To measure metrics per changefeed, you can define a "metrics label" for one or multiple changefeed(s). The changefeed(s) will increment each changefeed metric. Metrics label information is sent with time-series metrics to http://{host}:{http-port}/_status/vars, viewable via the Prometheus endpoint. An aggregated metric of all changefeeds is also measured.
It is necessary to consider the following when applying metrics labels to changefeeds:
- The server.child_metrics.enabledcluster setting must be set totruebefore using themetrics_labeloption.server.child_metrics.enabledis enabled by default in Standard and Basic.
- Metrics label information is sent to the _status/varsendpoint, but will not show up indebug.zipor the DB Console.
- Introducing labels to isolate a changefeed's metrics can increase cardinality significantly. There is a limit of 1024 unique labels in place to prevent cardinality explosion. That is, when labels are applied to high-cardinality data (data with a higher number of unique values), each changefeed with a label then results in more metrics data to multiply together, which will grow over time. This will have an impact on performance as the metric-series data per changefeed quickly populates against its label.
- The maximum length of a metrics label is 128 bytes.
To start a changefeed with a metrics label, set the following cluster setting to true:
SET CLUSTER SETTING server.child_metrics.enabled=true;
Create the changefeed, passing the metrics_label option with the label name as its value:
CREATE CHANGEFEED FOR TABLE movr.rides INTO 'kafka://host:port' WITH metrics_label=rides;
CREATE CHANGEFEED FOR TABLE movr.vehicles INTO 'kafka://host:port' WITH metrics_label=vehicles;
Multiple changefeeds can be added to a label:
CREATE CHANGEFEED FOR TABLE movr.vehicle_location_histories INTO 'kafka://host:port' WITH metrics_label=vehicles;
http://{host}:{http-port}/_status/vars shows the defined changefeed(s) by label and the aggregated metric for all changefeeds. This output also shows the default scope, which will include changefeeds started without a metrics label:
changefeed_running 4
changefeed_running{scope="default"} 1
changefeed_running{scope="rides"} 1
changefeed_running{scope="vehicles"} 2
changefeed_emitted_messages 4144
changefeed_emitted_messages{scope="default"} 0
changefeed_emitted_messages{scope="rides"} 2772
changefeed_emitted_messages{scope="vehicles"} 1372
changefeed_emitted_bytes 781591
changefeed_emitted_bytes{scope="default"} 0
changefeed_emitted_bytes{scope="rides"} 598034
changefeed_emitted_bytes{scope="vehicles"} 183557
Metrics
| Metric | Description | Unit | Type | 
|---|---|---|---|
| changefeed.admit_latency | Difference between the event's MVCC timestamp and the time the event is put into the memory buffer. | Nanoseconds | Histogram | 
| changefeed.aggregator_progress | The earliest timestamp up to which any aggregator is guaranteed to have emitted all values for which it is responsible. Note: This metric may regress when a changefeed restarts due to a transient error. Consider tracking the changefeed.checkpoint_progressmetric, which will not regress. | Timestamp | Gauge | 
| changefeed.backfill_count | Number of changefeeds currently executing a backfill (schema change or initial scan). | Changefeeds | Gauge | 
| changefeed.backfill_pending_ranges | Number of ranges in an ongoing backfill that are yet to be fully emitted. | Ranges | Gauge | 
| changefeed.checkpoint_hist_nanos | Time spent checkpointing changefeed progress. | Nanoseconds | Histogram | 
| changefeed.checkpoint_progress | The earliest timestamp of any changefeed's persisted checkpoint (values prior to this timestamp will never need to be re-emitted). | Timestamp | Histogram | 
| changefeed.commit_latency | Difference between the event's MVCC timestamp and the time it is acknowledged by the downstream sink. If the sink is batching events, then the difference is between the oldest event and when the acknowledgment is recorded. | Nanoseconds | Histogram | 
| New in v24.1: changefeed.emitted_batch_sizes | Size of batches emitted to the sink by all changefeeds. If changefeeds have the resolvedoption enabled, this metric will include resolved timestamp messages, which emit in a batch size of 1 message. | Messages | Histogram | 
| changefeed.emitted_bytes | Number of bytes emitted, which increments as messages are flushed. | Bytes | Counter | 
| changefeed.emitted_messages | Number of messages emitted, which increments when messages are flushed. | Messages | Counter | 
| changefeed.error_retries | Total retryable errors encountered by changefeeds. | Errors | Counter | 
| changefeed.flushed_bytes | Bytes emitted by all changefeeds. This may differ from emitted_byteswhencompressionis enabled. | Bytes | Counter | 
| changefeed.flush_hist_nanos | Time spent flushing messages across all changefeeds. | Nanoseconds | Histograms | 
| changefeed.flushes | Total number of flushes for a changefeed. | Flushes | Counter | 
| changefeed.lagging_ranges | Number of ranges which are behind in a changefeed. This is calculated based on the options: 
 lagging_ranges_thresholdduration are considered to be lagging. Starting a changefeed with an initial scan on a large table will likely increment the metric for each range in the table. As ranges complete the initial scan, the number of ranges lagging behind will decrease. | Nanoseconds | Gauge | 
| changefeed.message_size_hist | Distribution in the size of emitted messages. | Bytes | Histogram | 
| changefeed.running | Number of currently running changefeeds, including sinkless changefeeds. | Changefeeds | Gauge | 
| changefeed.sink_batch_hist_nanos | Time messages spend batched in the sink buffer before being flushed and acknowledged. | Nanoseconds | Histogram | 
| New in v24.1.6: changefeed.total_ranges | Total number of ranges that are watched by aggregator processors participating in the changefeed job. changefeed.total_rangesshares the same polling interval as thechangefeed.lagging_rangesmetric, which is controlled by thelagging_ranges_polling_intervaloption. For more details, refer to Lagging ranges. | 
Monitoring and measuring changefeed latency
Changefeeds can encounter latency in events processing. This latency is the total time CockroachDB takes to:
- Commit writes to the database.
- Encode changefeed messages.
- Deliver the message to the sink.
There are a couple of ways to measure if changefeeds are encountering latency or falling behind:
- Event latency: Measure the difference between an event's MVCC timestamp and when it is put into the memory buffer or acknowledged at the sink.
- Lagging ranges: Track the number of ranges that are behind in a changefeed.
Event latency
To monitor for changefeeds encountering latency in how events are emitting, track the following metrics:
- admit_latency: The difference between the event's MVCC timestamp and the time the event is put into the memory buffer.
- commit_latency: The difference between the event's MVCC timestamp and the time it is acknowledged by the downstream sink. If the sink is batching events, the difference is between the oldest event and when the acknowledgment is recorded.
The admit_latency and commit_latency metrics do not update for backfills during initial scans or backfills for schema changes. This is because a full table scan may contain rows that were written far in the past, which would lead to inaccurate changefeed latency measurements if the events from these scans were included in admit_latency adn commit_latency.
Both of these metrics support metrics labels. You can set the metrics_label option when starting a changefeed to differentiate metrics per changefeed.
We recommend using the p99 commit_latency aggregation for alerting and to set SLAs for your changefeeds. Refer to the Changefeed Dashboard Commit Latency graph to track this metric in the DB Console.
If your changefeed is experiencing elevated latency, you can use these metrics to:
- Review admit_latencyversuscommit_latencyto calculate the time events are moving from the memory buffer to the downstream sink.
- Compare the commit_latencyP99, P90, P50 latency percentiles to investigate performance over time.
Lagging ranges
Use the changefeed.lagging_ranges metric to track the number of ranges that are behind in a changefeed. This is calculated based on the changefeed options:
- lagging_ranges_thresholdsets a duration from the present that determines the length of time a range is considered to be lagging behind, which will then track in the- lagging_rangesmetric. Note that ranges undergoing an initial scan for longer than the threshold duration are considered to be lagging. Starting a changefeed with an initial scan on a large table will likely increment the metric for each range in the table. As ranges complete the initial scan, the number of ranges lagging behind will decrease.- Default: 3m
 
- Default: 
- lagging_ranges_polling_intervalsets the interval rate for when lagging ranges are checked and the- lagging_rangesmetric is updated. Polling adds latency to the- lagging_rangesmetric being updated. For example, if a range falls behind by 3 minutes, the metric may not update until an additional minute afterward.- Default: 1m
 
- Default: 
New in v24.1.6:
 Use the changefeed.total_ranges metric to monitor the number of ranges that are watched by aggregator processors participating in the changefeed job. If you're experiencing lagging ranges, changefeed.total_ranges may indicate that the number of ranges watched by aggregator processors in the job is unbalanced. You may want to try pausing the changefeed and then resuming it, so that the changefeed replans the work in the cluster. changefeed.total_ranges shares the same polling interval as the changefeed.lagging_ranges metric, which is controlled by the lagging_ranges_polling_interval option.
You can use the metrics_label option to track the lagging_ranges and total_ranges metric per changefeed.
Debug a changefeed
Using logs
For Enterprise changefeeds, use log information to debug connection issues (i.e., kafka: client has run out of available brokers to talk to (Is your cluster reachable?)). Debug by looking for lines in the logs with [kafka-producer] in them:
I190312 18:56:53.535646 585 vendor/github.com/Shopify/sarama/client.go:123  [kafka-producer] Initializing new client
I190312 18:56:53.535714 585 vendor/github.com/Shopify/sarama/client.go:724  [kafka-producer] client/metadata fetching metadata for all topics from broker localhost:9092
I190312 18:56:53.536730 569 vendor/github.com/Shopify/sarama/broker.go:148  [kafka-producer] Connected to broker at localhost:9092 (unregistered)
I190312 18:56:53.537661 585 vendor/github.com/Shopify/sarama/client.go:500  [kafka-producer] client/brokers registered new broker #0 at 172.16.94.87:9092
I190312 18:56:53.537686 585 vendor/github.com/Shopify/sarama/client.go:170  [kafka-producer] Successfully initialized new client
Using SHOW CHANGEFEED JOBS
For Enterprise changefeeds, use SHOW CHANGEFEED JOBS to check the status of your changefeed jobs:
SHOW CHANGEFEED JOBS;
        job_id       |               description                                                                                                                                      | user_name | status  |         running_status                    |          created           |          started           | finished |          modified          |      high_water_timestamp      | error |   sink_uri                                                                            |                    full_table_names                     |        topics         | format
---------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------+---------+-------------------------------------------+----------------------------+----------------------------+----------+----------------------------+--------------------------------+-------+---------------------------------------------------------------------------------------+---------------------------------------------------------+-----------------------+---------
786667716931878913   | CREATE CHANGEFEED FOR TABLE movr.users INTO 's3://changefeed_bucket?AWS_ACCESS_KEY_ID={ACCESS KEY}L&AWS_SECRET_ACCESS_KEY=redacted' WITH resolved              | user_name | running | running: resolved=1660144018.670219356,0  | 2022-08-10 14:38:55.098861 | 2022-08-10 14:38:55.12946  | NULL     | 2022-08-10 15:07:31.886757 | 1660144048833832849.0000000000 |       | s3://changefeed_bucket?AWS_ACCESS_KEY_ID={ACCESS KEY}&AWS_SECRET_ACCESS_KEY=redacted  | {movr.public.users}                                     | NULL                  | json
685724608744325121   | CREATE CHANGEFEED FOR TABLE mytable INTO 'kafka://localhost:9092' WITH confluent_schema_registry = 'http://localhost:8081', format = 'avro', resolved, updated | root      | running | running: resolved=1629336943.183631090,0  | 2021-08-19 01:35:43.19592  | 2021-08-19 01:35:43.225445 | NULL     | 2021-08-19 01:35:43.252318 | 1629336943183631090.0000000000 |       | kafka://localhost:9092                                                                | {defaultdb.public.mytable}                              | mytable               | avro
For more information, see SHOW JOBS.
Using the DB Console
On the Custom Chart debug page of the DB Console:
- To add a chart, click Add Chart.
- Select - changefeed.error_retriesfrom the Metric Name dropdown menu.- A graph of changefeed restarts due to retryable errors will display.