ClickHouse/docs/en/sql-reference/statements/system.md

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

443 lines
18 KiB
Markdown
Raw Normal View History

2020-04-03 13:23:32 +00:00
---
2022-08-28 14:53:34 +00:00
slug: /en/sql-reference/statements/system
sidebar_position: 36
sidebar_label: SYSTEM
2020-04-03 13:23:32 +00:00
---
2022-06-02 10:55:18 +00:00
# SYSTEM Statements
2022-06-02 10:55:18 +00:00
## RELOAD EMBEDDED DICTIONARIES
2023-03-18 02:45:43 +00:00
Reload all [Internal dictionaries](../../sql-reference/dictionaries/index.md).
By default, internal dictionaries are disabled.
Always returns `Ok.` regardless of the result of the internal dictionary update.
2022-06-02 10:55:18 +00:00
## RELOAD DICTIONARIES
Reloads all dictionaries that have been successfully loaded before.
2020-10-13 17:23:29 +00:00
By default, dictionaries are loaded lazily (see [dictionaries_lazy_load](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-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.
2022-06-02 10:55:18 +00:00
## RELOAD DICTIONARY
2020-10-13 17:23:29 +00:00
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.
2020-03-20 10:10:48 +00:00
``` sql
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**
```sql
SYSTEM RELOAD MODELS [ON CLUSTER cluster_name]
```
## RELOAD MODEL
Unloads a CatBoost model at `model_path`.
**Syntax**
```sql
SYSTEM RELOAD MODEL [ON CLUSTER cluster_name] <model_path>
```
2022-06-02 10:55:18 +00:00
## RELOAD FUNCTIONS
2021-09-25 19:32:42 +00:00
2021-09-27 20:24:12 +00:00
Reloads all registered [executable user defined functions](../functions/index.md#executable-user-defined-functions) or one of them from a configuration file.
2021-09-26 19:44:53 +00:00
2021-09-25 19:32:42 +00:00
**Syntax**
```sql
RELOAD FUNCTIONS [ON CLUSTER cluster_name]
RELOAD FUNCTION [ON CLUSTER cluster_name] function_name
2021-09-25 19:32:42 +00:00
```
2022-06-02 10:55:18 +00:00
## DROP DNS CACHE
2020-03-20 10:10:48 +00:00
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).
2020-10-13 17:23:29 +00:00
For more convenient (automatic) cache management, see disable_internal_dns_cache, dns_cache_update_period parameters.
2022-06-02 10:55:18 +00:00
## DROP MARK CACHE
2022-11-29 13:15:28 +00:00
Resets the mark cache.
2022-06-02 10:55:18 +00:00
## DROP REPLICA
2023-04-11 14:32:40 +00:00
Dead replicas of `ReplicatedMergeTree` tables can be dropped using following syntax:
``` sql
SYSTEM DROP REPLICA 'replica_name' FROM TABLE database.table;
2020-06-23 12:01:51 +00:00
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';
```
2023-04-11 14:32:40 +00:00
Queries will remove the `ReplicatedMergeTree` 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.
2020-06-23 12:01:51 +00:00
The first one removes metadata of `'replica_name'` replica of `database.table` table.
2020-06-23 12:01:51 +00:00
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.
2023-04-11 14:32:40 +00:00
## DROP DATABASE REPLICA
Dead replicas of `Replicated` databases can be dropped using following syntax:
``` sql
SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM DATABASE database;
SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'];
SYSTEM DROP DATABASE REPLICA 'replica_name' [FROM SHARD 'shard_name'] FROM ZKPATH '/path/to/table/in/zk';
```
Similar to `SYSTEM DROP REPLICA`, but removes the `Replicated` database replica path from ZooKeeper when there's no database to run `DROP DATABASE`. Please note that it does not remove `ReplicatedMergeTree` replicas (so you may need `SYSTEM DROP REPLICA` as well). Shard and replica names are the names that were specified in `Replicated` engine arguments when creating the database. Also, these names can be obtained from `database_shard_name` and `database_replica_name` columns in `system.clusters`. If the `FROM SHARD` clause is missing, then `replica_name` must be a full replica name in `shard_name|replica_name` format.
2022-06-02 10:55:18 +00:00
## DROP UNCOMPRESSED CACHE
2022-11-29 13:15:28 +00:00
Reset the uncompressed data cache.
The uncompressed data cache is enabled/disabled with the query/user/profile-level setting [use_uncompressed_cache](../../operations/settings/settings.md#setting-use_uncompressed_cache).
Its size can be configured using the server-level setting [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size).
2022-06-02 10:55:18 +00:00
## DROP COMPILED EXPRESSION CACHE
2022-11-29 13:15:28 +00:00
Reset the compiled expression cache.
The compiled expression cache is enabled/disabled with the query/user/profile-level setting [compile_expressions](../../operations/settings/settings.md#compile-expressions).
## DROP QUERY CACHE
2022-11-29 13:15:28 +00:00
Resets the [query cache](../../operations/query-cache.md).
```sql
SYSTEM DROP QUERY CACHE [ON CLUSTER cluster_name]
```
2022-06-02 10:55:18 +00:00
## 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.
```sql
SYSTEM FLUSH LOGS [ON CLUSTER cluster_name]
```
2022-06-02 10:55:18 +00:00
## 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`
```sql
SYSTEM RELOAD CONFIG [ON CLUSTER cluster_name]
```
2023-01-11 20:16:22 +00:00
## RELOAD USERS
Reloads all access storages, including: users.xml, local disk access storage, replicated (in ZooKeeper) access storage.
2023-01-11 20:16:22 +00:00
```sql
SYSTEM RELOAD USERS [ON CLUSTER cluster_name]
```
2022-06-02 10:55:18 +00:00
## SHUTDOWN
Normally shuts down ClickHouse (like `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`)
2022-06-02 10:55:18 +00:00
## KILL
Aborts ClickHouse process (like `kill -9 {$ pid_clickhouse-server}`)
2022-06-02 10:55:18 +00:00
## Managing Distributed Tables
ClickHouse can manage [distributed](../../engines/table-engines/special/distributed.md) 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](#query_language-system-stop-distributed-sends), [FLUSH DISTRIBUTED](#query_language-system-flush-distributed), and [START DISTRIBUTED SENDS](#query_language-system-start-distributed-sends) queries. You can also synchronously insert distributed data with the [insert_distributed_sync](../../operations/settings/settings.md#insert_distributed_sync) setting.
2022-06-02 10:55:18 +00:00
### STOP DISTRIBUTED SENDS
Disables background data distribution when inserting data into distributed tables.
2020-03-20 10:10:48 +00:00
``` sql
SYSTEM STOP DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
2022-06-02 10:55:18 +00:00
### 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.
2020-03-20 10:10:48 +00:00
``` sql
SYSTEM FLUSH DISTRIBUTED [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
2022-06-02 10:55:18 +00:00
### START DISTRIBUTED SENDS
Enables background data distribution when inserting data into distributed tables.
2020-03-20 10:10:48 +00:00
``` sql
SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
2022-06-02 10:55:18 +00:00
## Managing MergeTree Tables
ClickHouse can manage background processes in [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables.
2022-06-02 10:55:18 +00:00
### STOP MERGES
2019-11-01 14:17:49 +00:00
Provides possibility to stop background merges for tables in the MergeTree family:
2020-03-20 10:10:48 +00:00
``` sql
SYSTEM STOP MERGES [ON CLUSTER cluster_name] [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
2020-03-20 10:10:48 +00:00
2022-09-20 11:00:10 +00:00
:::note
`DETACH / ATTACH` table will start background merges for the table even in case when merges have been stopped for all MergeTree tables before.
:::
2019-11-01 14:17:49 +00:00
2022-06-02 10:55:18 +00:00
### START MERGES
2019-11-01 14:17:49 +00:00
Provides possibility to start background merges for tables in the MergeTree family:
2020-03-20 10:10:48 +00:00
``` sql
SYSTEM START MERGES [ON CLUSTER cluster_name] [ON VOLUME <volume_name> | [db.]merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### STOP TTL MERGES
Provides possibility to stop background delete old data according to [TTL expression](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) for tables in the MergeTree family:
2021-05-27 19:44:11 +00:00
Returns `Ok.` even if table does not exist or table has not MergeTree engine. Returns error when database does not exist:
``` sql
SYSTEM STOP TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### START TTL MERGES
Provides possibility to start background delete old data according to [TTL expression](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) for tables in the MergeTree family:
2021-05-27 19:44:11 +00:00
Returns `Ok.` even if table does not exist. Returns error when database does not exist:
``` sql
SYSTEM START TTL MERGES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### STOP MOVES
Provides possibility to stop background move data according to [TTL table expression with TO VOLUME or TO DISK clause](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) for tables in the MergeTree family:
2021-05-27 19:44:11 +00:00
Returns `Ok.` even if table does not exist. Returns error when database does not exist:
``` sql
SYSTEM STOP MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### START MOVES
Provides possibility to start background move data according to [TTL table expression with TO VOLUME and TO DISK clause](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) for tables in the MergeTree family:
2021-05-27 19:44:11 +00:00
Returns `Ok.` even if table does not exist. Returns error when database does not exist:
``` sql
SYSTEM START MOVES [ON CLUSTER cluster_name] [[db.]merge_tree_family_table_name]
```
2022-06-08 12:09:59 +00:00
### SYSTEM UNFREEZE {#query_language-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 ](alter/partition.md#alter_unfreeze-partition)
``` sql
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.
``` sql
SYSTEM WAIT LOADING PARTS [ON CLUSTER cluster_name] [db.]merge_tree_family_table_name
```
2022-06-02 10:55:18 +00:00
## Managing ReplicatedMergeTree Tables
ClickHouse can manage background replication related processes in [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md#table_engines-replication) tables.
2022-06-02 10:55:18 +00:00
### STOP FETCHES
Provides possibility to stop background fetches for inserted parts for tables in the `ReplicatedMergeTree` family:
2021-05-27 19:44:11 +00:00
Always returns `Ok.` regardless of the table engine and even if table or database does not exist.
``` sql
SYSTEM STOP FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### START FETCHES
Provides possibility to start background fetches for inserted parts for tables in the `ReplicatedMergeTree` family:
2021-05-27 19:44:11 +00:00
Always returns `Ok.` regardless of the table engine and even if table or database does not exist.
``` sql
SYSTEM START FETCHES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### STOP REPLICATED SENDS
Provides possibility to stop background sends to other replicas in cluster for new inserted parts for tables in the `ReplicatedMergeTree` family:
``` sql
SYSTEM STOP REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### START REPLICATED SENDS
Provides possibility to start background sends to other replicas in cluster for new inserted parts for tables in the `ReplicatedMergeTree` family:
``` sql
SYSTEM START REPLICATED SENDS [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### 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:
``` sql
SYSTEM STOP REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### 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:
``` sql
SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name]
```
2022-06-02 10:55:18 +00:00
### SYNC REPLICA
2023-03-17 13:15:03 +00:00
Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a cluster, but no more than `receive_timeout` seconds.
``` sql
2023-03-27 23:39:36 +00:00
SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT | PULL]
```
2023-03-27 23:39:36 +00:00
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.
2022-06-02 10:55:18 +00:00
### 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.
``` sql
SYSTEM RESTART REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name
```
2022-06-02 10:55:18 +00:00
### RESTORE REPLICA
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
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.
2021-07-29 07:44:44 +00:00
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).
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
2023-03-27 18:54:05 +00:00
:::note
Parts in all states are moved to `detached/` folder. Parts active before data loss (committed) are attached.
:::
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
2021-08-04 20:07:38 +00:00
**Syntax**
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
```sql
SYSTEM RESTORE REPLICA [db.]replicated_merge_tree_family_table_name [ON CLUSTER cluster_name]
```
Alternative syntax:
```sql
SYSTEM RESTORE REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name
```
2021-08-04 20:07:38 +00:00
**Example**
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
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.
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
2021-08-04 20:07:38 +00:00
```sql
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
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.
2021-08-04 20:07:38 +00:00
SYSTEM RESTART REPLICA test;
SYSTEM RESTORE REPLICA test;
2021-07-29 07:44:44 +00:00
```
Another way:
2021-08-04 20:07:38 +00:00
2021-07-29 07:44:44 +00:00
```sql
SYSTEM RESTORE REPLICA test ON CLUSTER cluster;
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
```
2022-06-02 10:55:18 +00:00
### 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
2022-09-20 11:00:10 +00:00
### DROP FILESYSTEM CACHE
Allows to drop filesystem cache.
```sql
SYSTEM DROP FILESYSTEM CACHE [ON CLUSTER cluster_name]
2022-09-20 11:00:10 +00:00
```
2023-01-27 01:14:42 +00:00
### SYNC FILE CACHE
:::note
It's too heavy and has potential for misuse.
:::
2023-03-18 02:45:43 +00:00
Will do sync syscall.
2023-01-27 01:14:42 +00:00
```sql
SYSTEM SYNC FILE CACHE [ON CLUSTER cluster_name]
2023-01-27 01:14:42 +00:00
```
### SYSTEM STOP LISTEN
Closes the socket and gracefully terminates the existing connections to the server on the specified port with the specified protocol.
However, if the corresponding protocol settings were not specified in the clickhouse-server configuration, this command will have no effect.
```sql
SYSTEM STOP LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP_WITH_PROXY | TCP_SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol']
```
- If `CUSTOM 'protocol'` modifier is specified, the custom protocol with the specified name defined in the protocols section of the server configuration will be stopped.
- If `QUERIES ALL` modifier is specified, all protocols are stopped.
- If `QUERIES DEFAULT` modifier is specified, all default protocols are stopped.
- If `QUERIES CUSTOM` modifier is specified, all custom protocols are stopped.
### SYSTEM START LISTEN
Allows new connections to be established on the specified protocols.
However, if the server on the specified port and protocol was not stopped using the SYSTEM STOP LISTEN command, this command will have no effect.
```sql
SYSTEM START LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP_WITH_PROXY | TCP_SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol']
```