ClickHouse/docs/en/sql-reference/statements/system.md
2023-04-05 01:34:25 +02:00

15 KiB
Raw Blame History

slug sidebar_position sidebar_label
/en/sql-reference/statements/system 36 SYSTEM

SYSTEM Statements

RELOAD EMBEDDED DICTIONARIES

Reload all Internal dictionaries. By default, internal dictionaries are disabled. Always returns Ok. regardless of the result of the internal dictionary update.

RELOAD DICTIONARIES

Reloads all dictionaries that have been successfully loaded before. By default, dictionaries are loaded lazily (see dictionaries_lazy_load), so instead of being loaded automatically at startup, they are initialized on first access through dictGet function or SELECT from tables with ENGINE = Dictionary. The SYSTEM RELOAD DICTIONARIES query reloads such dictionaries (LOADED). Always returns Ok. regardless of the result of the dictionary update.

RELOAD DICTIONARY

Completely reloads a dictionary dictionary_name, regardless of the state of the dictionary (LOADED / NOT_LOADED / FAILED). Always returns Ok. regardless of the result of updating the dictionary. The status of the dictionary can be checked by querying the system.dictionaries table.

SELECT name, status FROM system.dictionaries;

RELOAD MODELS

:::note This statement and SYSTEM RELOAD MODEL merely unload catboost models from the clickhouse-library-bridge. The function catboostEvaluate() loads a model upon first access if it is not loaded yet. :::

Unloads all CatBoost models.

Syntax

SYSTEM RELOAD MODELS [ON CLUSTER cluster_name]

RELOAD MODEL

Unloads a CatBoost model at model_path.

Syntax

SYSTEM RELOAD MODEL [ON CLUSTER cluster_name] <model_path>

RELOAD FUNCTIONS

Reloads all registered executable user defined functions or one of them from a configuration file.

Syntax

RELOAD FUNCTIONS [ON CLUSTER cluster_name]
RELOAD FUNCTION [ON CLUSTER cluster_name] function_name

DROP DNS CACHE

Resets ClickHouses internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries).

For more convenient (automatic) cache management, see disable_internal_dns_cache, dns_cache_update_period parameters.

DROP MARK CACHE

Resets the mark cache.

DROP REPLICA

Dead replicas can be dropped using following syntax:

SYSTEM DROP REPLICA 'replica_name' FROM TABLE database.table;
SYSTEM DROP REPLICA 'replica_name' FROM DATABASE database;
SYSTEM DROP REPLICA 'replica_name';
SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk';

Queries will remove the replica path in ZooKeeper. It is useful when the replica is dead and its metadata cannot be removed from ZooKeeper by DROP TABLE because there is no such table anymore. It will only drop the inactive/stale replica, and it cannot drop local replica, please use DROP TABLE for that. DROP REPLICA does not drop any tables and does not remove any data or metadata from disk.

The first one removes metadata of 'replica_name' replica of database.table table. The second one does the same for all replicated tables in the database. The third one does the same for all replicated tables on the local server. The fourth one is useful to remove metadata of dead replica when all other replicas of a table were dropped. It requires the table path to be specified explicitly. It must be the same path as was passed to the first argument of ReplicatedMergeTree engine on table creation.

DROP UNCOMPRESSED CACHE

Reset the uncompressed data cache. The uncompressed data cache is enabled/disabled with the query/user/profile-level setting use_uncompressed_cache. Its size can be configured using the server-level setting uncompressed_cache_size.

DROP COMPILED EXPRESSION CACHE

Reset the compiled expression cache. The compiled expression cache is enabled/disabled with the query/user/profile-level setting compile_expressions.

DROP QUERY CACHE

Resets the query cache.

FLUSH LOGS

Flushes buffered log messages to system tables, e.g. system.query_log. Mainly useful for debugging since most system tables have a default flush interval of 7.5 seconds. This will also create system tables even if message queue is empty.

RELOAD CONFIG

