Skip to main content
Skip to main content

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

SELECT * FROM system.metrics LIMIT 10
┌─metric───────────────────────────────┬─value─┬─description────────────────────────────────────────────────────────────┐
│ Query                                │     1 │ Number of executing queries                                            │
│ Merge                                │     0 │ Number of executing background merges                                  │
│ PartMutation                         │     0 │ Number of mutations (ALTER DELETE/UPDATE)                              │
│ ReplicatedFetch                      │     0 │ Number of data parts being fetched from replicas                       │
│ ReplicatedSend                       │     0 │ Number of data parts being sent to replicas                            │
│ ReplicatedChecks                     │     0 │ Number of data parts checking for consistency                          │
│ BackgroundMergesAndMutationsPoolTask │     0 │ Number of active merges and mutations in an associated background pool │
│ BackgroundFetchesPoolTask            │     0 │ Number of active fetches in an associated background pool              │
│ BackgroundCommonPoolTask             │     0 │ Number of active tasks in an associated background pool                │
│ BackgroundMovePoolTask               │     0 │ Number of active tasks in BackgroundProcessingPool for moves           │
└──────────────────────────────────────┴───────┴────────────────────────────────────────────────────────────────────────┘

Metric descriptions

MetricDescription
AggregatorThreadsNumber of threads in the Aggregator thread pool
AggregatorThreadsActiveNumber of threads in the Aggregator thread pool running a task
TablesLoaderForegroundThreadsNumber of threads in the async loader foreground thread pool
TablesLoaderForegroundThreadsActiveNumber of threads in the async loader foreground thread pool running a task
TablesLoaderBackgroundThreadsNumber of threads in the async loader background thread pool
TablesLoaderBackgroundThreadsActiveNumber of threads in the async loader background thread pool running a task
AsyncInsertCacheSizeNumber of async insert hash id in cache
AsynchronousInsertThreadsNumber of threads in the AsynchronousInsert thread pool
AsynchronousInsertThreadsActiveNumber of threads in the AsynchronousInsert thread pool running a task
AsynchronousReadWaitNumber of threads waiting for asynchronous read
BackgroundBufferFlushSchedulePoolSizeLimit on number of tasks in BackgroundBufferFlushSchedulePool
BackgroundBufferFlushSchedulePoolTaskNumber of active tasks in BackgroundBufferFlushSchedulePool (used for periodic Buffer flushes)
BackgroundCommonPoolSizeLimit on number of tasks in an associated background pool
BackgroundCommonPoolTaskNumber of active tasks in an associated background pool
BackgroundDistributedSchedulePoolSizeLimit on number of tasks in BackgroundDistributedSchedulePool
BackgroundDistributedSchedulePoolTaskNumber of active tasks in BackgroundDistributedSchedulePool (used for distributed sends in background)
BackgroundFetchesPoolSizeLimit on number of simultaneous fetches in an associated background pool
BackgroundFetchesPoolTaskNumber of active fetches in an associated background pool
BackgroundMergesAndMutationsPoolSizeLimit on number of active merges and mutations in an associated background pool
BackgroundMergesAndMutationsPoolTaskNumber of active merges and mutations in an associated background pool
BackgroundMessageBrokerSchedulePoolSizeLimit on number of tasks in BackgroundProcessingPool for message streaming
BackgroundMessageBrokerSchedulePoolTaskNumber of active tasks in BackgroundProcessingPool for message streaming
BackgroundMovePoolSizeLimit on number of tasks in BackgroundProcessingPool for moves
BackgroundMovePoolTaskNumber of active tasks in BackgroundProcessingPool for moves
BackgroundSchedulePoolSizeLimit on number of tasks in BackgroundSchedulePool (used for periodic ReplicatedMergeTree tasks like cleaning old data parts, altering data parts, replica re-initialization)
BackgroundSchedulePoolTaskNumber of active tasks in BackgroundSchedulePool (used for periodic ReplicatedMergeTree tasks like cleaning old data parts, altering data parts, replica re-initialization)
BackupsIOThreadsNumber of threads in the BackupsIO thread pool
BackupsIOThreadsActiveNumber of threads in the BackupsIO thread pool running a task
BackupsThreadsNumber of threads in the thread pool for BACKUP
BackupsThreadsActiveNumber of threads in thread pool for BACKUP running a task
BrokenDistributedFilesToInsertNumber of files for asynchronous insertion into Distributed tables marked as broken (starts from 0, summed for every shard)
CacheDetachedFileSegmentsNumber of existing detached cache file segments
CacheDictionaryThreadsNumber of threads in the CacheDictionary thread pool
CacheDictionaryThreadsActiveNumber of threads in the CacheDictionary thread pool running a task
CacheDictionaryUpdateQueueBatchesNumber of 'batches' (a set of keys) in update queue in CacheDictionaries
CacheDictionaryUpdateQueueKeysExact number of keys in update queue in CacheDictionaries
CacheFileSegmentsNumber of existing cache file segments
ContextLockWaitNumber of threads waiting for lock in Context (global lock)
DDLWorkerThreadsNumber of threads in the DDLWorker thread pool for ON CLUSTER queries
DDLWorkerThreadsActiveNumber of threads in the DDLWORKER thread pool for ON CLUSTER queries running a task
DatabaseCatalogThreadsNumber of threads in the DatabaseCatalog thread pool
DatabaseCatalogThreadsActiveNumber of threads in the DatabaseCatalog thread pool running a task
DatabaseOnDiskThreadsNumber of threads in the DatabaseOnDisk thread pool
DatabaseOnDiskThreadsActiveNumber of threads in the DatabaseOnDisk thread pool running a task
DelayedInsertsNumber of INSERT queries throttled due to high number of active data parts for partition in a MergeTree table
DestroyAggregatesThreadsNumber of threads in the thread pool for destroy aggregate states
DestroyAggregatesThreadsActiveNumber of threads in the thread pool for destroy aggregate states running a task
DictCacheRequestsNumber of requests in flight to data sources of dictionaries of cache type
DiskObjectStorageAsyncThreadsNumber of threads in the async thread pool for DiskObjectStorage
DiskObjectStorageAsyncThreadsActiveNumber of threads in the async thread pool for DiskObjectStorage running a task
DiskSpaceReservedForMergeDisk space reserved for currently running background merges (slightly more than total size of merging parts)
DistributedFilesToInsertNumber of pending files to process for asynchronous insertion into Distributed tables (summed for every shard)
DistributedSendNumber of connections to remote servers sending data INSERTed into Distributed tables (both synchronous and asynchronous mode)
EphemeralNodeNumber of ephemeral nodes held in ZooKeeper
FilesystemCacheElementsFilesystem cache elements (file segments)
FilesystemCacheReadBuffersNumber of active cache buffers
FilesystemCacheSizeFilesystem cache size in bytes
QueryCacheBytesTotal size of the query cache in bytes
QueryCacheEntriesTotal number of entries in the query cache
UncompressedCacheBytesTotal size of uncompressed cache in bytes (does not usually improve performance and should be mostly avoided)
UncompressedCacheCellsNumber of cells in uncompressed cache
CompiledExpressionCacheBytesTotal bytes used for the cache of JIT-compiled code
CompiledExpressionCacheCountTotal entries in the cache of JIT-compiled code
MMapCacheCellsNumber of files opened with mmap (mapped in memory, used with local_filesystem_read_method=mmap, cached to avoid TLB flushes)
MarkCacheBytesTotal size of mark cache in bytes
MarkCacheFilesTotal number of mark files cached in the mark cache
GlobalThreadNumber of threads in global thread pool
GlobalThreadActiveNumber of threads in global thread pool running a task
HTTPConnectionNumber of connections to HTTP server
HashedDictionaryThreadsNumber of threads in the HashedDictionary thread pool
HashedDictionaryThreadsActiveNumber of threads in the HashedDictionary thread pool running a task
IOPrefetchThreadsNumber of threads in the IO prefetch thread pool
IOPrefetchThreadsActiveNumber of threads in the IO prefetch thread pool running a task
IOThreadsNumber of threads in the IO thread pool
IOThreadsActiveNumber of threads in the IO thread pool running a task
IOUringInFlightEventsNumber of io_uring SQEs in flight
IOUringPendingEventsNumber of io_uring SQEs waiting to be submitted
IOWriterThreadsNumber of threads in the IO writer thread pool
IOWriterThreadsActiveNumber of threads in the IO writer thread pool running a task
InterserverConnectionNumber of connections from other replicas to fetch parts
KafkaAssignedPartitionsNumber of partitions Kafka tables currently assigned to
KafkaBackgroundReadsNumber of background reads currently working (populating materialized views from Kafka)
KafkaConsumersNumber of active Kafka consumers
KafkaConsumersInUseNumber of consumers currently used by direct or background reads
KafkaConsumersWithAssignmentNumber of active Kafka consumers with assigned partitions
KafkaLibrdkafkaThreadsNumber of active librdkafka threads
KafkaProducersNumber of active Kafka producers created
KafkaWritesNumber of currently running inserts to Kafka
KeeperAliveConnectionsNumber of alive connections
KeeperOutstandingRequestsNumber of outstanding requests
LocalThreadNumber of threads in local thread pools (taken from the global thread pool)
LocalThreadActiveNumber of threads in local thread pools running a task
MMappedAllocBytesSum bytes of mmapped allocations
MMappedAllocsTotal number of mmapped allocations
MMappedFileBytesSum size of mmapped file regions
MMappedFilesTotal number of mmapped files
MarksLoaderThreadsNumber of threads in thread pool for loading marks
MarksLoaderThreadsActiveNumber of threads in the thread pool for loading marks running a task
MaxDDLEntryIDMax processed DDL entry of DDLWorker
MaxPushedDDLEntryIDMax DDL entry of DDLWorker pushed to ZooKeeper
MemoryTrackingTotal amount of memory (bytes) allocated by the server
MergeNumber of executing background merges
MergeTreeAllRangesAnnouncementsSentCurrent number of announcements in flight from remote server to initiator about data parts set (for MergeTree tables, measured on remote server)
MergeTreeBackgroundExecutorThreadsNumber of threads in the MergeTreeBackgroundExecutor thread pool
MergeTreeBackgroundExecutorThreadsActiveNumber of threads in the MergeTreeBackgroundExecutor thread pool running a task
MergeTreeDataSelectExecutorThreadsNumber of threads in the MergeTreeDataSelectExecutor thread pool
MergeTreeDataSelectExecutorThreadsActiveNumber of threads in the MergeTreeDataSelectExecutor thread pool running a task
MergeTreePartsCleanerThreadsNumber of threads in the MergeTree parts cleaner thread pool
MergeTreePartsCleanerThreadsActiveNumber of threads in the MergeTree parts cleaner thread pool running a task
MergeTreePartsLoaderThreadsNumber of threads in the MergeTree parts loader thread pool
MergeTreePartsLoaderThreadsActiveNumber of threads in the MergeTree parts loader thread pool running a task
MergeTreeReadTaskRequestsSentCurrent number of callback requests in flight from remote server to initiator to choose read task (for MergeTree tables, measured on remote server)
MoveNumber of currently executing moves
MySQLConnectionNumber of client connections using MySQL protocol
NetworkReceiveNumber of threads receiving data from network (only ClickHouse-related, not 3rd party libraries)
NetworkSendNumber of threads sending data to network (only ClickHouse-related, not 3rd party libraries)
OpenFileForReadNumber of files open for reading
OpenFileForWriteNumber of files open for writing
ParallelFormattingOutputFormatThreadsNumber of threads in the ParallelFormattingOutputFormatThreads thread pool
ParallelFormattingOutputFormatThreadsActiveNumber of threads in the ParallelFormattingOutputFormatThreads thread pool running a task
PartMutationNumber of mutations (ALTER DELETE/UPDATE)
PartsActiveActive data part, used by current and upcoming SELECTs
PartsCommittedDeprecated. See PartsActive
PartsCompactCompact parts
PartsDeleteOnDestroyPart moved to another disk and should be deleted in own destructor
PartsDeletingNot active data part with identity refcounter, deleting right now by cleaner
PartsOutdatedNot active data part, but could be used by only current SELECTs, could be deleted after SELECTs finish
PartsPreActiveThe part is in data_parts, but not used for SELECTs
PartsPreCommittedDeprecated. See PartsPreActive
PartsTemporaryThe part is generating now, not in data_parts list
PartsWideWide parts
PendingAsyncInsertNumber of asynchronous inserts waiting for flush
PostgreSQLConnectionNumber of client connections using PostgreSQL protocol
QueryNumber of executing queries
QueryPreemptedNumber of queries stopped and waiting due to 'priority' setting
QueryThreadNumber of query processing threads
RWLockActiveReadersNumber of threads holding read lock in a table RWLock
RWLockActiveWritersNumber of threads holding write lock in a table RWLock
RWLockWaitingReadersNumber of threads waiting for read on a table RWLock
RWLockWaitingWritersNumber of threads waiting for write on a table RWLock
ReadNumber of read (read, pread, io_getevents, etc.) syscalls in flight
ReadTaskRequestsSentCurrent number of callback requests in flight from remote server to initiator to choose read task (for s3Cluster and similar, measured on remote server)
ReadonlyReplicaNumber of Replicated tables in readonly state due to re-initialization after ZooKeeper session loss or startup without ZooKeeper configured
RemoteReadNumber of reads with remote reader in flight
ReplicatedChecksNumber of data parts checking for consistency
ReplicatedFetchNumber of data parts being fetched from replica
ReplicatedSendNumber of data parts being sent to replicas
RestartReplicaThreadsNumber of threads in the RESTART REPLICA thread pool
RestartReplicaThreadsActiveNumber of threads in the RESTART REPLICA thread pool running a task
RestoreThreadsNumber of threads in the thread pool for RESTORE
RestoreThreadsActiveNumber of threads in the thread pool for RESTORE running a task
RevisionRevision of the server (number incremented for every release or release candidate except patch releases)
S3RequestsS3 requests
SendExternalTablesNumber of connections sending data for external tables to remote servers (used for GLOBAL IN and GLOBAL JOIN with distributed subqueries)
SendScalarsNumber of connections sending data for scalars to remote servers
StorageBufferBytesNumber of bytes in buffers of Buffer tables
StorageBufferRowsNumber of rows in buffers of Buffer tables
StorageDistributedThreadsNumber of threads in the StorageDistributed thread pool
StorageDistributedThreadsActiveNumber of threads in the StorageDistributed thread pool running a task
StorageHiveThreadsNumber of threads in the StorageHive thread pool
StorageHiveThreadsActiveNumber of threads in the StorageHive thread pool running a task
StorageS3ThreadsNumber of threads in the StorageS3 thread pool
StorageS3ThreadsActiveNumber of threads in the StorageS3 thread pool running a task
SystemReplicasThreadsNumber of threads in the system.replicas thread pool
SystemReplicasThreadsActiveNumber of threads in the system.replicas thread pool running a task
TCPConnectionNumber of connections to TCP server (clients with native interface), includes server-server distributed query connections
TablesToDropQueueSizeNumber of dropped tables waiting for background data removal
TemporaryFilesForAggregationNumber of temporary files created for external aggregation
TemporaryFilesForJoinNumber of temporary files created for JOIN
TemporaryFilesForSortNumber of temporary files created for external sorting
TemporaryFilesUnknownNumber of temporary files created without known purpose
ThreadPoolFSReaderThreadsNumber of threads in the thread pool for local_filesystem_read_method=threadpool
ThreadPoolFSReaderThreadsActiveNumber of threads in the thread pool for local_filesystem_read_method=threadpool running a task
ThreadPoolRemoteFSReaderThreadsNumber of threads in the thread pool for remote_filesystem_read_method=threadpool
ThreadPoolRemoteFSReaderThreadsActiveNumber of threads in the thread pool for remote_filesystem_read_method=threadpool running a task
ThreadsInOvercommitTrackerNumber of waiting threads inside of OvercommitTracker
TotalTemporaryFilesNumber of temporary files created
VersionIntegerVersion of the server in a single integer number in base-1000 (e.g., version 11.22.33 = 11022033)
WriteNumber of write (write, pwrite, io_getevents, etc.) syscalls in flight
ZooKeeperRequestNumber of requests to ZooKeeper in flight
ZooKeeperSessionNumber of sessions (connections) to ZooKeeper (should be no more than one due to lack of linearizability)
ZooKeeperWatchNumber of watches (event subscriptions) in ZooKeeper
ConcurrencyControlAcquiredTotal number of acquired CPU slots
ConcurrencyControlSoftLimitValue of soft limit on number of CPU slots

See Also