system.metrics
Querying in ClickHouse Cloud
The data in this system table is held locally on each node in ClickHouse Cloud. Obtaining a complete view of all data, therefore, requires the clusterAllReplicas function. See here for further details.
Contains metrics which can be calculated instantly, or have a current value. For example, the number of simultaneously processed queries or the current replica delay. This table is always up to date.
Columns:
metric(String) — Metric name.value(Int64) — Metric value.description(String) — Metric description.
You can find all supported metrics in source file src/Common/CurrentMetrics.cpp.
Example
Metric descriptions
| Metric | Description |
|---|---|
| AggregatorThreads | Number of threads in the Aggregator thread pool |
| AggregatorThreadsActive | Number of threads in the Aggregator thread pool running a task |
| TablesLoaderForegroundThreads | Number of threads in the async loader foreground thread pool |
| TablesLoaderForegroundThreadsActive | Number of threads in the async loader foreground thread pool running a task |
| TablesLoaderBackgroundThreads | Number of threads in the async loader background thread pool |
| TablesLoaderBackgroundThreadsActive | Number of threads in the async loader background thread pool running a task |
| AsyncInsertCacheSize | Number of async insert hash id in cache |
| AsynchronousInsertThreads | Number of threads in the AsynchronousInsert thread pool |
| AsynchronousInsertThreadsActive | Number of threads in the AsynchronousInsert thread pool running a task |
| AsynchronousReadWait | Number of threads waiting for asynchronous read |
| BackgroundBufferFlushSchedulePoolSize | Limit on number of tasks in BackgroundBufferFlushSchedulePool |
| BackgroundBufferFlushSchedulePoolTask | Number of active tasks in BackgroundBufferFlushSchedulePool (used for periodic Buffer flushes) |
| BackgroundCommonPoolSize | Limit on number of tasks in an associated background pool |
| BackgroundCommonPoolTask | Number of active tasks in an associated background pool |
| BackgroundDistributedSchedulePoolSize | Limit on number of tasks in BackgroundDistributedSchedulePool |
| BackgroundDistributedSchedulePoolTask | Number of active tasks in BackgroundDistributedSchedulePool (used for distributed sends in background) |
| BackgroundFetchesPoolSize | Limit on number of simultaneous fetches in an associated background pool |
| BackgroundFetchesPoolTask | Number of active fetches in an associated background pool |
| BackgroundMergesAndMutationsPoolSize | Limit on number of active merges and mutations in an associated background pool |
| BackgroundMergesAndMutationsPoolTask | Number of active merges and mutations in an associated background pool |
| BackgroundMessageBrokerSchedulePoolSize | Limit on number of tasks in BackgroundProcessingPool for message streaming |
| BackgroundMessageBrokerSchedulePoolTask | Number of active tasks in BackgroundProcessingPool for message streaming |
| BackgroundMovePoolSize | Limit on number of tasks in BackgroundProcessingPool for moves |
| BackgroundMovePoolTask | Number of active tasks in BackgroundProcessingPool for moves |
| BackgroundSchedulePoolSize | Limit on number of tasks in BackgroundSchedulePool (used for periodic ReplicatedMergeTree tasks like cleaning old data parts, altering data parts, replica re-initialization) |
| BackgroundSchedulePoolTask | Number of active tasks in BackgroundSchedulePool (used for periodic ReplicatedMergeTree tasks like cleaning old data parts, altering data parts, replica re-initialization) |
| BackupsIOThreads | Number of threads in the BackupsIO thread pool |
| BackupsIOThreadsActive | Number of threads in the BackupsIO thread pool running a task |
| BackupsThreads | Number of threads in the thread pool for BACKUP |
| BackupsThreadsActive | Number of threads in thread pool for BACKUP running a task |
| BrokenDistributedFilesToInsert | Number of files for asynchronous insertion into Distributed tables marked as broken (starts from 0, summed for every shard) |
| CacheDetachedFileSegments | Number of existing detached cache file segments |
| CacheDictionaryThreads | Number of threads in the CacheDictionary thread pool |
| CacheDictionaryThreadsActive | Number of threads in the CacheDictionary thread pool running a task |
| CacheDictionaryUpdateQueueBatches | Number of 'batches' (a set of keys) in update queue in CacheDictionaries |
| CacheDictionaryUpdateQueueKeys | Exact number of keys in update queue in CacheDictionaries |
| CacheFileSegments | Number of existing cache file segments |
| ContextLockWait | Number of threads waiting for lock in Context (global lock) |
| DDLWorkerThreads | Number of threads in the DDLWorker thread pool for ON CLUSTER queries |
| DDLWorkerThreadsActive | Number of threads in the DDLWORKER thread pool for ON CLUSTER queries running a task |
| DatabaseCatalogThreads | Number of threads in the DatabaseCatalog thread pool |
| DatabaseCatalogThreadsActive | Number of threads in the DatabaseCatalog thread pool running a task |
| DatabaseOnDiskThreads | Number of threads in the DatabaseOnDisk thread pool |
| DatabaseOnDiskThreadsActive | Number of threads in the DatabaseOnDisk thread pool running a task |
| DelayedInserts | Number of INSERT queries throttled due to high number of active data parts for partition in a MergeTree table |
| DestroyAggregatesThreads | Number of threads in the thread pool for destroy aggregate states |
| DestroyAggregatesThreadsActive | Number of threads in the thread pool for destroy aggregate states running a task |
| DictCacheRequests | Number of requests in flight to data sources of dictionaries of cache type |
| DiskObjectStorageAsyncThreads | Number of threads in the async thread pool for DiskObjectStorage |
| DiskObjectStorageAsyncThreadsActive | Number of threads in the async thread pool for DiskObjectStorage running a task |
| DiskSpaceReservedForMerge | Disk space reserved for currently running background merges (slightly more than total size of merging parts) |
| DistributedFilesToInsert | Number of pending files to process for asynchronous insertion into Distributed tables (summed for every shard) |
| DistributedSend | Number of connections to remote servers sending data INSERTed into Distributed tables (both synchronous and asynchronous mode) |
| EphemeralNode | Number of ephemeral nodes held in ZooKeeper |
| FilesystemCacheElements | Filesystem cache elements (file segments) |
| FilesystemCacheReadBuffers | Number of active cache buffers |
| FilesystemCacheSize | Filesystem cache size in bytes |
| QueryCacheBytes | Total size of the query cache in bytes |
| QueryCacheEntries | Total number of entries in the query cache |
| UncompressedCacheBytes | Total size of uncompressed cache in bytes (does not usually improve performance and should be mostly avoided) |
| UncompressedCacheCells | Number of cells in uncompressed cache |
| CompiledExpressionCacheBytes | Total bytes used for the cache of JIT-compiled code |
| CompiledExpressionCacheCount | Total entries in the cache of JIT-compiled code |
| MMapCacheCells | Number of files opened with mmap (mapped in memory, used with local_filesystem_read_method=mmap, cached to avoid TLB flushes) |
| MarkCacheBytes | Total size of mark cache in bytes |
| MarkCacheFiles | Total number of mark files cached in the mark cache |
| GlobalThread | Number of threads in global thread pool |
| GlobalThreadActive | Number of threads in global thread pool running a task |
| HTTPConnection | Number of connections to HTTP server |
| HashedDictionaryThreads | Number of threads in the HashedDictionary thread pool |
| HashedDictionaryThreadsActive | Number of threads in the HashedDictionary thread pool running a task |
| IOPrefetchThreads | Number of threads in the IO prefetch thread pool |
| IOPrefetchThreadsActive | Number of threads in the IO prefetch thread pool running a task |
| IOThreads | Number of threads in the IO thread pool |
| IOThreadsActive | Number of threads in the IO thread pool running a task |
| IOUringInFlightEvents | Number of io_uring SQEs in flight |
| IOUringPendingEvents | Number of io_uring SQEs waiting to be submitted |
| IOWriterThreads | Number of threads in the IO writer thread pool |
| IOWriterThreadsActive | Number of threads in the IO writer thread pool running a task |
| InterserverConnection | Number of connections from other replicas to fetch parts |
| KafkaAssignedPartitions | Number of partitions Kafka tables currently assigned to |
| KafkaBackgroundReads | Number of background reads currently working (populating materialized views from Kafka) |
| KafkaConsumers | Number of active Kafka consumers |
| KafkaConsumersInUse | Number of consumers currently used by direct or background reads |
| KafkaConsumersWithAssignment | Number of active Kafka consumers with assigned partitions |
| KafkaLibrdkafkaThreads | Number of active librdkafka threads |
| KafkaProducers | Number of active Kafka producers created |
| KafkaWrites | Number of currently running inserts to Kafka |
| KeeperAliveConnections | Number of alive connections |
| KeeperOutstandingRequests | Number of outstanding requests |
| LocalThread | Number of threads in local thread pools (taken from the global thread pool) |
| LocalThreadActive | Number of threads in local thread pools running a task |
| MMappedAllocBytes | Sum bytes of mmapped allocations |
| MMappedAllocs | Total number of mmapped allocations |
| MMappedFileBytes | Sum size of mmapped file regions |
| MMappedFiles | Total number of mmapped files |
| MarksLoaderThreads | Number of threads in thread pool for loading marks |
| MarksLoaderThreadsActive | Number of threads in the thread pool for loading marks running a task |
| MaxDDLEntryID | Max processed DDL entry of DDLWorker |
| MaxPushedDDLEntryID | Max DDL entry of DDLWorker pushed to ZooKeeper |
| MemoryTracking | Total amount of memory (bytes) allocated by the server |
| Merge | Number of executing background merges |
| MergeTreeAllRangesAnnouncementsSent | Current number of announcements in flight from remote server to initiator about data parts set (for MergeTree tables, measured on remote server) |
| MergeTreeBackgroundExecutorThreads | Number of threads in the MergeTreeBackgroundExecutor thread pool |
| MergeTreeBackgroundExecutorThreadsActive | Number of threads in the MergeTreeBackgroundExecutor thread pool running a task |
| MergeTreeDataSelectExecutorThreads | Number of threads in the MergeTreeDataSelectExecutor thread pool |
| MergeTreeDataSelectExecutorThreadsActive | Number of threads in the MergeTreeDataSelectExecutor thread pool running a task |
| MergeTreePartsCleanerThreads | Number of threads in the MergeTree parts cleaner thread pool |
| MergeTreePartsCleanerThreadsActive | Number of threads in the MergeTree parts cleaner thread pool running a task |
| MergeTreePartsLoaderThreads | Number of threads in the MergeTree parts loader thread pool |
| MergeTreePartsLoaderThreadsActive | Number of threads in the MergeTree parts loader thread pool running a task |
| MergeTreeReadTaskRequestsSent | Current number of callback requests in flight from remote server to initiator to choose read task (for MergeTree tables, measured on remote server) |
| Move | Number of currently executing moves |
| MySQLConnection | Number of client connections using MySQL protocol |
| NetworkReceive | Number of threads receiving data from network (only ClickHouse-related, not 3rd party libraries) |
| NetworkSend | Number of threads sending data to network (only ClickHouse-related, not 3rd party libraries) |
| OpenFileForRead | Number of files open for reading |
| OpenFileForWrite | Number of files open for writing |
| ParallelFormattingOutputFormatThreads | Number of threads in the ParallelFormattingOutputFormatThreads thread pool |
| ParallelFormattingOutputFormatThreadsActive | Number of threads in the ParallelFormattingOutputFormatThreads thread pool running a task |
| PartMutation | Number of mutations (ALTER DELETE/UPDATE) |
| PartsActive | Active data part, used by current and upcoming SELECTs |
| PartsCommitted | Deprecated. See PartsActive |
| PartsCompact | Compact parts |
| PartsDeleteOnDestroy | Part moved to another disk and should be deleted in own destructor |
| PartsDeleting | Not active data part with identity refcounter, deleting right now by cleaner |
| PartsOutdated | Not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finish |
| PartsPreActive | The part is in data_parts, but not used for SELECTs |
| PartsPreCommitted | Deprecated. See PartsPreActive |
| PartsTemporary | The part is generating now, not in data_parts list |
| PartsWide | Wide parts |
| PendingAsyncInsert | Number of asynchronous inserts waiting for flush |
| PostgreSQLConnection | Number of client connections using PostgreSQL protocol |
| Query | Number of executing queries |
| QueryPreempted | Number of queries stopped and waiting due to 'priority' setting |
| QueryThread | Number of query processing threads |
| RWLockActiveReaders | Number of threads holding read lock in a table RWLock |
| RWLockActiveWriters | Number of threads holding write lock in a table RWLock |
| RWLockWaitingReaders | Number of threads waiting for read on a table RWLock |
| RWLockWaitingWriters | Number of threads waiting for write on a table RWLock |
| Read | Number of read (read, pread, io_getevents, etc.) syscalls in flight |
| ReadTaskRequestsSent | Current number of callback requests in flight from remote server to initiator to choose read task (for s3Cluster and similar, measured on remote server) |
| ReadonlyReplica | Number of Replicated tables in readonly state due to re-initialization after ZooKeeper session loss or startup without ZooKeeper configured |
| RemoteRead | Number of reads with remote reader in flight |
| ReplicatedChecks | Number of data parts checking for consistency |
| ReplicatedFetch | Number of data parts being fetched from replica |
| ReplicatedSend | Number of data parts being sent to replicas |
| RestartReplicaThreads | Number of threads in the RESTART REPLICA thread pool |
| RestartReplicaThreadsActive | Number of threads in the RESTART REPLICA thread pool running a task |
| RestoreThreads | Number of threads in the thread pool for RESTORE |
| RestoreThreadsActive | Number of threads in the thread pool for RESTORE running a task |
| Revision | Revision of the server (number incremented for every release or release candidate except patch releases) |
| S3Requests | S3 requests |
| SendExternalTables | Number of connections sending data for external tables to remote servers (used for GLOBAL IN and GLOBAL JOIN with distributed subqueries) |
| SendScalars | Number of connections sending data for scalars to remote servers |
| StorageBufferBytes | Number of bytes in buffers of Buffer tables |
| StorageBufferRows | Number of rows in buffers of Buffer tables |
| StorageDistributedThreads | Number of threads in the StorageDistributed thread pool |
| StorageDistributedThreadsActive | Number of threads in the StorageDistributed thread pool running a task |
| StorageHiveThreads | Number of threads in the StorageHive thread pool |
| StorageHiveThreadsActive | Number of threads in the StorageHive thread pool running a task |
| StorageS3Threads | Number of threads in the StorageS3 thread pool |
| StorageS3ThreadsActive | Number of threads in the StorageS3 thread pool running a task |
| SystemReplicasThreads | Number of threads in the system.replicas thread pool |
| SystemReplicasThreadsActive | Number of threads in the system.replicas thread pool running a task |
| TCPConnection | Number of connections to TCP server (clients with native interface), includes server-server distributed query connections |
| TablesToDropQueueSize | Number of dropped tables waiting for background data removal |
| TemporaryFilesForAggregation | Number of temporary files created for external aggregation |
| TemporaryFilesForJoin | Number of temporary files created for JOIN |
| TemporaryFilesForSort | Number of temporary files created for external sorting |
| TemporaryFilesUnknown | Number of temporary files created without known purpose |
| ThreadPoolFSReaderThreads | Number of threads in the thread pool for local_filesystem_read_method=threadpool |
| ThreadPoolFSReaderThreadsActive | Number of threads in the thread pool for local_filesystem_read_method=threadpool running a task |
| ThreadPoolRemoteFSReaderThreads | Number of threads in the thread pool for remote_filesystem_read_method=threadpool |
| ThreadPoolRemoteFSReaderThreadsActive | Number of threads in the thread pool for remote_filesystem_read_method=threadpool running a task |
| ThreadsInOvercommitTracker | Number of waiting threads inside of OvercommitTracker |
| TotalTemporaryFiles | Number of temporary files created |
| VersionInteger | Version of the server in a single integer number in base-1000 (e.g., version 11.22.33 = 11022033) |
| Write | Number of write (write, pwrite, io_getevents, etc.) syscalls in flight |
| ZooKeeperRequest | Number of requests to ZooKeeper in flight |
| ZooKeeperSession | Number of sessions (connections) to ZooKeeper (should be no more than one due to lack of linearizability) |
| ZooKeeperWatch | Number of watches (event subscriptions) in ZooKeeper |
| ConcurrencyControlAcquired | Total number of acquired CPU slots |
| ConcurrencyControlSoftLimit | Value of soft limit on number of CPU slots |
See Also
- system.asynchronous_metrics — Contains periodically calculated metrics.
- system.events — Contains a number of events that occurred.
- system.metric_log — Contains a history of metrics values from tables
system.metricsandsystem.events. - Monitoring — Base concepts of ClickHouse monitoring.