-`shard_num` (UInt32) — The shard number in the cluster, starting from 1.
-`shard_weight` (UInt32) — The relative weight of the shard when writing data.
-`replica_num` (UInt32) — The replica number in the shard, starting from 1.
-`host_name` (String) — The host name, as specified in the config.
-`host_address` (String) — The host IP address obtained from DNS.
-`port` (UInt16) — The port to use for connecting to the server.
-`user` (String) — The name of the user for connecting to the server.
-`errors_count` (UInt32) - number of times this host failed to reach replica.
-`estimated_recovery_time` (UInt32) - seconds left until replica error count is zeroed and it is considered to be back to normal.
Please note that `errors_count` is updated once per query to the cluster, but `estimated_recovery_time` is recalculated on-demand. So there could be a case of non-zero `errors_count` and zero `estimated_recovery_time`, that next query will zero `errors_count` and try to use replica as if it has no errors.
You can use this table to get information similar to the [DESCRIBE TABLE](../query_language/misc.md#misc-describe-table) query, but for multiple tables at once.
Contains information about detached parts of [MergeTree](table_engines/mergetree.md) tables. The `reason` column specifies why the part was detached. For user-detached parts, the reason is empty. Such parts can be attached with [ALTER TABLE ATTACH PARTITION|PART](../query_language/query_language/alter/#alter_attach-partition) command. For the description of other columns, see [system.parts](#system_tables-parts). If part name is invalid, values of some columns may be `NULL`. Such parts can be deleted with [ALTER TABLE DROP DETACHED PART](../query_language/query_language/alter/#alter_drop-detached).
Note that the amount of memory used by the dictionary is not proportional to the number of items stored in it. So for flat and cached dictionaries, all the memory cells are pre-assigned, regardless of how full the dictionary actually is.
Contains information about the number of events that have occurred in the system. For example, in the table, you can find how many `SELECT` queries were processed since the ClickHouse server started.
│ Query │ 12 │ Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries. │
│ ReadBufferFromFileDescriptorReadBytes │ 9931 │ Number of bytes read from file descriptors. If the file is compressed, this will show the compressed data size. │
Contains information about parameters [graphite_rollup](server_settings/settings.md#server_settings-graphite_rollup) which are used in tables with [\*GraphiteMergeTree](table_engines/graphitemergetree.md) engines.
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.
The list of supported metrics you can find in the [dbms/src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/dbms/src/Common/CurrentMetrics.cpp) source file of ClickHouse.
│ BackgroundPoolTask │ 0 │ Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping) │
│ BackgroundSchedulePoolTask │ 0 │ Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc. │
│ DiskSpaceReservedForMerge │ 0 │ Disk space reserved for currently running background merges. It is slightly more than the total size of currently merging parts. │
│ DistributedSend │ 0 │ Number of connections to remote servers sending data that was INSERTed into Distributed tables. Both synchronous and asynchronous mode. │
-`partition` (String) – The partition name. To learn what a partition is, see the description of the [ALTER](../query_language/alter.md#query_language_queries_alter) query.
-`active` (`UInt8`) – Flag that indicates whether the data part is active. If a data part is active, it's used in a table. Otherwise, it's deleted. Inactive data parts remain after merging.
-`marks` (`UInt64`) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` by the index granularity (usually 8192) (this hint doesn't work for adaptive granularity).
-`rows` (`UInt64`) – The number of rows.
-`bytes_on_disk` (`UInt64`) – Total size of all the data part files in bytes.
-`data_compressed_bytes` (`UInt64`) – Total size of compressed data in the data part. All the auxiliary files (for example, files with marks) are not included.
-`data_uncompressed_bytes` (`UInt64`) – Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included.
-`marks_bytes` (`UInt64`) – The size of the file with marks.
-`modification_time` (`DateTime`) – The time the directory with the data part was modified. This usually corresponds to the time of data part creation.|
-`remove_time` (`DateTime`) – The time when the data part became inactive.
-`refcount` (`UInt32`) – The number of places where the data part is used. A value greater than 2 indicates that the data part is used in queries or merges.
-`min_date` (`Date`) – The minimum value of the date key in the data part.
-`max_date` (`Date`) – The maximum value of the date key in the data part.
-`min_time` (`DateTime`) – The minimum value of the date and time key in the data part.
-`min_block_number` (`UInt64`) – The minimum number of data parts that make up the current part after merging.
-`max_block_number` (`UInt64`) – The maximum number of data parts that make up the current part after merging.
-`level` (`UInt32`) – Depth of the merge tree. Zero means that the current part was created by insert rather than by merging other parts.
-`data_version` (`UInt64`) – Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than `data_version`).
-`primary_key_bytes_in_memory` (`UInt64`) – The amount of memory (in bytes) used by primary key values.
-`primary_key_bytes_in_memory_allocated` (`UInt64`) – The amount of memory (in bytes) reserved for primary key values.
-`is_frozen` (`UInt8`) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup doesn't exist. For more details, see [FREEZE PARTITION](../query_language/alter.md#alter_freeze-partition)
-`database` (`String`) – Name of the database.
-`table` (`String`) – Name of the table.
-`engine` (`String`) – Name of the table engine without parameters.
-`path` (`String`) – Absolute path to the folder with data part files.
-`hash_of_all_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of compressed files.
-`hash_of_uncompressed_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of uncompressed files (files with marks, index file etc.).
-`uncompressed_hash_of_compressed_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of data in the compressed files as if they were uncompressed.
-`bytes` (`UInt64`) – Alias for `bytes_on_disk`.
-`marks_size` (`UInt64`) – Alias for `marks_bytes`.
This table contains information about events that occurred with [data parts](table_engines/custom_partitioning_key.md) in the [MergeTree](table_engines/mergetree.md) family tables, such as adding or merging data.
-`database` (String) — Name of the database the data part is in.
-`table` (String) — Name of the table the data part is in.
-`part_name` (String) — Name of the data part.
-`partition_id` (String) — ID of the partition that the data part was inserted to. The column takes the 'all' value if the partitioning is by `tuple()`.
-`rows` (UInt64) — The number of rows in the data part.
-`size_in_bytes` (UInt64) — Size of the data part in bytes.
-`merged_from` (Array(String)) — An array of names of the parts which the current part was made up from (after the merge).
-`bytes_uncompressed` (UInt64) — Size of uncompressed bytes.
-`read_rows` (UInt64) — The number of rows was read during the merge.
-`read_bytes` (UInt64) — The number of bytes was read during the merge.
-`error` (UInt16) — The code number of the occurred error.
-`exception` (String) — Text message of the occurred error.
The `system.part_log` table is created after the first inserting data to the `MergeTree` table.
-`user` (String) – The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` user. The field contains the username for a specific query, not for a query that this query initiated.
-`address` (String) – The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` on the query requestor server.
-`elapsed` (Float64) – The time in seconds since request execution started.
-`rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
-`bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
-`total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known.
-`memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../operations/settings/query_complexity.md#settings_max_memory_usage) setting.
-`query` (String) – The query text. For `INSERT`, it doesn't include the data to insert.
Contains information about execution of queries. For each query, you can see processing start time, duration of processing, error messages and other information.
ClickHouse creates this table only if the [query_log](server_settings/settings.md#server_settings-query-log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in.
To enable query logging, set the [log_queries](settings/settings.md#settings-log-queries) parameter to 1. For details, see the [Settings](settings/settings.md) section.
2. Child queries that were initiated by other queries (for distributed query execution). For these types of queries, information about the parent queries is shown in the `initial_*` columns.
-`ProfileEvents.Names` (Array(String)) — Counters that measure different metrics. The description of them could be found in the table [system.events](#system_tables-events)
-`ProfileEvents.Values` (Array(UInt64)) — Values of metrics that are listed in the `ProfileEvents.Names` column.
-`Settings.Names` (Array(String)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1.
By default, logs are added to the table at intervals of 7.5 seconds. You can set this interval in the [query_log](server_settings/settings.md#server_settings-query-log) server setting (see the `flush_interval_milliseconds` parameter). To flush the logs forcibly from the memory buffer into the table, use the `SYSTEM FLUSH LOGS` query.
The storage period for logs is unlimited. Logs aren't automatically deleted from the table. You need to organize the removal of outdated logs yourself.
You can specify an arbitrary partitioning key for the `system.query_log` table in the [query_log](server_settings/settings.md#server_settings-query-log) server setting (see the `partition_by` parameter).
ClickHouse creates this table only if the [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in.
To enable query logging, set the [log_query_threads](settings/settings.md#settings-log-query-threads) parameter to 1. For details, see the [Settings](settings/settings.md) section.
-`ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [system.events](#system_tables-events)
-`ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` column.
By default, logs are added to the table at intervals of 7.5 seconds. You can set this interval in the [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) server setting (see the `flush_interval_milliseconds` parameter). To flush the logs forcibly from the memory buffer into the table, use the `SYSTEM FLUSH LOGS` query.
When the table is deleted manually, it will be automatically created on the fly. Note that all the previous logs will be deleted.
!!! note
The storage period for logs is unlimited. Logs aren't automatically deleted from the table. You need to organize the removal of outdated logs yourself.
You can specify an arbitrary partitioning key for the `system.query_thread_log` table in the [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) server setting (see the `partition_by` parameter).
ClickHouse creates this table when the [trace_log](server_settings/settings.md#server_settings-trace_log) server configuration section is set. Also the [query_profiler_real_time_period_ns](settings/settings.md#query_profiler_real_time_period_ns) and [query_profiler_cpu_time_period_ns](settings/settings.md#query_profiler_cpu_time_period_ns) settings should be set.
To analyze logs, use the `addressToLine`, `addressToSymbol` and `demangle` introspection functions.
Columns:
-`event_date`([Date](../data_types/date.md)) — Date of sampling moment.
-`event_time`([DateTime](../data_types/datetime.md)) — Timestamp of sampling moment.
-`revision`([UInt32](../data_types/int_uint.md)) — ClickHouse server build revision.
When connecting to server by `clickhouse-client`, you see the string similar to `Connected to ClickHouse server version 19.18.1 revision 54429.`. This field contains the `revision`, but not the `version` of a server.
-`query_id`([String](../data_types/string.md)) — Query identifier that can be used to get details about a query that was running from the [query_log](#system_tables-query_log) system table.
-`trace`([Array(UInt64)](../data_types/array.md)) — Stack trace at the moment of sampling. Each element is a virtual memory address inside ClickHouse server process.
This mode is turned on if the config doesn't have sections with ZooKeeper, if an unknown error occurred when reinitializing sessions in ZooKeeper, and during session reinitialization in ZooKeeper.
The table contains information about [mutations](../query_language/alter.md#alter-mutations) of MergeTree tables and their progress. Each mutation command is represented by a single row. The table has the following columns:
**database**, **table** - The name of the database and table to which the mutation was applied.
**mutation_id** - The ID of the mutation. For replicated tables these IDs correspond to znode names in the `<table_path_in_zookeeper>/mutations/` directory in ZooKeeper. For unreplicated tables the IDs correspond to file names in the data directory of the table.
**command** - The mutation command string (the part of the query after `ALTER TABLE [db.]table`).
**create_time** - When this mutation command was submitted for execution.
**block_numbers.partition_id**, **block_numbers.number** - A nested column. For mutations of replicated tables, it contains one record for each partition: the partition ID and the block number that was acquired by the mutation (in each partition, only parts that contain blocks with numbers less than the block number acquired by the mutation in that partition will be mutated). In non-replicated tables, block numbers in all partitions form a single sequence. This means that for mutations of non-replicated tables, the column will contain one record with a single block number acquired by the mutation.
**parts_to_do** - The number of data parts that need to be mutated for the mutation to finish.
**is_done** - Is the mutation done? Note that even if `parts_to_do = 0` it is possible that a mutation of a replicated table is not done yet because of a long-running INSERT that will create a new data part that will need to be mutated.
Contains information about disks defined in the [server configuration](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure).
Columns:
-`name` ([String](../data_types/string.md)) — Name of a disk in the server configuration.
-`path` ([String](../data_types/string.md)) — Path to the mount point in the file system.
-`free_space` ([UInt64](../data_types/int_uint.md)) — Free space on disk in bytes.
-`total_space` ([UInt64](../data_types/int_uint.md)) — Disk volume in bytes.
-`keep_free_space` ([UInt64](../data_types/int_uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` parameter of disk configuration.
Contains information about storage policies and volumes defined in the [server configuration](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure).
Columns:
-`policy_name` ([String](../data_types/string.md)) — Name of the storage policy.
-`volume_name` ([String](../data_types/string.md)) — Volume name defined in the storage policy.
-`volume_priority` ([UInt64](../data_types/int_uint.md)) — Volume order number in the configuration.
-`disks` ([Array(String)](../data_types/array.md)) — Disk names, defined in the storage policy.
-`max_data_part_size` ([UInt64](../data_types/int_uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit).
-`move_factor` ([Float64](../data_types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order.