Reloads ClickHouse configuration. Used when configuration is stored in ZooKeeper. Note that SYSTEM RELOAD CONFIG does not reload USER configuration stored in ZooKeeper, it only reloads USER configuration that is stored in users.xml. To reload all USER config use SYSTEM RELOAD USERS

RELOAD USERS

Reloads all access storages, including: users.xml, local disk access storage, replicated (in ZooKeeper) access storage.

SHUTDOWN

Normally shuts down ClickHouse (like service clickhouse-server stop / kill {$pid_clickhouse-server})

KILL

Aborts ClickHouse process (like kill -9 {$ pid_clickhouse-server})

Managing Distributed Tables

ClickHouse can manage distributed tables. When a user inserts data into these tables, ClickHouse first creates a queue of the data that should be sent to cluster nodes, then asynchronously sends it. You can manage queue processing with the STOP DISTRIBUTED SENDS, FLUSH DISTRIBUTED, and START DISTRIBUTED SENDS queries. You can also synchronously insert distributed data with the insert_distributed_sync setting.

STOP DISTRIBUTED SENDS

Disables background data distribution when inserting data into distributed tables.

SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name>

FLUSH DISTRIBUTED

Forces ClickHouse to send data to cluster nodes synchronously. If any nodes are unavailable, ClickHouse throws an exception and stops query execution. You can retry the query until it succeeds, which will happen when all nodes are back online.

SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name>

START DISTRIBUTED SENDS

Enables background data distribution when inserting data into distributed tables.

SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name>

Managing MergeTree Tables

ClickHouse can manage background processes in MergeTree tables.

STOP MERGES

Provides possibility to stop background merges for tables in the MergeTree family:

SYSTEM STOP MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]

:::note DETACH / ATTACH table will start background merges for the table even in case when merges have been stopped for all MergeTree tables before. :::

START MERGES

Provides possibility to start background merges for tables in the MergeTree family:

SYSTEM START MERGES [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]

STOP TTL MERGES

Provides possibility to stop background delete old data according to TTL expression for tables in the MergeTree family: Returns Ok. even if table does not exist or table has not MergeTree engine. Returns error when database does not exist:

SYSTEM STOP TTL MERGES [[db.]merge_tree_family_table_name]

START TTL MERGES

Provides possibility to start background delete old data according to TTL expression for tables in the MergeTree family: Returns Ok. even if table does not exist. Returns error when database does not exist:

SYSTEM START TTL MERGES [[db.]merge_tree_family_table_name]

STOP MOVES

Provides possibility to stop background move data according to TTL table expression with TO VOLUME or TO DISK clause for tables in the MergeTree family: Returns Ok. even if table does not exist. Returns error when database does not exist:

SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]

START MOVES

Provides possibility to start background move data according to TTL table expression with TO VOLUME and TO DISK clause for tables in the MergeTree family: Returns Ok. even if table does not exist. Returns error when database does not exist:

SYSTEM START MOVES [[db.]merge_tree_family_table_name]

SYSTEM UNFREEZE

Clears freezed backup with the specified name from all the disks. See more about unfreezing separate parts in ALTER TABLE table_name UNFREEZE WITH NAME

SYSTEM UNFREEZE WITH NAME <backup_name>

WAIT LOADING PARTS

Wait until all asynchronously loading data parts of a table (outdated data parts) will became loaded.

SYSTEM WAIT LOADING PARTS [db.]merge_tree_family_table_name

Managing ReplicatedMergeTree Tables

ClickHouse can manage background replication related processes in ReplicatedMergeTree tables.

STOP FETCHES

Provides possibility to stop background fetches for inserted parts for tables in the ReplicatedMergeTree family: Always returns Ok. regardless of the table engine and even if table or database does not exist.

SYSTEM STOP FETCHES [[db.]replicated_merge_tree_family_table_name]

START FETCHES

Provides possibility to start background fetches for inserted parts for tables in the ReplicatedMergeTree family: Always returns Ok. regardless of the table engine and even if table or database does not exist.

SYSTEM START FETCHES [[db.]replicated_merge_tree_family_table_name]

STOP REPLICATED SENDS

Provides possibility to stop background sends to other replicas in cluster for new inserted parts for tables in the ReplicatedMergeTree family:

