diff --git a/modules/deploy/pages/deployment-option/self-hosted/manual/sizing.adoc b/modules/deploy/pages/deployment-option/self-hosted/manual/sizing.adoc index 7cc874eb3..8afce7ada 100644 --- a/modules/deploy/pages/deployment-option/self-hosted/manual/sizing.adoc +++ b/modules/deploy/pages/deployment-option/self-hosted/manual/sizing.adoc @@ -153,6 +153,89 @@ The https://github.com/redpanda-data/openmessaging-benchmark[Open Messaging Benc See also: https://github.com/redpanda-data/openmessaging-benchmark/blob/main/driver-redpanda/README.md[Redpanda Benchmarks^] +== Assess throughput + +This section describes how to use the xref:reference:rpk/rpk-topic-analyze.adoc[`rpk topic analyze`] command to check how much work your Redpanda cluster is handling. It shows the number of messages the cluster is processing and the size of the data groups (batches). This information helps you decide if you need to add more servers or make changes to your setup. + +This command shows you the throughput of your Redpanda cluster: + +[source,bash] +---- +rpk topic analyze --regex '*' --print-all --time-range -1m:end +---- + +The arguments are: + +* `--regex '*'`: Analyzes all topics. +* `--print-all`: Prints all the metrics. +* `--time-range -1m:end`: Analyzes the last minute of data. + +Example output: + +[,bash,role="no-copy no-wrap"] +---- +SUMMARY +======= +TOPICS 6 +PARTITIONS 17 +TOTAL THROUGHPUT (BYTES/S) 1361.9166666666667 +TOTAL BATCH RATE (BATCHES/S) 2.9833333333333334 +AVERAGE BATCH SIZE (BYTES) 456.50837988826817 + +TOPIC SUMMARY +============= +TOPIC PARTITIONS BYTES-PER-SECOND BATCHES-PER-SECOND AVERAGE-BYTES-PER-BATCH +_redpanda.audit_log 12 61 0.1 610 +_redpanda.transform_logs 1 890.2666666666667 0.7833333333333333 1136.5106382978724 +_schemas 1 0 0 0 +edu-filtered-domains 1 14.283333333333333 0.1 142.83333333333334 +logins 1 144.61666666666667 1 144.61666666666667 +transactions 1 251.75 1 251.75 + +PARTITION BATCH RATE (BATCHES/S) +================================ +TOPIC P25 P50 P75 P99 +_redpanda.audit_log 0.016666666666666666 0.016666666666666666 0.03333333333333333 0.03333333333333333 +_redpanda.transform_logs 0.7833333333333333 0.7833333333333333 0.7833333333333333 0.7833333333333333 +_schemas 0 0 0 0 +edu-filtered-domains 0.1 0.1 0.1 0.1 +logins 1 1 1 1 +transactions 1 1 1 1 + +PARTITION BATCH SIZE (BYTES) +============================ +TOPIC P25 P50 P75 P99 +_redpanda.audit_log 608 610 610 611 +_redpanda.transform_logs 895 895 895 895 +_schemas 0 0 0 0 +edu-filtered-domains 141 141 141 141 +logins 144 144 144 144 +transactions 255 255 255 255 +---- + +* **Total throughput:** + Indicates the total amount of data processed by the cluster every second. + +* **Total batch rate:** + Shows the number of message batches processed per second. A higher rate suggests increased activity, which may require more CPU or I/O resources. + +* **Average batch size:** + Reflects the average size of each message batch. Large or inconsistent batch sizes may indicate the need to adjust producer settings or verify storage capacity. + +* **Topic and partition summaries:** + Provides details on resource usage by individual topics. For example, if a single topic (such as `_redpanda.transform_logs` in the example output) is responsible for most throughput, it may need optimization or additional resources. + +* **Percentiles (P25, P50, P75, P99):** + Offers insights into workload distribution across partitions. Consistent values suggest balanced workloads, while significant variations may highlight areas that need rebalancing or capacity adjustments. + +=== Plan for capacity + +Compare the current throughput and batch rate with your cluster's hardware limits, such as network bandwidth, disk IOPS, or CPU capacity. If usage is nearing these limits, consider scaling up (upgrading hardware) or scaling out (adding brokers). Monitor trends over time to anticipate when expansion is necessary. + +=== Address bottlenecks + +If specific topics or partitions consistently show higher loads, it may indicate uneven workload distribution. Redistribute partitions or adjust replication factors to balance the load more effectively. + include::shared:partial$suggested-reading.adoc[] * https://redpanda.com/blog/sizing-redpanda-cluster-best-practices[Four sizing principles for Redpanda production clusters^] diff --git a/modules/reference/pages/internal-metrics-reference.adoc b/modules/reference/pages/internal-metrics-reference.adoc index 84bd8c692..738bd625b 100644 --- a/modules/reference/pages/internal-metrics-reference.adoc +++ b/modules/reference/pages/internal-metrics-reference.adoc @@ -18,293 +18,4673 @@ curl http://:9644/metrics | grep "[HELP|TYPE]" ---- ==== -== Internal metrics +== vectorized_alien_receive_batch_queue_length -Most internal metrics are useful for debugging. The following subset of internal metrics can be useful to monitor system health. +Indicates the current length of the alien receive batch queue. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_alien_total_received_messages +Tracks the cumulative number of alien messages that have been received. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_alien_total_sent_messages +Tracks the cumulative number of alien messages that have been sent. + +*Type*: counter + +*Labels*: + +- `shard` --- -=== vectorized_application_uptime +== vectorized_application_build +Provides build details for Redpanda including the build revision, shard identifier, and version. -Redpanda uptime in milliseconds. +*Type*: gauge + +*Labels*: + +- `revision` +- `shard` +- `version` --- -=== vectorized_cloud_storage_read_bytes +== vectorized_application_fips_mode +Indicates whether Redpanda is running in FIPS (Federal Information Processing Standards) mode. + +*Type*: gauge -Number of bytes Redpanda has read from cloud storage. This is tracked on a per-topic and per-partition basis, and increments each time a cloud storage read operation successfully completes. +*Labels*: + +- `shard` --- -=== vectorized_cluster_partition_last_stable_offset +== vectorized_application_uptime +Reports the uptime of the Redpanda application in milliseconds. -Last stable offset. +*Type*: gauge + +*Labels*: -If this is the last record received by the cluster, then the cluster is up-to-date and ready for maintenance. +- `shard` --- -=== vectorized_cluster_features_enterprise_license_expiry_sec +== vectorized_archival_upload_backlog_controller_backlog_size +Shows the current backlog size in the archival upload controller. -Number of seconds remaining until the Enterprise Edition license expires. +*Type*: gauge + +*Labels*: + +- `shard` --- -=== vectorized_cluster_partition_schema_id_validation_records_failed +== vectorized_archival_upload_backlog_controller_error +Reports the current error value (difference between set point and backlog size) in the archival upload controller. + +*Type*: gauge -Number of records that failed schema ID validation. +*Labels*: + +- `shard` --- -=== vectorized_cluster_partition_start_offset +== vectorized_archival_upload_backlog_controller_shares +Represents the number of shares (or work units) output by the archival upload controller. + +*Type*: gauge + +*Labels*: -Raft snapshot start offset. +- `shard` --- -=== vectorized_io_queue_delay +== vectorized_authorization_result +Counts the total number of authorization results, broken down by result type. + +*Type*: counter -Total delay time in the queue. +*Labels*: -Can indicate latency caused by disk operations in seconds. +- `type` --- -=== vectorized_io_queue_queue_length +== vectorized_chunk_cache_available_size_bytes +Measures the total size (in bytes) of all free segment appender chunks in the cache. -Number of requests in the queue. +*Type*: gauge -Can indicate latency caused by disk operations. +*Labels*: + +- `shard` --- -=== vectorized_kafka_quotas_balancer_runs +== vectorized_chunk_cache_total_size_bytes +Reports the total size (in bytes) of all segment appender chunks regardless of state. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- -Number of times the throughput quota balancer has executed. +== vectorized_chunk_cache_wait_count +Counts how many times a request had to wait for a segment appender chunk to become available. *Type*: counter +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_client_active_downloads +Indicates the number of active GET requests to the cloud storage. + +*Type*: gauge + +*Labels*: + +- `endpoint` +- `region` +- `shard` + +--- + +== vectorized_cloud_client_active_requests +Reports the number of active HTTP requests (both PUT and GET) currently in progress. + +*Type*: gauge + +*Labels*: + +- `endpoint` +- `region` +- `shard` + --- -=== vectorized_kafka_quotas_quota_effective +== vectorized_cloud_client_active_uploads +Shows the number of active PUT requests currently executing. + +*Type*: gauge + +*Labels*: + +- `endpoint` +- `region` +- `shard` + +--- -Current effective quota for the quota balancer, in bytes per second. +== vectorized_cloud_client_all_requests +Counts the total number of completed HTTP requests (includes both PUT and GET). *Type*: counter +*Labels*: + +- `endpoint` +- `region` +- `shard` + --- -=== vectorized_kafka_quotas_client_quota_throttle_time +== vectorized_cloud_client_client_pool_utilization +Measures the utilization of the cloud storage client pool as a percentage (0 means unused, 100 means fully utilized). + +*Type*: gauge + +*Labels*: + +- `endpoint` +- `region` +- `shard` + +--- -Client quota throttling delay, in seconds, per rule and quota type based on xref:manage:cluster-maintenance/manage-throughput.adoc#client-throughput-limits[client throughput limits]. +== vectorized_cloud_client_lease_duration +Provides a histogram of cloud client lease durations. *Type*: histogram +--- + +== vectorized_cloud_client_num_borrows +Counts how many times a shard has borrowed a cloud storage client from another shard. + +*Type*: counter + *Labels*: -* `quota_rule=("not_applicable" | "kafka_client_default" | "cluster_client_default" | "kafka_client_prefix" | "cluster_client_prefix" | "kafka_client_id")` -* `quota_type=("produce_quota" | "fetch_quota" | "partition_mutation_quota")` +- `endpoint` +- `region` +- `shard` --- -=== vectorized_kafka_quotas_client_quota_throughput +== vectorized_cloud_client_num_nosuchkey +Counts the total number of NoSuchKey errors returned by the cloud storage provider. -Client quota throughput per rule and quota type. +*Type*: counter -*Type*: histogram +*Labels*: + +- `endpoint` +- `region` +- `shard` + +--- + +== vectorized_cloud_client_num_rpc_errors +Counts the total number of REST API errors encountered from the cloud storage provider. + +*Type*: counter *Labels*: -* `quota_rule=("not_applicable" | "kafka_client_default" | "cluster_client_default" | "kafka_client_prefix" | "cluster_client_prefix" | "kafka_client_id")` -* `quota_type=("produce_quota" | "fetch_quota" | "partition_mutation_quota")` +- `endpoint` +- `region` +- `shard` --- -=== vectorized_kafka_quotas_throttle_time +== vectorized_cloud_client_num_slowdowns +Counts the total number of SlowDown errors received from the cloud storage provider. -Histogram of throttle times, in seconds, based on xref:manage:cluster-maintenance/manage-throughput.adoc#broker-wide-throughput-limits[broker-wide throughput limits]. +*Type*: counter -*Type*: histogram +*Labels*: + +- `endpoint` +- `region` +- `shard` --- -=== vectorized_kafka_quotas_traffic_intake +== vectorized_cloud_client_num_transport_errors +Counts the total number of transport errors (including TCP and TLS errors) from the cloud storage provider. + +*Type*: counter + +*Labels*: + +- `endpoint` +- `region` +- `shard` + +--- -Total amount of Kafka traffic (in bytes) taken in from clients for processing that was considered by the throttler. +== vectorized_cloud_client_total_downloads +Counts the total number of completed `GET` requests to cloud storage. *Type*: counter +*Labels*: + +- `endpoint` +- `region` +- `shard` + --- -=== vectorized_kafka_quotas_traffic_egress +== vectorized_cloud_client_total_inbound_bytes +Reports the total number of bytes received from cloud storage. + +*Type*: counter + +*Labels*: + +- `endpoint` +- `region` +- `shard` + +--- -Total amount of Kafka traffic (in bytes) published to clients that was considered by the throttler. +== vectorized_cloud_client_total_outbound_bytes +Reports the total number of bytes sent to cloud storage. *Type*: counter +*Labels*: + +- `endpoint` +- `region` +- `shard` + --- -=== vectorized_kafka_rpc_active_connections +== vectorized_cloud_client_total_uploads +Counts the total number of completed PUT requests to cloud storage. + +*Type*: counter + +*Labels*: -Number of currently active Kafka RPC connections, or clients. +- `endpoint` +- `region` +- `shard` --- -=== vectorized_kafka_rpc_connects +== vectorized_cloud_storage_bytes_received +Reports the total number of bytes received from cloud storage. + +*Type*: counter -Number of accepted Kafka RPC connections. +*Labels*: -Compare to the value at a previous time to derive the rate of accepted connections. +- `shard` --- -[[vectorized_kafka_rpc_produce_bad_create_time]] -=== vectorized_kafka_rpc_produce_bad_create_time +== vectorized_cloud_storage_bytes_sent +Reports the total number of bytes sent to cloud storage. -An incrementing counter for the number of times a producer created a message with a timestamp skewed from the broker's date and time. This metric is related to the following properties: +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_cache_cached_gets +Counts the number of cache get requests that were served from the cache. + +*Type*: counter + +*Labels*: -* `log_message_timestamp_alert_before_ms`: Increment this gauge when the `create_timestamp` on a message is too far in the past as compared to the broker's time. -* `log_message_timestamp_alert_after_ms`: Increment this gauge when the `create_timestamp` on a message is too far in the future as compared to the broker's time. +- `shard` --- -=== vectorized_kafka_rpc_received_bytes +== vectorized_cloud_storage_cache_files +Indicates the current number of files stored in the cloud storage cache. + +*Type*: gauge -Number of bytes received from Kafka RPC clients in valid requests. +*Labels*: -Compare to the value at a previous time to derive the throughput in Kafka layer in bytes/sec received. +- `shard` --- -=== vectorized_kafka_rpc_requests_completed +== vectorized_cloud_storage_cache_gets +Counts the total number of get requests directed at the cloud storage cache. + +*Type*: counter -Number of successful Kafka RPC requests. +*Labels*: -Compare to the value at a previous time to derive the messages per second per shard. +- `shard` --- -=== vectorized_kafka_rpc_requests_pending +== vectorized_cloud_storage_cache_in_progress_files +Shows the current number of files that are in the process of being cached. + +*Type*: gauge + +*Labels*: -Number of Kafka RPC requests being processed by a server. +- `shard` --- -=== vectorized_kafka_rpc_sent_bytes +== vectorized_cloud_storage_cache_puts +Counts the total number of files successfully placed into the cache. + +*Type*: counter + +*Labels*: -Number of bytes sent to Kafka RPC clients. +- `shard` --- -=== vectorized_kafka_rpc_service_errors +== vectorized_cloud_storage_cache_size_bytes +Reports the current size of the cloud storage cache in bytes. + +*Type*: gauge + +*Labels*: -Number of Kafka RPC service errors. +- `shard` --- -=== vectorized_ntp_archiver_compacted_replaced_bytes +== vectorized_cloud_storage_client_acquisition_latency +Provides a histogram of the latency experienced when acquiring a cloud storage client. -Number of bytes removed from cloud storage by compaction operations. This is tracked on a per-topic and per-partition basis. +*Type*: histogram -This metric resets every time partition leadership changes. It tracks whether or not compaction is performing operations on cloud storage. +--- -The `namespace` label supports the following options for this metric: +== vectorized_cloud_storage_cluster_metadata_manifest_downloads +Counts the number of partition manifest downloads initiated by the cluster. -* `kafka` - User topics -* `kafka_internal` - Internal Kafka topic, such as consumer groups -* `redpanda` - Redpanda-only internal data +*Type*: counter *Labels*: -* `namespace=("kafka" | "kafka_internal" | "redpanda")` -* `topic` -* `partition` +- `shard` --- -=== vectorized_ntp_archiver_pending +== vectorized_cloud_storage_cluster_metadata_manifest_uploads +Counts the number of partition manifest uploads initiated by the cluster. -The difference between the last committed offset and the last offset uploaded to Tiered Storage for each partition. A value of zero for this metric indicates that all data for a partition is uploaded to Tiered Storage. +*Type*: counter -This metric is impacted by the xref:reference:tunable-properties.adoc#cloud_storage_segment_max_upload_interval_sec[`cloud_storage_segment_max_upload_interval_sec`] tunable property. If this interval is set to 5 minutes, the archiver will upload committed segments to Tiered Storage every 5 minutes or less. If this metric continues growing for longer than the configured interval, it can indicate a potential network issue with the upload path for that partition. +*Labels*: -The `namespace` label supports the following options for this metric: +- `shard` -* `kafka` - User topics -* `kafka_internal` - Internal Kafka topic, such as consumer groups -* `redpanda` - Redpanda-only internal data +--- + +== vectorized_cloud_storage_controller_snapshot_failed_uploads +Counts the number of controller snapshot uploads that have failed. + +*Type*: counter *Labels*: -* `namespace=("kafka" | "kafka_internal" | "redpanda")` -* `topic` -* `partition` +- `shard` --- -=== vectorized_raft_leadership_changes +== vectorized_cloud_storage_controller_snapshot_successful_uploads +Counts the number of successful controller snapshot uploads. + +*Type*: counter -Number of leadership changes. +*Labels*: -High value can indicate nodes failing and causing leadership changes. +- `shard` --- -=== vectorized_reactor_utilization +== vectorized_cloud_storage_controller_snapshot_upload_backoff +Counts the number of times a backoff was applied during controller snapshot uploads. + +*Type*: counter -Redpanda process utilization. +*Labels*: -Shows the true utilization of the CPU by a Redpanda process. This metric has per-broker and per-shard granularity. If a shard (CPU core) is at 100% utilization for a continuous period of real-time processing, for example more than a few seconds, you will likely observe high latency for partitions assigned to that shard. Use topic-aware xref:manage:cluster-maintenance/cluster-balancing.adoc#intra-broker-partition-balancing[intra-broker partition balancing] to balance partition assignments and alleviate load on individual shards. +- `shard` --- -=== vectorized_storage_log_compacted_segment +== vectorized_cloud_storage_download_backoff +Counts how many times backoff was applied during log-segment downloads. + +*Type*: counter + +*Labels*: -Number of compacted segments. +- `shard` --- -=== vectorized_storage_log_compaction_removed_bytes +== vectorized_cloud_storage_failed_downloads +Counts the number of failed log-segment download attempts. -Number of bytes removed from local storage by compaction operations. This is tracked on a per-topic and per-partition basis. It tracks whether compaction is performing operations on local storage. +*Type*: counter -The `namespace` label supports the following options for this metric: +*Labels*: -* `kafka` - User topics -* `kafka_internal` - Internal Kafka topic, such as consumer groups -* `redpanda` - Redpanda-only internal data +- `shard` + +--- + +== vectorized_cloud_storage_failed_index_downloads +Counts the number of failed attempts to download segment indices. + +*Type*: counter *Labels*: -* `namespace=("kafka" | "kafka_internal" | "redpanda")` -* `topic` -* `partition` +- `shard` + +--- + +== vectorized_cloud_storage_failed_index_uploads +Counts the number of failed attempts to upload segment indices. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_failed_manifest_downloads +Counts the number of manifest download failures. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_failed_manifest_uploads +Counts the number of manifest upload failures. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_failed_uploads +Counts the total number of failed log-segment upload attempts. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_index_downloads +Counts the total number of segment index downloads. + +*Type*: counter + +*Labels*: + +- `shard` --- -=== vectorized_storage_log_log_segments_created +== vectorized_cloud_storage_index_uploads +Counts the total number of segment index uploads. + +*Type*: counter + +*Labels*: -Number of created log segments. +- `shard` --- -=== vectorized_storage_log_partition_size +== vectorized_cloud_storage_manifest_download_backoff +Counts how many times backoff was applied during manifest downloads. + +*Type*: counter + +*Labels*: -Current size of partition in bytes. +- `shard` --- -=== vectorized_storage_log_read_bytes +== vectorized_cloud_storage_manifest_upload_backoff +Counts how many times backoff was applied during manifest uploads. + +*Type*: counter + +*Labels*: -Total number of bytes read. +- `shard` --- -=== vectorized_storage_log_written_bytes +== vectorized_cloud_storage_partition_chunk_size +Reports the size (in bytes) of a chunk downloaded from cloud storage for a given partition. -Total number of bytes written. +*Type*: gauge -== Related topics +*Labels*: + +- `namespace` +- `partition` +- `topic` + +--- + +== vectorized_cloud_storage_partition_manifest_downloads +Counts the number of partition manifest downloads. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_partition_manifest_uploads +Counts the number of (re)uploads for partition manifests. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_partition_read_bytes +Reports the total number of bytes read from a remote partition. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `topic` + +--- + +== vectorized_cloud_storage_partition_read_records +Counts the total number of records read from a remote partition. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `topic` + +--- + +== vectorized_cloud_storage_read_path_chunk_hydration_latency +Provides a histogram of the latency for hydrating individual chunks. + +*Type*: histogram + +--- + +== vectorized_cloud_storage_read_path_chunks_hydrated +Counts the total number of chunks hydrated (note: some may later be evicted from the cache). + +*Type*: counter + +--- + +== vectorized_cloud_storage_read_path_downloads_throttled_sum +Reports the cumulative time (in milliseconds) that downloads were throttled. + +*Type*: counter + +--- + +== vectorized_cloud_storage_read_path_hydrations_in_progress +Indicates the number of chunk hydrations currently in progress. + +*Type*: counter + +--- + +== vectorized_cloud_storage_read_path_materialized_segments +Shows the current number of remote segments that have been materialized. + +*Type*: gauge + +--- + +== vectorized_cloud_storage_read_path_readers +Indicates the current number of remote partition readers active. + +*Type*: gauge + +--- + +== vectorized_cloud_storage_read_path_segment_readers +Indicates the current number of remote segment readers active. + +*Type*: gauge + +--- + +== vectorized_cloud_storage_read_path_spillover_manifest_bytes +Reports the total memory (in bytes) used by spillover manifests. + +*Type*: gauge + +--- + +== vectorized_cloud_storage_read_path_spillover_manifest_hydrated +Counts the number of times spillover manifests have been successfully cached. + +*Type*: counter + +--- + +== vectorized_cloud_storage_read_path_spillover_manifest_instances +Shows the current number of spillover manifest instances stored in memory. + +*Type*: gauge + +--- + +== vectorized_cloud_storage_read_path_spillover_manifest_latency +Provides a histogram of the latency for materializing spillover manifests. + +*Type*: histogram + +--- + +== vectorized_cloud_storage_read_path_spillover_manifest_materialized +Counts the number of times spillover manifests were loaded from the cache. + +*Type*: counter + +--- + +== vectorized_cloud_storage_segment_download_latency +Provides a histogram of the latency experienced during segment downloads. + +*Type*: histogram + +--- + +== vectorized_cloud_storage_spillover_manifest_downloads +Counts the number of spillover manifest downloads. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_spillover_manifest_uploads +Counts the number of spillover manifest (re)uploads. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_successful_downloads +Counts the number of successful log-segment downloads. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_successful_uploads +Counts the number of successful log-segment uploads. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_topic_manifest_downloads +Counts the number of topic manifest downloads. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_topic_manifest_uploads +Counts the number of topic manifest uploads. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cloud_storage_upload_backoff +Counts the number of times backoff was applied during log-segment uploads. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cluster_controller_pending_partition_operations +Indicates the number of partitions with pending or ongoing operations at the controller level. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_cluster_features_enterprise_license_expiry_sec +Reports the number of seconds remaining until the Enterprise Edition license expires. + +*Type*: gauge + +--- + +== vectorized_cluster_partition_batches_produced +Counts the total number of batches produced for a partition. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_bytes_fetched_from_follower_total +Reports the total number of bytes fetched from follower replicas (may exceed client-returned values). + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_bytes_fetched_total +Reports the total number of bytes fetched (some may not be returned to the client). + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_bytes_produced_total +Counts the total number of bytes produced for a partition. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_cloud_storage_segments_metadata_bytes +Reports the total number of bytes consumed by remote segments metadata for a partition. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_committed_offset +Shows the committed offset for a partition (safely persisted across a majority of replicas). + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_end_offset +Reports the last offset stored on the current broker for a partition. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_high_watermark +Indicates the high watermark (highest consumable offset) for a partition. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_last_stable_offset +Reports the last stable offset for a partition. A stable offset implies that the partition is fully caught up. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_leader +A flag indicating whether the current instance is the leader for the partition. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_leader_id +Reports the identifier of the current partition leader. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_moving_from_node +Indicates the number of partitions that are currently in the process of moving away from this broker. + +*Type*: gauge + +--- + +== vectorized_cluster_partition_moving_to_node +Indicates the number of partitions that are currently moving to this broker. + +*Type*: gauge + +--- + +== vectorized_cluster_partition_node_cancelling_movements +Reports the number of partition movements being cancelled on this broker. + +*Type*: gauge + +--- + +== vectorized_cluster_partition_num_with_broken_rack_constraint +Counts the number of partitions that do not meet rack awareness constraints. + +*Type*: gauge + +--- + +== vectorized_cluster_partition_records_fetched +Counts the total number of records fetched from a partition. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_records_produced +Counts the total number of records produced to a partition. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_start_offset +Indicates the starting offset (raft snapshot offset) for a partition. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_partition_under_replicated_replicas +Reports the number of under-replicated replicas for a partition. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_cluster_producer_state_manager_evicted_producers +Counts the total number of producers that have been evicted. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_cluster_producer_state_manager_producer_manager_total_active_producers +Reports the total number of active idempotent and transactional producers. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_cluster_shard_placement_assigned_partitions +Indicates the number of partitions assigned to a particular shard. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_cluster_shard_placement_hosted_partitions +Indicates the number of partitions hosted on a specific shard. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_cluster_shard_placement_partitions_to_reconcile +Counts the number of partitions that require reconciliation of shard-local state. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_data_transforms_log_manager_buffer_usage_ratio +Reports the buffer usage ratio for the data transforms log manager. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_data_transforms_log_manager_write_errors_total +Counts the total number of write errors encountered by the transform log manager. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_debug_bundle_failed_generation_count +Counts the total number of failed attempts to generate debug bundles. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_debug_bundle_last_failed_bundle_timestamp_seconds +Reports the timestamp (in seconds since the epoch) of the last failed debug bundle generation. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_debug_bundle_last_successful_bundle_timestamp_seconds +Reports the timestamp (in seconds since the epoch) of the last successful debug bundle generation. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_debug_bundle_successful_generation_count +Counts the total number of successfully generated debug bundles. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_host_diskstats_discards + +Reports the total number of discard operations initiated on the host disk. Discard operations free unused blocks and can affect disk longevity and performance. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_discards_merged + +Captures the number of discard operations that have been merged together, reducing overhead by consolidating multiple discard requests. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_discards_ms + +Measures the cumulative time in milliseconds spent on discard operations. This metric helps in understanding the performance impact of these operations on disk throughput. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_flushes + +Counts the number of flush operations performed by the host disk. Flushes ensure that buffered data is written to disk, which is important for data integrity. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_flushes_ms + +Measures the total time in milliseconds spent on flush operations. Monitoring flush duration can indicate disk performance under heavy I/O workloads. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_io_in_progress + +Shows the number of I/O operations currently in progress on the disk. This metric provides an insight into the active workload and concurrent operations. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_io_ms + +Captures the cumulative time in milliseconds taken to complete I/O operations. This metric helps in assessing the responsiveness of disk operations under load. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_io_weighted_ms + +Reports the weighted average time in milliseconds for I/O operations. Weighting factors in the duration and frequency of operations, offering a refined perspective on disk performance. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_reads + +Counts the number of read operations executed by the host disk. Tracking read operations is useful for understanding disk usage patterns. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_reads_merged + +Indicates the number of merged read operations, where multiple reads are consolidated to optimize performance. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_reads_ms + +Measures the total time in milliseconds spent on read operations. This metric is useful for evaluating the efficiency of read processes under varying load conditions. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_sectors_discarded + +Reports the number of disk sectors discarded during discard operations. Monitoring this value can help in understanding the effectiveness and impact of discard commands on disk space. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_sectors_read + +Indicates the total number of sectors read from the disk. This metric is useful for assessing disk throughput and read performance. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_sectors_written + +Captures the total number of disk sectors written. This metric helps in monitoring the disk's write activity and overall I/O load. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_writes + +Counts the number of write operations performed on the disk. Tracking this metric can be useful for analyzing write-intensive workloads. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_writes_merged + +Reflects the number of merged write operations, where individual writes are combined to optimize disk utilization. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_diskstats_writes_ms + +Measures the total time in milliseconds taken by write operations. Monitoring this duration helps in evaluating the efficiency of disk writes. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `disk` +- `shard` + +--- + +== vectorized_host_netstat_bytes_received + +Tracks the total number of bytes received on the host's network interface. This metric helps in monitoring incoming network traffic for performance and capacity planning. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_host_netstat_bytes_sent + +Reports the total number of bytes sent from the host's network interface. Monitoring outbound traffic can help in understanding network load and throughput. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_host_snmp_packets_received + +Indicates the number of SNMP packets received by the host. This metric is useful for tracking network management and monitoring activities. + + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_host_snmp_packets_sent + +Counts the SNMP packets sent from the host, providing insight into outgoing SNMP communication which is useful for network monitoring systems. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_host_snmp_tcp_established + +Measures the number of established TCP connections related to SNMP activity on the host. It aids in assessing the status and load of TCP-based SNMP communications. + +This metric is available only when the xref:reference:properties/cluster-properties.adoc#enable_host_metrics[`enable_host_metrics`] cluster property is set to `true`. + +*Type*: gauge + +*Labels*: + +- `shard` + + +== vectorized_httpd_connections_current +Indicates the current number of open HTTP connections. + +*Type*: gauge + +*Labels*: + +- `service` +- `shard` + +--- + +== vectorized_httpd_connections_total +Counts the total number of HTTP connections opened. + +*Type*: counter + +*Labels*: + +- `service` +- `shard` + +--- + +== vectorized_httpd_read_errors +Counts the total number of errors encountered while reading HTTP requests. + +*Type*: counter + +*Labels*: + +- `service` +- `shard` + +--- + +== vectorized_httpd_reply_errors +Counts the total number of errors encountered while sending HTTP responses. + +*Type*: counter + +*Labels*: + +- `service` +- `shard` + +--- + +== vectorized_httpd_requests_served +Counts the total number of HTTP requests served. + +*Type*: counter + +*Labels*: + +- `service` +- `shard` + +--- + +== vectorized_internal_rpc_active_connections +Shows the current number of active internal RPC connections. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_connection_close_errors +Counts the errors that occurred during the shutdown of internal RPC connections. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_connections_rejected +Counts the number of internal RPC connection attempts rejected due to open connection limits. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_connections_rejected_rate_limit +Counts the number of internal RPC connection attempts rejected due to rate limits. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_connections_wait_rate +Counts the number of internal RPC connections delayed due to rate limiting. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_connects +Counts the total number of accepted internal RPC connections. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_consumed_mem_bytes +Reports the memory consumed by internal RPC request processing. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_corrupted_headers +Counts the number of internal RPC requests with corrupted headers. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_dispatch_handler_latency +Provides a histogram of the latency experienced in internal RPC dispatch handlers. + +*Type*: histogram + +--- + +== vectorized_internal_rpc_latency +Reports the overall latency for internal RPC service requests. + +*Type*: histogram + +--- + +== vectorized_internal_rpc_max_service_mem_bytes +Reports the maximum memory allocated for internal RPC services. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_method_not_found_errors +Counts the number of internal RPC requests that referenced a non-existent method. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_received_bytes +Counts the total number of bytes received from internal RPC clients. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_requests_blocked_memory +Counts the number of internal RPC requests blocked due to memory backpressure. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_requests_completed +Counts the total number of successfully completed internal RPC requests. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_requests_pending +Reports the number of internal RPC requests currently pending. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_sent_bytes +Counts the total number of bytes sent in internal RPC responses. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_internal_rpc_service_errors +Counts the total number of service errors encountered by the internal RPC layer. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_io_queue_activations +Counts the number of times the IO queue was activated from an idle state. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` +- `stream` + +--- + +== vectorized_io_queue_adjusted_consumption +Reports the disk capacity units consumed by a class, adjusted for class shares and idling preemption. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` +- `stream` + +--- + +== vectorized_io_queue_consumption +Indicates the total disk capacity units consumed by a class. A per-second increment suggests full utilization. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` +- `stream` + +--- + +== vectorized_io_queue_delay +Measures the random delay time experienced in the IO queue. + +*Type*: gauge + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_disk_queue_length +Reports the current number of requests waiting in the disk queue. + +*Type*: gauge + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_flow_ratio +Indicates the ratio of the dispatch rate to the completion rate in the IO queue. Values greater than 1.0 may indicate stalls or disk issues. + +*Type*: gauge + +*Labels*: + +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_queue_length +Reports the current number of requests in the general IO queue. + +*Type*: gauge + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_shares +Indicates the current share allocation for the IO queue. + +*Type*: gauge + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_starvation_time_sec +Reports the total time (in seconds) the IO queue has experienced starvation. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_bytes +Reports the total number of bytes processed by the IO queue. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_delay_sec +Counts the total delay time (in seconds) experienced within the IO queue. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_exec_sec +Reports the total time (in seconds) spent executing IO operations. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_operations +Counts the total number of operations that have passed through the IO queue. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_read_bytes +Reports the total number of bytes read through the IO queue. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_read_ops +Counts the total number of read operations processed by the IO queue. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_split_bytes +Reports the total number of bytes that have been split by IO operations. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_split_ops +Counts the total number of IO requests that were split. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_write_bytes +Reports the total number of bytes written through the IO queue. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_io_queue_total_write_ops +Counts the total number of write operations processed by the IO queue. + +*Type*: counter + +*Labels*: + +- `class` +- `iogroup` +- `mountpoint` +- `shard` + +--- + +== vectorized_kafka_batch_size +Provides a histogram of the batch sizes across all topics at the Kafka layer. + +*Type*: histogram + +--- + +== vectorized_kafka_fetch_pid_delay_seconds_total +Reports the cumulative fetch delay (in seconds) set by the PID controller. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_fetch_pid_error_total +Reports the cumulative error in the fetch PID controller. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_fetch_read_distribution +Provides a histogram distribution of read path timings at the Kafka layer. + +*Type*: histogram + +--- + +== vectorized_kafka_fetch_sessions_cache_mem_usage_bytes +Reports the memory usage (in bytes) of the Kafka fetch sessions cache. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_fetch_sessions_cache_sessions_count +Indicates the total number of active Kafka fetch sessions. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_group_offset +Displays the offset for a given group, partition, and topic. + +*Type*: gauge + +*Labels*: + +- `group` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_kafka_handler_latency_microseconds +Provides a histogram of Kafka request latencies (in microseconds). + +*Type*: histogram + +--- + +== vectorized_kafka_handler_received_bytes_total +Counts the total number of bytes received by the Kafka request handler. + +*Type*: counter + +*Labels*: + +- `handler` +- `shard` + +--- + +== vectorized_kafka_handler_requests_completed_total +Counts the total number of successfully completed Kafka requests. + +*Type*: counter + +*Labels*: + +- `handler` +- `shard` + +--- + +== vectorized_kafka_handler_requests_errored_total +Counts the total number of Kafka requests that resulted in an error. + +*Type*: counter + +*Labels*: + +- `handler` +- `shard` + +--- + +== vectorized_kafka_handler_requests_in_progress_total +Reports the current number of Kafka requests in progress. + +*Type*: counter + +*Labels*: + +- `handler` +- `shard` + +--- + +== vectorized_kafka_handler_sent_bytes_total +Counts the total number of bytes sent in Kafka responses. + +*Type*: counter + +*Labels*: + +- `handler` +- `shard` + +--- + +== vectorized_kafka_latency_fetch_latency_us +Provides a histogram of fetch latencies (in microseconds) at the Kafka layer. + +*Type*: histogram + +--- + +== vectorized_kafka_latency_produce_latency_us +Provides a histogram of produce latencies (in microseconds) at the Kafka layer. + +*Type*: histogram + +--- + +== vectorized_kafka_produced_bytes +Counts the total number of bytes produced, segmented by compression type. + +*Type*: counter + +*Labels*: + +- `compression_type` +- `shard` + +--- + +== vectorized_kafka_quotas_client_quota_throttle_time +Provides a histogram of client quota throttling delays (in seconds) per rule and quota type based on xref:manage:cluster-maintenance/manage-throughput.adoc#client-throughput-limits[client throughput limits]. + +*Type*: histogram + +--- + +== vectorized_kafka_quotas_client_quota_throughput +Provides a histogram of client quota throughput per rule and quota type. + +*Type*: histogram + +--- + +== vectorized_kafka_quotas_quota_effective +Reports the currently effective quota in bytes per second. + +*Type*: counter + +*Labels*: + +- `direction` +- `shard` + +--- + +== vectorized_kafka_quotas_throttle_time +Provides a histogram of throttling times (in seconds) based on xref:manage:cluster-maintenance/manage-throughput.adoc#broker-wide-throughput-limits[broker-wide throughput limits] across the Kafka layer. + +*Type*: histogram + +--- + +== vectorized_kafka_quotas_traffic_egress +Counts the total amount of Kafka traffic (in bytes) sent to clients. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_quotas_traffic_intake +Counts the total amount of Kafka traffic (in bytes) received from clients. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_active_connections +Reports the number of active Kafka RPC connections. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_connection_close_errors +Counts the number of errors encountered while closing Kafka RPC connections. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_connections_rejected +Counts the number of Kafka RPC connection attempts rejected due to connection limits. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_connections_rejected_rate_limit +Counts the number of Kafka RPC connection attempts rejected due to rate limits. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_connections_wait_rate +Counts the number of Kafka RPC connections delayed due to rate limiting. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_connects +Counts the total number of accepted Kafka RPC connections. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_consumed_mem_bytes +Reports the memory (in bytes) consumed by Kafka RPC request processing. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_corrupted_headers +Counts the number of Kafka RPC requests with corrupted headers. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_dispatch_handler_latency +Provides a histogram of latencies (in seconds) in Kafka RPC dispatch handlers. + +*Type*: histogram + +--- + +== vectorized_kafka_rpc_fetch_avail_mem_bytes +Reports the available memory for processing Kafka RPC fetch requests. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_max_service_mem_bytes +Reports the maximum memory allocated for Kafka RPC services. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_method_not_found_errors +Counts the number of Kafka RPC requests for methods that do not exist. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_produce_bad_create_time +An incrementing counter for the number of times a producer created a message with a timestamp skewed from the broker's date and time. This metric is related to the following properties: + +- xref:reference:properties/cluster-properties.adoc#log_message_timestamp_alert_before_ms[`log_message_timestamp_alert_before_ms`]: Increment this gauge when the create_timestamp on a message is too far in the past as compared to the broker's time. + +- xref:reference:properties/cluster-properties.adoc#log_message_timestamp_alert_after_ms[`log_message_timestamp_alert_after_ms`]: Increment this gauge when the create_timestamp on a message is too far in the future as compared to the broker's time. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_received_bytes +Counts the total number of bytes received by the Kafka RPC layer. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_requests_blocked_memory +Counts the number of Kafka RPC requests blocked due to insufficient memory. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_requests_completed +Counts the total number of successfully completed Kafka RPC requests. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_requests_pending +Reports the number of Kafka RPC requests that are currently pending. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_sasl_session_expiration_total +Counts the total number of SASL session expirations in Kafka RPC. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_sasl_session_reauth_attempts_total +Counts the total number of SASL reauthentication attempts in Kafka RPC. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_sasl_session_revoked_total +Counts the total number of revoked SASL sessions in Kafka RPC. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_sent_bytes +Counts the total number of bytes sent by the Kafka RPC layer. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_rpc_service_errors +Counts the total number of service errors encountered in Kafka RPC. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_schema_id_cache_batches_decompressed +Counts the number of batches decompressed for server-side schema ID validation. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_schema_id_cache_hits +Counts the number of cache hits in the server-side schema ID validation cache. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_kafka_schema_id_cache_misses +Counts the number of cache misses in the server-side schema ID validation cache. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_leader_balancer_leader_transfer_error +Counts the number of errors encountered during leader transfer attempts. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_leader_balancer_leader_transfer_no_improvement +Counts the number of leader transfer attempts that did not result in improved balance. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_leader_balancer_leader_transfer_succeeded +Counts the total number of successful leader transfers. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_leader_balancer_leader_transfer_timeout +Counts the number of leader transfer attempts that timed out. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_allocated_memory +Reports the total amount of memory allocated (in bytes) by the process. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_available_memory +Reports the total amount of potentially available memory (free plus reclaimable) per shard. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_available_memory_low_water_mark +Reports the low-water mark for available memory since process start. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_cross_cpu_free_operations +Counts the number of cross-CPU free operations executed. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_free_memory +Reports the current free memory (in bytes) per shard. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_free_operations +Counts the total number of free operations performed. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_malloc_failed +Counts the total number of failed memory allocation attempts. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_malloc_live_objects +Reports the current number of live objects allocated via malloc. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_malloc_operations +Counts the total number of malloc operations performed. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_reclaims_operations +Counts the total number of memory reclaim operations performed. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_memory_total_memory +Reports the total memory size (in bytes) available per shard. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_network_bytes_received +Counts the total number of bytes received over network sockets. + +*Type*: counter + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_network_bytes_sent +Counts the total number of bytes sent over network sockets. + +*Type*: counter + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_node_status_rpcs_received +Reports the number of broker status RPCs received by this broker. + +*Type*: gauge + +--- + +== vectorized_node_status_rpcs_sent +Reports the number of broker status RPCs sent by this broker. + +*Type*: gauge + +--- + +== vectorized_node_status_rpcs_timed_out +Reports the number of broker status RPCs that have timed out. + +*Type*: gauge + +--- + +== vectorized_ntp_archiver_compacted_replaced_bytes + +Number of bytes removed from cloud storage by compaction operations. This is tracked on a per-topic and per-partition basis. + +This metric resets every time partition leadership changes. It tracks whether or not compaction is performing operations on cloud storage. + +The `namespace` label supports the following options for this metric: + +* `kafka` - User topics +* `kafka_internal` - Internal Kafka topic, such as consumer groups +* `redpanda` - Redpanda-only internal data + +*Labels*: + +* `namespace=("kafka" | "kafka_internal" | "redpanda")` +* `topic` +* `partition` + +--- + +== vectorized_ntp_archiver_pending + +The difference between the last committed offset and the last offset uploaded to Tiered Storage for each partition. A value of zero for this metric indicates that all data for a partition is uploaded to Tiered Storage. + +This metric is impacted by the xref:reference:tunable-properties.adoc#cloud_storage_segment_max_upload_interval_sec[`cloud_storage_segment_max_upload_interval_sec`] tunable property. If this interval is set to 5 minutes, the archiver will upload committed segments to Tiered Storage every 5 minutes or less. If this metric continues growing for longer than the configured interval, it can indicate a potential network issue with the upload path for that partition. + +The `namespace` label supports the following options for this metric: + +* `kafka` - User topics +* `kafka_internal` - Internal Kafka topic, such as consumer groups +* `redpanda` - Redpanda-only internal data + +*Labels*: + +* `namespace=("kafka" | "kafka_internal" | "redpanda")` +* `topic` +* `partition` + +--- + +== vectorized_pandaproxy_request_errors_total +Counts the total number of REST proxy server errors, segmented by operation and status. + +*Type*: counter + +*Labels*: + +- `operation` +- `shard` +- `status` + +--- + +== vectorized_pandaproxy_request_latency +Provides a histogram of REST proxy request latencies. + +*Type*: histogram + +--- + +== vectorized_raft_append_entries_buffer_flushes +Counts the number of flushes performed on the append entries buffer. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_buffered_protocol_buffered_bytes +Reports the total size (in bytes) of append entries requests currently buffered. + +*Type*: gauge + +*Labels*: + +- `shard` +- `target_node_id` + +--- + +== vectorized_raft_buffered_protocol_buffered_requests +Counts the number of append entries requests currently buffered. + +*Type*: gauge + +*Labels*: + +- `shard` +- `target_node_id` + +--- + +== vectorized_raft_buffered_protocol_inflight_requests +Counts the number of append entries requests sent to a target broker that are awaiting responses. + +*Type*: gauge + +*Labels*: + +- `shard` +- `target_node_id` + +--- + +== vectorized_raft_configuration_change_in_progress +Indicates if the current raft group configuration is in a joint state (i.e. undergoing change). + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_done_replicate_requests +Counts the total number of replicate requests that have completed. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_full_heartbeat_requests +Counts the number of full heartbeat messages sent by the raft leader. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_group_configuration_updates +Counts the total number of raft group configuration updates. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_group_count +Reports the total number of raft groups present. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_raft_heartbeat_requests_errors +Counts the number of failed heartbeat requests within a raft group. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_leader_for +Indicates the number of raft groups for which this broker is the leader. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_leadership_changes +Counts the number of leader elections won by this broker. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_learners_gap_bytes +Reports the total number of bytes that must be sent to learners to catch up. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_raft_lightweight_heartbeat_requests +Counts the number of lightweight heartbeat messages sent by the raft leader. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_log_flushes +Counts the number of log flush operations performed. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_log_truncations +Counts the number of log truncations performed. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_offset_translator_inconsistency_errors +Counts the number of append entries requests that failed the offset translator consistency check. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_received_append_requests +Counts the total number of append requests received. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_received_vote_requests +Counts the total number of vote requests received. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_recovery_offsets_pending +Reports the sum of offsets that need recovery on this broker. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_raft_recovery_partition_movement_assigned_bandwidth +Reports the bandwidth (bytes/sec) assigned for partition movement in the last tick. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_raft_recovery_partition_movement_available_bandwidth +Reports the available bandwidth (bytes/sec) for partition movement. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_raft_recovery_partitions_active +Counts the number of partition replicas currently in recovery on this broker. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_raft_recovery_partitions_to_recover +Counts the number of partition replicas that still require recovery on this broker. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_raft_recovery_requests +Counts the total number of recovery requests issued. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_recovery_requests_errors +Counts the total number of failed recovery requests. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_replicate_ack_all_requests +Counts replicate requests that required quorum acknowledgments with an explicit flush. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_replicate_ack_all_requests_no_flush +Counts replicate requests that required quorum acknowledgments but did not use an explicit flush. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_replicate_ack_leader_requests +Counts replicate requests that acknowledged only the leader. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_replicate_ack_none_requests +Counts replicate requests that did not require any acknowledgment. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_replicate_batch_flush_requests +Counts the number of batch flush operations in replicate requests. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_replicate_request_errors +Counts the number of failed replicate requests. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_raft_sent_vote_requests +Counts the total number of vote requests sent. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_reactor_abandoned_failed_futures +Counts the total number of futures that were abandoned while still containing an exception. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_aio_bytes_read +Counts the total number of bytes read via asynchronous IO. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_aio_bytes_write +Counts the total number of bytes written via asynchronous IO. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_aio_errors +Counts the total number of asynchronous IO errors. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_aio_outsizes +Counts the total number of AIO operations that exceeded configured IO limits. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_aio_reads +Counts the total number of AIO read operations. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_aio_writes +Counts the total number of AIO write operations. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_awake_time_ms_total +Reports the total wall-clock time (in milliseconds) the reactor was awake. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_cpp_exceptions +Counts the total number of C++ exceptions thrown by the reactor. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_cpu_busy_ms +Reports the total CPU busy time (in milliseconds) for the reactor thread. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_cpu_steal_time_ms +Reports the total amount of time (in milliseconds) the reactor was runnable but was not scheduled (steal time). + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_cpu_used_time_ms +Reports the total CPU time used by the reactor thread (from CLOCK_THREAD_CPUTIME). + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_fstream_read_bytes +Counts the total number of bytes read from disk file streams. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_fstream_read_bytes_blocked +Counts the number of bytes read from disk that had to block due to insufficient read-ahead buffers. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_fstream_reads +Counts the total number of read operations from disk file streams. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_fstream_reads_ahead_bytes_discarded +Counts the number of buffered bytes that were discarded after being read ahead but not used. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_fstream_reads_aheads_discarded +Counts the number of times that pre-read buffers were discarded due to being unnecessary. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_fstream_reads_blocked +Counts the number of times a disk read had to block because pre-read buffers were insufficient. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_fsyncs +Counts the total number of fsync operations performed by the reactor. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_io_threaded_fallbacks +Counts the number of times the reactor fell back to an IO-threaded path. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_logging_failures +Counts the total number of logging failures encountered by the reactor. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_polls +Counts the total number of poll cycles executed by the reactor. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_sleep_time_ms_total +Reports the total wall-clock sleep time (in milliseconds) of the reactor. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_stalls +Provides a histogram of durations for which the reactor experienced stalls. + +*Type*: histogram + +--- + +== vectorized_reactor_tasks_pending +Reports the current number of pending tasks in the reactor's task queue. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_tasks_processed +Counts the total number of tasks processed by the reactor. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_timers_pending +Reports the current number of pending timer tasks in the reactor. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_reactor_utilization +Indicates the CPU utilization of the reactor thread. + +Shows the true utilization of the CPU by a Redpanda process. This metric has per-broker and per-shard granularity. If a shard (CPU core) is at 100% utilization for a continuous period of real-time processing, for example more than a few seconds, you will likely observe high latency for partitions assigned to that shard. Use topic-aware xref:manage:cluster-maintenance/cluster-balancing.adoc#intra-broker-partition-balancing[intra-broker partition balancing] to balance partition assignments and alleviate load on individual shards. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_rest_proxy_inflight_requests_memory_usage_ratio +Reports the memory usage ratio for in-flight REST proxy requests. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_rest_proxy_inflight_requests_usage_ratio +Reports the overall usage ratio of in-flight REST proxy requests. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_rest_proxy_queued_requests_memory_blocked +Reports the number of REST proxy requests that are queued due to memory limitations. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_rpc_client_active_connections +Indicates the number of currently active RPC client connections. + +*Type*: gauge + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_client_correlation_errors +Counts errors in client correlation IDs within RPC responses. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_connection_errors +Counts the number of connection errors encountered by the RPC client. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_connects +Counts the total number of RPC client connection attempts. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_corrupted_headers +Counts the number of RPC responses received with corrupted headers. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_in_bytes +Reports the total number of bytes received by the RPC client. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_out_bytes +Reports the total number of bytes sent (including headers) by the RPC client. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_read_dispatch_errors +Counts the number of errors encountered while dispatching RPC responses. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_request_errors +Counts the total number of request errors encountered by the RPC client. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_request_timeouts +Counts the number of RPC client request timeouts. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_requests +Counts the total number of RPC client requests made. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_requests_blocked_memory +Counts the number of RPC client requests that were blocked due to insufficient memory. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_requests_pending +Reports the current number of pending RPC client requests. + +*Type*: gauge + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_rpc_client_server_correlation_errors +Counts the number of RPC responses with mismatched server correlation IDs. + +*Type*: counter + +*Labels*: + +- `connection_cache_label` +- `shard` +- `target` + +--- + +== vectorized_scheduler_queue_length +Reports the current size of the scheduler queue (in tasks), indicating backlog. + +*Type*: gauge + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_scheduler_runtime_ms +Reports the accumulated runtime (in milliseconds) of a scheduler queue. A rate of 1000ms per second suggests full utilization. + +*Type*: counter + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_scheduler_shares +Indicates the share allocation for a scheduler queue. + +*Type*: gauge + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_scheduler_starvetime_ms +Reports the accumulated starvation time (in milliseconds) for a scheduler queue. + +*Type*: counter + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_scheduler_tasks_processed +Counts the total number of tasks processed by the scheduler. + +*Type*: counter + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_scheduler_time_spent_on_task_quota_violations_ms +Reports the total time (in milliseconds) that the scheduler has violated task quotas. + +*Type*: counter + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_scheduler_waittime_ms +Reports the accumulated wait time (in milliseconds) for a scheduler queue. + +*Type*: counter + +*Labels*: + +- `group` +- `shard` + +--- + +== vectorized_schema_registry_cache_schema_count +Reports the total number of schemas stored in the schema registry cache. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_schema_registry_cache_schema_memory_bytes +Reports the memory usage (in bytes) of schemas stored in the registry. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_schema_registry_cache_subject_count +Reports the total number of subjects in the schema registry. + +*Type*: gauge + +*Labels*: + +- `deleted` +- `shard` + +--- + +== vectorized_schema_registry_cache_subject_version_count +Reports the number of versions for a given subject in the schema registry. + +*Type*: gauge + +*Labels*: + +- `deleted` +- `shard` +- `subject` + +--- + +== vectorized_schema_registry_inflight_requests_memory_usage_ratio +Reports the memory usage ratio for in-flight schema registry requests. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_schema_registry_inflight_requests_usage_ratio +Reports the usage ratio for in-flight schema registry requests. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_schema_registry_queued_requests_memory_blocked +Reports the number of schema registry requests queued due to memory constraints. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_security_audit_buffer_usage_ratio +Reports the usage ratio of the audit event buffer. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_security_audit_client_buffer_usage_ratio +Reports the usage ratio of the audit client's send buffer. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_security_audit_errors_total +Counts the total number of errors encountered while creating or publishing audit log entries. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_security_audit_last_event_timestamp_seconds +Reports the timestamp (in seconds since the epoch) of the last successful audit log event. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_available_reclaimable_bytes +Reports the total amount of reclaimable data available under space management. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_disk_usage_bytes +Reports the total disk usage (in bytes) managed by space management. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_local_retention_reclaimable_bytes +Reports the amount of data above the local retention target that is reclaimable. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_reclaim_active_segment_bytes +Estimates the amount of data above the active segment that can be reclaimed. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_reclaim_estimate_bytes +Provides an estimate of the data (in bytes) to be reclaimed in the last space management schedule. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_reclaim_local_bytes +Estimates the amount of data above local retention that will be reclaimed. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_reclaim_low_hinted_bytes +Estimates the amount of data above the low-space threshold (with hints) to be reclaimed. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_reclaim_low_non_hinted_bytes +Estimates the amount of data above the low-space threshold (without hints) to be reclaimed. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_retention_reclaimable_bytes +Reports the total reclaimable data as per the standard retention policy. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_target_disk_size_bytes +Specifies the target maximum disk usage (in bytes) for space management. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_space_management_target_excess_bytes +Reports the amount of disk usage exceeding the target threshold. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_stall_detector_reported +Counts the total number of stalls reported (see detailed traces for context). + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_compaction_backlog_controller_backlog_size +Reports the current backlog size for the storage compaction controller. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_compaction_backlog_controller_error +Reports the error (difference between target and current backlog) in the compaction controller. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_compaction_backlog_controller_shares +Reports the number of shares output by the storage compaction controller. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_kvstore_cached_bytes +Reports the size (in bytes) of the in-memory key-value store cache. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_kvstore_entries_fetched +Counts the number of key-value entries fetched from the store. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_kvstore_entries_removed +Counts the number of key-value entries removed from the store. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_kvstore_entries_written +Counts the number of key-value entries written to the store. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_kvstore_key_count +Reports the total number of keys currently stored in the key-value store. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_kvstore_segments_rolled +Counts the number of segments that have been rolled in the key-value store. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_log_batch_parse_errors +Counts the number of errors encountered while parsing log batches. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_batch_write_errors +Counts the number of errors encountered while writing log batches. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_batches_read +Counts the total number of log batches read. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_batches_written +Counts the total number of log batches written. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_bytes_prefix_truncated +Counts the number of bytes removed due to prefix truncation of log segments. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_cache_hits +Counts the number of cache hits in the log reader cache. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_cache_misses +Counts the number of cache misses in the log reader cache. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_cached_batches_read +Counts the total number of log batches read from cache. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_cached_read_bytes +Counts the total number of bytes read from the log cache. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_chunked_compaction_runs +Counts the number of times chunked compaction has been executed, which can also indicate failures in building key–offset maps. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_cleanly_compacted_segment +Counts the number of segments that were successfully compacted with proper deduplication. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_closed_segment_bytes +Reports the total number of bytes contained in closed log segments. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_compacted_segment +Counts the total number of compacted log segments. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_compaction_ratio +Reports the average compaction ratio for log segments. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `topic` + +--- + +== vectorized_storage_log_compaction_removed_bytes +Counts the total number of bytes removed from local storage by compaction operations. This is tracked on a per-topic and per-partition basis. It tracks whether compaction is performing operations on local storage. + +The namespace label supports the following options for this metric: + +- `kafka`: User topics + +- `kafka_internal`: Internal Kafka topic, such as consumer groups + +- `redpanda`: Redpanda-only internal data + +*Type*: counter + +*Labels*: + +- `namespace=("kafka" | "kafka_internal" | "redpanda")` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_complete_sliding_window_rounds +Counts the number of complete rounds executed in sliding window compaction. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_corrupted_compaction_indices +Counts the number of times the compaction index had to be reconstructed due to corruption. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_dirty_segment_bytes +Reports the total number of bytes contained in dirty (non-compacted) log segments. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_log_segments_active +Counts the current number of active local log segments. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_log_segments_created +Counts the total number of local log segments created since broker startup. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_log_segments_removed +Counts the total number of local log segments removed since broker startup. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_partition_size +Reports the current size (in bytes) of a partition's log. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_read_bytes +Counts the total number of bytes read from log segments. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_readers_added +Counts the total number of log readers that have been added to the cache. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_readers_evicted +Counts the total number of log readers evicted from the cache. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_segments_marked_tombstone_free +Counts the number of log segments verified to be free of tombstones via compaction. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_tombstones_removed +Counts the number of tombstone records removed during compaction (per retention policy). + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_log_written_bytes +Counts the total number of bytes written to log segments. + +*Type*: counter + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_storage_manager_housekeeping_log_processed +Counts the number of logs processed by the storage housekeeping routine. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_manager_logs +Reports the number of logs currently managed by the storage manager. + +*Type*: gauge + +*Labels*: + +- `shard` + +--- + +== vectorized_storage_manager_urgent_gc_runs +Counts the number of urgent garbage collection runs executed by the storage manager. + +*Type*: counter + +*Labels*: + +- `shard` + +--- + +== vectorized_tx_partition_idempotency_pid_cache_size +Reports the number of active producer ID–sequence pairs used for idempotency in transactions. + +*Type*: gauge + +*Labels*: + +- `namespace` +- `partition` +- `shard` +- `topic` + +--- + +== vectorized_tx_partition_tx_num_inflight_requests +Reports the number of ongoing transactional requests for a partition. + +*Type*: gauge + +*Labels*: -* xref:manage:monitoring.adoc[Learn how to monitor Redpanda] -* xref:./public-metrics-reference.adoc[Public metrics reference] +- `namespace` +- `partition` +- `shard` +- `topic` diff --git a/modules/reference/pages/properties/cluster-properties.adoc b/modules/reference/pages/properties/cluster-properties.adoc index b31c95973..0bead09dc 100644 --- a/modules/reference/pages/properties/cluster-properties.adoc +++ b/modules/reference/pages/properties/cluster-properties.adoc @@ -1180,6 +1180,20 @@ Enable idempotent producers. --- +=== enable_host_metrics + +Enable exporting of some host metrics like `/proc/diskstats`, `/proc/snmp` and `/proc/net/netstat`. + +*Requires restart:* Yes + +*Visibility:* `tunable` + +*Type:* boolean + +*Default:* `false` + +--- + === enable_leader_balancer Enable automatic leadership rebalancing. @@ -1902,7 +1916,7 @@ Iceberg REST catalog user ID. This ID is used to query the catalog API for the O === iceberg_rest_catalog_client_secret -Secret to authenticate against Iceberg REST catalog. Required if catalog type is set to `rest`. +Secret to authenticate against Iceberg REST catalog. Required if catalog type is set to `rest` and `iceberg_rest_catalog_authentication_mode` is set to `oauth2`. *Requires restart:* Yes @@ -1920,6 +1934,20 @@ Secret to authenticate against Iceberg REST catalog. Required if catalog type is --- +=== iceberg_rest_catalog_crl + +The contents of a certificate revocation list for `iceberg_rest_catalog_trust`. Takes precedence over `iceberg_rest_catalog_crl_file`. + +*Requires restart:* Yes + +*Visibility:* `user` + +*Type:* string + +*Default:* `null` + +--- + === iceberg_rest_catalog_crl_file Path to certificate revocation list for `iceberg_rest_catalog_trust_file`. @@ -2028,6 +2056,8 @@ Maximum length of time that Redpanda waits for a response from the REST catalog Token used to access the REST Iceberg catalog. If the token is present, Redpanda ignores credentials stored in the properties <> and <>. +Required if <> is set to `bearer`. + *Requires restart:* Yes *Visibility:* `user` @@ -2044,6 +2074,20 @@ Token used to access the REST Iceberg catalog. If the token is present, Redpanda --- +=== iceberg_rest_catalog_trust + +The contents of a certificate chain to trust for the REST Iceberg catalog. Takes precedence over <>. + +*Requires restart:* Yes + +*Visibility:* `user` + +*Type:* string + +*Default:* `null` + +--- + === iceberg_rest_catalog_trust_file Path to a file containing a certificate chain to trust for the REST Iceberg catalog. @@ -3974,7 +4018,7 @@ Quota manager GC frequency in milliseconds. *Unit:* seconds -*Requires restart:* Yes +*Requires restart:* No *Visibility:* `tunable` @@ -5280,6 +5324,32 @@ List of superuser usernames. --- +=== tls_certificate_name_format + +The format of the certificates's distinguished name to use for mTLS principal mapping. The `legacy` format would appear as 'C=US,ST=California,L=San Francisco,O=Redpanda,CN=redpanda', while the `rfc2253` format would appear as 'CN=redpanda,O=Redpanda,L=San Francisco,ST=California,C=US'. + +*Requires restart:* No + +*Visibility:* `user` + +*Default:* `legacy` + +--- + +=== tls_enable_renegotiation + +TLS client-initiated renegotiation is considered unsafe and is disabled by default . Only re-enable it if you are experiencing issues with your TLS-enabled client. This option has no effect on TLSv1.3 connections as client-initiated renegotiation was removed. + +*Requires restart:* Yes + +*Visibility:* `tunable` + +*Type:* boolean + +*Default:* `false` + +--- + === tls_min_version The minimum TLS version that Redpanda clusters support. This property prevents client applications from negotiating a downgrade to the TLS version when they make a connection to a Redpanda cluster. diff --git a/modules/reference/pages/public-metrics-reference.adoc b/modules/reference/pages/public-metrics-reference.adoc index bff50b08c..4876be27a 100644 --- a/modules/reference/pages/public-metrics-reference.adoc +++ b/modules/reference/pages/public-metrics-reference.adoc @@ -337,6 +337,24 @@ Current number of active RPC client connections on a shard. --- +=== redpanda_rpc_received_bytes + +Number of bytes received from the clients in valid requests. + +The `redpanda_server` label supports the following options for this metric: + +- `kafka`: Data sent over the Kafka API + +- `internal`: Inter-broker traffic + +*Type*: counter + +*Labels*: + +- `redpanda_server` + +--- + === redpanda_rpc_request_errors_total Cumulative count of RPC errors encountered, segmented by server type. @@ -363,6 +381,24 @@ Histogram capturing the latency (in seconds) for RPC requests. --- +=== redpanda_rpc_sent_bytes + +Number of bytes sent to clients. + +The `redpanda_server` label supports the following options for this metric: + +- `kafka`: Data sent over the Kafka API + +- `internal`: Inter-broker traffic + +*Type*: counter + +*Labels*: + +- `redpanda_server` + +--- + === redpanda_scheduler_runtime_seconds_total Total accumulated runtime (in seconds) for the task queue associated with each scheduling group per shard. @@ -623,6 +659,23 @@ High watermark offset for a partition, used to calculate consumer group lag. --- +=== redpanda_kafka_request_bytes_total + +Total number of bytes read from or written to the partitions of a topic. +The total may include fetched bytes that are not returned to the client. + +*Type*: counter + +*Labels*: + +* `redpanda_namespace` + +* `redpanda_topic` + +* `redpanda_request=("produce" | "consume")` + +--- + === redpanda_kafka_under_replicated_replicas Number of partition replicas that are live yet lag behind the latest offset, <>. @@ -765,22 +818,6 @@ Configured number of replicas for a topic. --- -=== redpanda_kafka_request_bytes_total - -Total number of bytes processed (produced or consumed) for a topic. - -*Type*: counter - -*Labels*: - -* `redpanda_namespace` - -* `redpanda_topic` - -* `redpanda_request=("produce" | "consume")` - ---- - === redpanda_raft_leadership_changes Total number of leadership changes across partitions for a given topic. diff --git a/tests/docker-compose/bootstrap.yml b/tests/docker-compose/bootstrap.yml index 98ab6f6a5..00f3003bf 100644 --- a/tests/docker-compose/bootstrap.yml +++ b/tests/docker-compose/bootstrap.yml @@ -50,4 +50,7 @@ enable_consumer_group_metrics: - "consumer_lag" # Lower the interval for the quickstart # https://docs.redpanda.com/current/reference/properties/cluster-properties/#consumer_group_lag_collection_interval_sec -consumer_group_lag_collection_interval_sec: 60 \ No newline at end of file +consumer_group_lag_collection_interval_sec: 60 +# Enable Redpanda to collect host metrics. +# https://docs.redpanda.com/current/reference/properties/cluster-properties/#enable_host_metrics +enable_host_metrics: true \ No newline at end of file