Docs Self-Managed Reference Monitoring Metrics Public Metrics You are viewing the Self-Managed v25.1 beta documentation. We welcome your feedback at the Redpanda Community Slack #beta-feedback channel. To view the latest available version of the docs, see v24.3. Public Metrics This section provides reference descriptions for the public metrics exported from Redpanda’s /public_metrics endpoint. In a live system, Redpanda metrics are exported only for features that are in use. For example, Redpanda does not export metrics for consumer groups if no groups are registered. To see the available public metrics in your system, query the /public_metrics endpoint: curl http://<node-addr>:9644/public_metrics | grep "[HELP|TYPE]" Use /public_metrics for your primary dashboards for monitoring system health. Use /metrics for detailed analysis and debugging. Cluster metrics redpanda_cluster_brokers Total number of fully commissioned brokers configured in the cluster. Type: gauge Usage: Create an alert if this gauge falls below a steady-state threshold, which may indicate that a broker has become unresponsive. redpanda_cluster_controller_log_limit_requests_available_rps The upper limit on the requests per second (RPS) that the cluster controller log is allowed to process, segmented by command group. Type: gauge Labels: redpanda_cmd_group=("move_operations" | "topic_operations" | "configuration_operations" | "node_management_operations" | "acls_and_users_operations") redpanda_cluster_controller_log_limit_requests_dropped The cumulative number of requests dropped by the controller log because the incoming rate exceeded the available RPS limit. Type: counter Labels: redpanda_cmd_group=("move_operations" | "topic_operations" | "configuration_operations" | "node_management_operations" | "acls_and_users_operations") Usage: A rising counter indicates that requests are being dropped, which could signal overload or misconfiguration. redpanda_cluster_features_enterprise_license_expiry_sec Number of seconds remaining until the Enterprise Edition license expires. Type: gauge Usage: A value of -1 indicates that no license is present. A value of 0 signifies an expired license. Use this metric to proactively monitor license status and trigger alerts for timely renewal. redpanda_cluster_latest_cluster_metadata_manifest_age The elapsed time in seconds since the most recent cluster metadata manifest was uploaded to Tiered Storage. Type: gauge Usage: A value of 0 means that metadata has not yet been uploaded. On a healthy system, this age should not exceed the configured value for cloud_storage_cluster_metadata_upload_interval_ms. Consider setting an alert if the age remains 0 for longer than 1.5x the configured interval, as this may indicate a configuration issue. redpanda_cluster_members_backend_queued_node_operations The number of node operations queued per shard that are awaiting processing by the backend. Type: gauge Labels: shard redpanda_cluster_members_backend_queued_node_operations Number of queued node operations. Type: gauge Labels: shard redpanda_cluster_non_homogenous_fips_mode Count of brokers whose FIPS mode configuration differs from the rest of the cluster. Type: gauge Usage: Indicates inconsistencies in security configurations that might affect compliance or interoperability. redpanda_cluster_partition_moving_from_node Number of partition replicas that are in the process of being removed from a broker. Type: gauge Usage: A non-zero value can indicate ongoing or unexpected partition reassignments. Investigate if this metric remains elevated. redpanda_cluster_partition_moving_to_node Number of partition replicas in the cluster currently being added or moved to a broker. Type: gauge Usage: When this gauge is non-zero, determine whether there is an expected or unexpected reassignment of partitions causing partition replicas movement. redpanda_cluster_partition_node_cancelling_movements During a partition movement cancellation operation, the number of partition replicas that were being moved that now need to be canceled. Type: gauge Usage: Track this metric to verify that partition reassignments are proceeding as expected; persistent non-zero values may warrant further investigation. redpanda_cluster_partition_num_with_broken_rack_constraint During a partition movement cancellation operation, the number of partition replicas that were scheduled for movement but now require cancellation. Type: gauge Usage: A non-zero value may indicate issues in the partition reassignment process that need attention. redpanda_cluster_partitions Total number of logical partitions managed by the cluster. This includes partitions for the controller topic but excludes replicas. Type: gauge redpanda_cluster_topics The total number of topics configured within the cluster. Type: gauge redpanda_cluster_unavailable_partitions Number of partitions that are unavailable due to a lack of quorum among their replica set. Type: gauge Usage: A non-zero value indicates that some partitions do not have an active leader. Consider increasing the number of brokers or the replication factor if this persists. redpanda_cluster_latest_cluster_metadata_manifest_age The amount of time in seconds since the last time Redpanda uploaded metadata files to Tiered Storage for your cluster. A value of 0 indicates metadata has not yet been uploaded. When performing a whole cluster restore operation, the metadata for the new cluster will not have any changes made to a source cluster that is newer than this age. Type: gauge Usage: On a healthy system, this should not exceed the value set for cloud_storage_cluster_metadata_upload_interval_ms. You may consider setting an alert if this remains 0 for longer than 1.5 * cloud_storage_cluster_metadata_upload_interval_ms as that may indicate a configuration issue. Related topics: Whole Cluster Restore for Disaster Recovery Debug bundle metrics redpanda_debug_bundle_failed_generation_count Running count of debug bundle generation failures, reported per shard. Type: counter Labels: shard redpanda_debug_bundle_last_failed_bundle_timestamp_seconds Unix epoch timestamp of the last failed debug bundle generation, per shard. Type: gauge Labels: shard redpanda_debug_bundle_last_successful_bundle_timestamp_seconds Unix epoch timestamp of the last successfully generated debug bundle, per shard. Type: gauge Labels: shard redpanda_debug_bundle_successful_generation_count Running count of successfully generated debug bundles, reported per shard. Type: counter Labels: shard Infrastructure metrics redpanda_cpu_busy_seconds_total Total time (in seconds) the CPU has been actively processing tasks. Type: counter Usage: Useful for tracking overall CPU utilization. redpanda_io_queue_total_read_ops Cumulative count of read operations processed by the I/O queue. Type: counter Labels: class=("default" | "compaction" | "raft") ioshard mountpoint shard redpanda_io_queue_total_write_ops Cumulative count of write operations processed by the I/O queue. Type: counter Labels: class=("default" | "compaction" | "raft") ioshard mountpoint shard redpanda_memory_allocated_memory Total memory allocated (in bytes) per CPU shard. Type: gauge Labels: shard redpanda_memory_available_memory Total memory (in bytes) available to a CPU shard—including both free and reclaimable memory. Type: gauge Labels: shard Usage: Indicates memory pressure on each shard. redpanda_memory_available_memory_low_water_mark The lowest recorded available memory (in bytes) per CPU shard since the process started. Type: gauge Labels: shard redpanda_memory_free_memory Total unallocated (free) memory in bytes available for each CPU shard. Type: gauge Labels: shard redpanda_rpc_active_connections Current number of active RPC client connections on a shard. Type: gauge Labels: redpanda_server=("kafka" | "internal") redpanda_rpc_request_errors_total Cumulative count of RPC errors encountered, segmented by server type. Type: counter Labels: redpanda_server=("kafka" | "internal") Usage: Use this metric to diagnose potential issues in RPC communication. redpanda_rpc_request_latency_seconds Histogram capturing the latency (in seconds) for RPC requests. Type: histogram Labels: redpanda_server=("kafka" | "internal") redpanda_scheduler_runtime_seconds_total Total accumulated runtime (in seconds) for the task queue associated with each scheduling group per shard. Type: counter Labels: redpanda_scheduling_group=("admin" | "archival_upload" | "cache_background_reclaim" | "cluster" | "coproc" | "kafka" | "log_compaction" | "main" | "node_status" | "raft" | "raft_learner_recovery") shard redpanda_storage_cache_disk_free_bytes Amount of free disk space (in bytes) available on the cache storage. Type: gauge Usage: Monitor this to ensure sufficient cache storage capacity. redpanda_storage_cache_disk_free_space_alert Alert indicator for cache storage free space, where: 0 = OK 1 = Low space 2 = Degraded Type: gauge redpanda_storage_disk_free_bytes Amount of free disk space (in bytes) available on attached storage. Type: gauge redpanda_storage_disk_free_space_alert Alert indicator for overall disk storage free space, where: 0 = OK 1 = Low space 2 = Degraded Type: gauge redpanda_storage_disk_total_bytes Total capacity (in bytes) of the attached storage. Type: gauge redpanda_uptime_seconds_total Total system uptime (in seconds) representing the overall CPU runtime. Type: gauge Raft metrics redpanda_node_status_rpcs_received Total count of node status RPCs received by a broker. Type: gauge redpanda_node_status_rpcs_sent Total count of node status RPCs sent by a broker. Type: gauge redpanda_node_status_rpcs_timed_out Total count of node status RPCs that timed out on a broker. Type: gauge Service metrics redpanda_authorization_result Cumulative count of authorization results, categorized by result type. Type: counter Labels: type redpanda_kafka_rpc_sasl_session_expiration_total Total number of SASL session expirations observed. Type: counter redpanda_kafka_rpc_sasl_session_reauth_attempts_total Total number of SASL reauthentication attempts made by clients. Type: counter redpanda_kafka_rpc_sasl_session_revoked_total Total number of SASL sessions that have been revoked. Type: counter redpanda_rest_proxy_request_latency_seconds Histogram capturing the latency (in seconds) for REST proxy requests. The measurement includes waiting for resource availability, processing, and response dispatch. Type: histogram redpanda_schema_registry_cache_schema_count Total number of schemas currently stored in the Schema Registry cache. Type: gauge redpanda_schema_registry_cache_schema_memory_bytes Memory usage (in bytes) by schemas stored in the Schema Registry cache. Type: gauge redpanda_schema_registry_cache_subject_count Count of subjects stored in the Schema Registry cache. Type: gauge Labels: deleted redpanda_schema_registry_cache_subject_version_count Count of versions available for each subject in the Schema Registry cache. Type: gauge Labels: deleted subject redpanda_schema_registry_inflight_requests_memory_usage_ratio Ratio of memory used by in-flight requests in the Schema Registry, reported per shard. Type: gauge Labels: shard redpanda_schema_registry_inflight_requests_usage_ratio Usage ratio for in-flight Schema Registry requests, reported per shard. Type: gauge Labels: shard redpanda_schema_registry_queued_requests_memory_blocked Count of Schema Registry requests queued due to memory constraints, reported per shard. Type: gauge Labels: shard redpanda_schema_registry_request_errors_total Total number of errors encountered by the Schema Registry, categorized by status code. Type: counter Labels: redpanda_status=("5xx" | "4xx" | "3xx") redpanda_schema_registry_request_latency_seconds Histogram capturing the latency (in seconds) for Schema Registry requests. Type: histogram Partition metrics redpanda_kafka_max_offset High watermark offset for a partition, used to calculate consumer group lag. Type: gauge Labels: redpanda_namespace redpanda_partition redpanda_topic Related topics: Consumer group lag redpanda_kafka_under_replicated_replicas Number of partition replicas that are live yet lag behind the latest offset, redpanda_kafka_max_offset. Type: gauge Labels: redpanda_namespace redpanda_partition redpanda_topic redpanda_raft_leadership_changes Total count of leadership changes (such as successful leader elections) across all partitions for a given topic. Type: counter Labels: redpanda_namespace redpanda_topic redpanda_raft_learners_gap_bytes Total number of bytes that must be delivered to learner replicas to bring them up to date. Type: gauge Labels: shard redpanda_raft_recovery_offsets_pending Sum of offsets across partitions on a broker that still need to be recovered. Type: gauge redpanda_raft_recovery_partition_movement_available_bandwidth Available network bandwidth (in bytes per second) for partition movement operations. Type: gauge Labels: shard redpanda_raft_recovery_partition_movement_consumed_bandwidth Network bandwidth (in bytes per second) currently being consumed for partition movement. Type: gauge Labels: shard redpanda_raft_recovery_partitions_active Number of partition replicas currently undergoing recovery on a broker. Type: gauge redpanda_raft_recovery_partitions_to_recover Total count of partition replicas that are pending recovery on a broker. Type: gauge Topic metrics redpanda_cluster_partition_schema_id_validation_records_failed Count of records that failed schema ID validation during ingestion. Type: counter redpanda_kafka_partitions Configured number of partitions for a topic. Type: gauge Labels: redpanda_namespace redpanda_topic redpanda_kafka_records_fetched_total Total number of records fetched from a topic. Type: counter redpanda_kafka_records_produced_total Total number of records produced to a topic. Type: counter redpanda_kafka_replicas Configured number of replicas for a topic. Type: gauge Labels: redpanda_namespace redpanda_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. Type: counter Labels: redpanda_namespace redpanda_topic redpanda_security_audit_errors_total Cumulative count of errors encountered when creating or publishing audit event log entries. Type: counter redpanda_security_audit_last_event_timestamp_seconds Unix epoch timestamp of the last successful audit log event publication. Type: counter Broker metrics redpanda_kafka_handler_latency_seconds Histogram capturing the latency for processing Kafka requests at the broker level. Type: histogram redpanda_kafka_request_latency_seconds Histogram capturing the latency (in seconds) for produce/consume requests at the broker. This duration spans from request initiation to response fulfillment. Type: histogram Labels: redpanda_request=("produce" | "consume") redpanda_kafka_quotas_client_quota_throttle_time Histogram of client quota throttling delays (in seconds) per quota rule and type. Type: histogram 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") redpanda_kafka_quotas_client_quota_throughput Histogram of client quota throughput per quota rule and type. Type: histogram 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") Consumer group metrics redpanda_kafka_consumer_group_committed_offset Committed offset for a consumer group, segmented by topic and partition. Type: gauge Labels: group topic partition redpanda_kafka_consumer_group_consumers Number of active consumers within a consumer group. Type: gauge Labels: group redpanda_kafka_consumer_group_topics Number of topics being consumed by a consumer group. Type: gauge Labels: group REST proxy metrics redpanda_rest_proxy_inflight_requests_memory_usage_ratio Ratio of memory used by in-flight REST proxy requests, measured per shard. Type: gauge Labels: shard redpanda_rest_proxy_inflight_requests_usage_ratio Usage ratio for in-flight REST proxy requests, measured per shard. Type: gauge Labels: shard redpanda_rest_proxy_queued_requests_memory_blocked Count of REST proxy requests queued due to memory limitations, measured per shard. Type: gauge Labels: shard redpanda_rest_proxy_inflight_requests_memory_usage_ratio Memory usage ratio of in-flight requests in the rest_proxy. Type: gauge Labels: shard redpanda_rest_proxy_inflight_requests_usage_ratio Usage ratio of in-flight requests in the rest_proxy. Type: gauge Labels: shard redpanda_rest_proxy_queued_requests_memory_blocked Number of requests queued in rest_proxy, due to memory limitations. Type: gauge Labels: shard redpanda_rest_proxy_request_errors_total Cumulative count of REST proxy errors, categorized by HTTP status code. Type: counter Labels: redpanda_status("5xx" | "4xx" | "3xx") redpanda_rest_proxy_request_latency_seconds_bucket Histogram representing the internal latency buckets for REST proxy requests. Type: histogram Application metrics redpanda_application_build Build information for Redpanda, including the revision and version details. Type: gauge Labels: redpanda_revision redpanda_version redpanda_application_fips_mode Indicates whether Redpanda is running in FIPS mode. Possible values: 0 = disabled 1 = permissive 2 = enabled Type: gauge redpanda_application_uptime_seconds_total Total runtime (in seconds) of the Redpanda application. Type: gauge Cloud metrics redpanda_cloud_client_backoff Total number of object storage requests that experienced backoff delays. Type: counter Labels: For S3 and GCP: redpanda_endpoint redpanda_region For Azure Blob Storage (ABS): redpanda_endpoint redpanda_storage_account redpanda_cloud_client_client_pool_utilization Current utilization (as a percentage, 0–100) of the object storage client pool. Type: gauge Labels: redpanda_endpoint redpanda_region shard redpanda_cloud_client_client_pool_utilization Utilization of the object storage pool(0 - unused, 100 - fully utilized). Type: gauge Labels: redpanda_endpoint redpanda_region shard redpanda_cloud_client_download_backoff Total number of object storage download requests that experienced backoff delays. Type: counter Labels: For S3 and GCP: redpanda_endpoint redpanda_region For Azure Blob Storage (ABS): redpanda_endpoint redpanda_storage_account redpanda_cloud_client_downloads Total number of successful download requests from object storage. Type: counter Labels: For S3 and GCP: redpanda_endpoint redpanda_region For Azure Blob Storage (ABS): redpanda_endpoint redpanda_storage_account redpanda_cloud_client_lease_duration Histogram representing the lease duration for object storage clients. Type: histogram redpanda_cloud_client_lease_duration Lease duration histogram. Type: histogram redpanda_cloud_client_not_found Total number of object storage requests that resulted in a “not found” error. Type: counter Labels: For S3 and GCP: redpanda_endpoint redpanda_region For Azure Blob Storage (ABS): redpanda_endpoint redpanda_storage_account redpanda_cloud_client_num_borrows Count of instances where a shard borrowed a object storage client from another shard. Type: counter Labels: redpanda_endpoint redpanda_region shard redpanda_cloud_client_num_borrows Number of times the current shard had to borrow a object storage client from another shard. Type: counter Labels: redpanda_endpoint redpanda_region shard redpanda_cloud_client_upload_backoff Total number of object storage upload requests that experienced backoff delays. Type: counter Labels: For S3 and GCP: redpanda_endpoint redpanda_region For Azure Blob Storage (ABS): redpanda_endpoint redpanda_storage_account redpanda_cloud_client_uploads Total number of successful upload requests to object storage. Type: counter Labels: For S3 and GCP: redpanda_endpoint redpanda_region For Azure Blob Storage (ABS): redpanda_endpoint redpanda_storage_account redpanda_cloud_storage_cloud_log_size Total size (in bytes) of user-visible log data stored in Tiered Storage. This value increases with every segment offload and decreases when segments are deleted due to retention or compaction. Type: gauge Usage: Segmented by redpanda_namespace (e.g., kafka, kafka_internal, or redpanda), redpanda_topic, and redpanda_partition. TLS metrics redpanda_tls_truststore_expires_at_timestamp_seconds Unix epoch timestamp representing the expiration time of the shortest-lived certificate authority (CA) in the installed truststore. Type: gauge Labels: area detail Usage: Helps identify when any CA in the chain is nearing expiration. redpanda_tls_certificate_expires_at_timestamp_seconds Unix epoch timestamp for the expiration of the shortest-lived installed TLS certificate. Type: gauge Labels: area detail Usage: Useful for proactive certificate renewal by indicating the next certificate set to expire. redpanda_tls_certificate_serial The least significant 4 bytes of the serial number for the certificate that will expire next. Type: gauge Labels: area detail Usage: Provides a quick reference to identify the certificate in question. redpanda_tls_loaded_at_timestamp_seconds Unix epoch timestamp marking the last time a TLS certificate was loaded for a resource. Type: gauge Labels: area detail Usage: Indicates recent certificate updates across resources. redpanda_tls_certificate_valid Indicator of whether a resource has at least one valid TLS certificate installed. Returns 1 if a valid certificate is present and 0 if not. Type: gauge Labels: area detail Usage: Aids in continuous monitoring of certificate validity across resources. Data transforms metrics redpanda_transform_execution_latency_sec Histogram tracking the execution latency (in seconds) for processing a single record via data transforms. Type: histogram Labels: function_name redpanda_transform_execution_errors Counter for the number of errors encountered during the invocation of data transforms. Type: counter Labels: function_name redpanda_wasm_engine_cpu_seconds_total Total CPU time (in seconds) consumed by WebAssembly functions. Type: counter Labels: function_name redpanda_wasm_engine_memory_usage Current memory usage (in bytes) by a WebAssembly function. Type: gauge Labels: function_name redpanda_wasm_engine_max_memory Maximum allowed memory (in bytes) allocated for a WebAssembly function. Type: gauge Labels: function_name redpanda_wasm_binary_executable_memory_usage Amount of memory (in bytes) used by executable WebAssembly binaries. Type: gauge redpanda_transform_read_bytes Cumulative count of bytes read as input to data transforms. Type: counter Labels: function_name redpanda_transform_write_bytes Cumulative count of bytes output from data transforms. Type: counter Labels: function_name redpanda_transform_processor_lag Number of records pending processing in the input topic for a data transform. Type: gauge Labels: function_name redpanda_transform_failures Counter for each failure encountered by a data transform processor. Type: counter Labels: function_name redpanda_transform_state Current count of transform processors in a specific state (running, inactive, or errored). Type: gauge Labels: function_name state=("running" | "inactive" | "errored") Object storage metrics Object storage metrics are available only if you have: Tiered Storage enabled The cluster property cloud_storage_enabled set to true redpanda_cloud_storage_active_segments Number of remote log segments that are currently hydrated and available for read operations. Type: gauge redpanda_cloud_storage_anomalies Count of missing partition manifest anomalies detected for the topic. Type: gauge redpanda_cloud_storage_cache_op_hit Total number of successful get requests that found the requested object in the cache. Type: counter redpanda_cloud_storage_cache_op_in_progress_files Number of files currently being written to the cache. Type: gauge redpanda_cloud_storage_cache_op_miss Total count of get requests that did not find the requested object in the cache. Type: counter redpanda_cloud_storage_cache_op_put Total number of objects successfully written into the cache. Type: counter redpanda_cloud_storage_cache_space_files Current number of objects stored in the cache. Type: gauge redpanda_cloud_storage_cache_space_hwm_files High watermark for the number of objects stored in the cache. Type: gauge redpanda_cloud_storage_cache_space_hwm_size_bytes High watermark for the total size (in bytes) of cached objects. Type: gauge redpanda_cloud_storage_cache_space_size_bytes Total size (in bytes) of objects currently stored in the cache. Type: gauge redpanda_cloud_storage_cache_space_tracker_size Current count of entries in the cache access tracker. Type: gauge redpanda_cloud_storage_cache_space_tracker_syncs Total number of times the cache access tracker was synchronized with disk data. Type: counter redpanda_cloud_storage_cache_trim_carryover_trims Count of times the cache trim operation was invoked using a carryover strategy. Type: counter redpanda_cloud_storage_cache_trim_exhaustive_trims Count of instances where a fast cache trim was insufficient and an exhaustive trim was required. Type: counter redpanda_cloud_storage_cache_trim_failed_trims Count of cache trim operations that failed to free the expected amount of space, possibly indicating a bug or misconfiguration. Type: counter redpanda_cloud_storage_cache_trim_fast_trims Count of successful fast cache trim operations. Type: counter redpanda_cloud_storage_cache_trim_in_mem_trims Count of cache trim operations performed using the in-memory access tracker. Type: counter redpanda_cloud_storage_cloud_log_size Total size (in bytes) of the user-visible log data stored in Tiered Storage. This metric increases with every offloaded segment and decreases when segments are deleted due to retention or compaction. Type: gauge Usage: Labeled by redpanda_namespace (e.g., kafka, kafka_internal, or redpanda), redpanda_topic, and redpanda_partition. redpanda_cloud_storage_deleted_segments Count of log segments that have been deleted from object storage due to retention policies or compaction processes. Type: counter redpanda_cloud_storage_errors_total Cumulative count of errors encountered during object storage operations, segmented by direction. Type: counter Labels: redpanda_direction redpanda_cloud_storage_housekeeping_drains Count of times the object storage upload housekeeping queue was fully drained. Type: gauge redpanda_cloud_storage_housekeeping_jobs_completed Total number of successfully executed object storage housekeeping jobs. Type: counter redpanda_cloud_storage_housekeeping_jobs_failed Total number of object storage housekeeping jobs that failed. Type: counter redpanda_cloud_storage_housekeeping_jobs_skipped Count of object storage housekeeping jobs that were skipped during execution. Type: counter redpanda_cloud_storage_housekeeping_pauses Count of times object storage upload housekeeping was paused. Type: gauge redpanda_cloud_storage_housekeeping_requests_throttled_average_rate Average rate (per shard) of requests that were throttled during object storage operations. Type: gauge redpanda_cloud_storage_housekeeping_resumes Count of instances when object storage upload housekeeping resumed after a pause. Type: gauge redpanda_cloud_storage_housekeeping_rounds Total number of rounds executed by the object storage upload housekeeping process. Type: counter redpanda_cloud_storage_jobs_cloud_segment_reuploads Count of log segments reuploaded from object storage sources (either from the cache or via direct download). Type: gauge redpanda_cloud_storage_jobs_local_segment_reuploads Count of log segments reuploaded from the local data directory. Type: gauge redpanda_cloud_storage_jobs_manifest_reuploads Total number of partition manifest reuploads performed by housekeeping jobs. Type: gauge redpanda_cloud_storage_jobs_metadata_syncs Total number of archival configuration updates (metadata synchronizations) executed by housekeeping jobs. Type: gauge redpanda_cloud_storage_jobs_segment_deletions Total count of log segments deleted by housekeeping jobs. Type: gauge redpanda_cloud_storage_limits_downloads_throttled_sum Total cumulative time (in milliseconds) during which downloads were throttled. Type: counter redpanda_cloud_storage_partition_manifest_uploads_total Total number of successful partition manifest uploads to object storage. Type: counter redpanda_cloud_storage_partition_readers Number of active partition reader instances (fetch/timequery operations) reading from Tiered Storage. Type: gauge redpanda_cloud_storage_partition_readers_delayed Count of partition read operations delayed due to reaching the reader limit, suggesting potential saturation of Tiered Storage reads. Type: counter redpanda_cloud_storage_readers Total number of segment read cursors for hydrated remote log segments. Type: gauge redpanda_cloud_storage_segment_index_uploads_total Total number of successful segment index uploads to object storage. Type: counter redpanda_cloud_storage_segment_materializations_delayed Count of segment materialization operations that were delayed because of reader limit constraints. Type: counter redpanda_cloud_storage_segment_readers_delayed Count of segment reader operations delayed due to reaching the reader limit. This indicates a cluster is saturated with Tiered Storage reads. Type: counter redpanda_cloud_storage_segment_uploads_total Total number of successful data segment uploads to object storage. Type: counter redpanda_cloud_storage_segments Total number of log segments accounted for in object storage for the topic. Type: gauge redpanda_cloud_storage_segments_pending_deletion Count of log segments pending deletion from object storage for the topic. Type: gauge redpanda_cloud_storage_spillover_manifest_uploads_total Total number of successful spillover manifest uploads to object storage. Type: counter redpanda_cloud_storage_spillover_manifests_materialized_bytes Total bytes of memory used by spilled manifests that are currently cached in memory. Type: gauge redpanda_cloud_storage_spillover_manifests_materialized_count Count of spilled manifests currently held in memory cache. Type: gauge redpanda_cloud_storage_uploaded_bytes Total number of bytes uploaded for the topic to object storage. Type: counter Related topics Learn how to monitor Redpanda Internal metrics reference Back to top × Simple online edits For simple changes, such as fixing a typo, you can edit the content directly on GitHub. Edit on GitHub Or, open an issue to let us know about something that you want us to change. Open an issue Contribution guide For extensive content updates, or if you prefer to work locally, read our contribution guide . Was this helpful? thumb_up thumb_down group Ask in the community mail Share your feedback group_add Make a contribution Monitoring Metrics Internal Metrics