SYSTEM STOP REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]

START REPLICATED SENDS

Provides possibility to start background sends to other replicas in cluster for new inserted parts for tables in the ReplicatedMergeTree family:

SYSTEM START REPLICATED SENDS [[db.]replicated_merge_tree_family_table_name]

STOP REPLICATION QUEUES

Provides possibility to stop background fetch tasks from replication queues which stored in Zookeeper for tables in the ReplicatedMergeTree family. Possible background tasks types - merges, fetches, mutation, DDL statements with ON CLUSTER clause:

SYSTEM STOP REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]

START REPLICATION QUEUES

Provides possibility to start background fetch tasks from replication queues which stored in Zookeeper for tables in the ReplicatedMergeTree family. Possible background tasks types - merges, fetches, mutation, DDL statements with ON CLUSTER clause:

SYSTEM START REPLICATION QUEUES [[db.]replicated_merge_tree_family_table_name]

SYNC REPLICA

Wait until a ReplicatedMergeTree table will be synced with other replicas in a cluster, but no more than receive_timeout seconds.

SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT | PULL]

After running this statement the [db.]replicated_merge_tree_family_table_name fetches commands from the common replicated log into its own replication queue, and then the query waits till the replica processes all of the fetched commands. The following modifiers are supported:

  • If a STRICT modifier was specified then the query waits for the replication queue to become empty. The STRICT version may never succeed if new entries constantly appear in the replication queue.
  • If a LIGHTWEIGHT modifier was specified then the query waits only for GET_PART, ATTACH_PART, DROP_RANGE, REPLACE_RANGE and DROP_PART entries to be processed.
  • If a PULL modifier was specified then the query pulls new replication queue entries from ZooKeeper, but does not wait for anything to be processed.

RESTART REPLICA

Provides possibility to reinitialize Zookeeper session's state for ReplicatedMergeTree table, will compare current state with Zookeeper as source of truth and add tasks to Zookeeper queue if needed. Initialization of replication queue based on ZooKeeper data happens in the same way as for ATTACH TABLE statement. For a short time, the table will be unavailable for any operations.

SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name

RESTORE REPLICA

Restores a replica if data is [possibly] present but Zookeeper metadata is lost.

Works only on readonly ReplicatedMergeTree tables.

One may execute query after:

  • ZooKeeper root / loss.
  • Replicas path /replicas loss.
  • Individual replica path /replicas/replica_name/ loss.

Replica attaches locally found parts and sends info about them to Zookeeper. Parts present on a replica before metadata loss are not re-fetched from other ones if not being outdated (so replica restoration does not mean re-downloading all data over the network).

:::note Parts in all states are moved to detached/ folder. Parts active before data loss (committed) are attached. :::

Syntax

SYSTEM RESTORE REPLICA [db.]replicated_merge_tree_family_table_name [ON CLUSTER cluster_name]

Alternative syntax:

SYSTEM RESTORE REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name

Example

Creating a table on multiple servers. After the replica's metadata in ZooKeeper is lost, the table will attach as read-only as metadata is missing. The last query needs to execute on every replica.

CREATE TABLE test(n UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/', '{replica}')
ORDER BY n PARTITION BY n % 10;

INSERT INTO test SELECT * FROM numbers(1000);

-- zookeeper_delete_path("/clickhouse/tables/test", recursive=True) <- root loss.

SYSTEM RESTART REPLICA test;
SYSTEM RESTORE REPLICA test;

Another way:

SYSTEM RESTORE REPLICA test ON CLUSTER cluster;

RESTART REPLICAS

Provides possibility to reinitialize Zookeeper sessions state for all ReplicatedMergeTree tables, will compare current state with Zookeeper as source of true and add tasks to Zookeeper queue if needed

DROP FILESYSTEM CACHE

Allows to drop filesystem cache.

SYSTEM DROP FILESYSTEM CACHE

SYNC FILE CACHE

:::note It's too heavy and has potential for misuse. :::

Will do sync syscall.

SYSTEM SYNC FILE CACHE