--- toc_priority: 66 toc_title: ClickHouse Keeper --- # [pre-production] ClickHouse Keeper {#clickHouse-keeper} ClickHouse server uses [ZooKeeper](https://zookeeper.apache.org/) coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is an alternative coordination system compatible with ZooKeeper. !!! warning "Warning" This feature is currently in the pre-production stage. We test it in our CI and on small internal installations. ## Implementation details {#implementation-details} ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, has quite a simple and powerful data model. ZooKeeper's coordination algorithm called ZAB (ZooKeeper Atomic Broadcast) doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper ClickHouse Keeper is written in C++ and uses [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows to have linearizability for reads and writes, has several open-source implementations in different languages. By default, ClickHouse Keeper provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with ClickHouse Keeper. Snapshots and logs have an incompatible format with ZooKeeper, but `clickhouse-keeper-converter` tool allows to convert ZooKeeper data to ClickHouse Keeper snapshot. Interserver protocol in ClickHouse Keeper is also incompatible with ZooKeeper so mixed ZooKeeper / ClickHouse Keeper cluster is impossible. ClickHouse Keeper supports Access Control List (ACL) the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth`, `digest`, `host` and `ip`. Digest authentication scheme uses pair `username:password`. Password is encoded in Base64. !!! info "Note" External integrations are not supported. ## Configuration {#configuration} ClickHouse Keeper can be used as a standalone replacement for ZooKeeper or as an internal part of the ClickHouse server, but in both cases configuration is almost the same `.xml` file. The main ClickHouse Keeper configuration tag is ``. Keeper configuration has the following parameters: - `tcp_port` — Port for a client to connect (default for ZooKeeper is `2181`). - `tcp_port_secure` — Secure port for an SSL connection between client and keeper-server. - `server_id` — Unique server id, each participant of the ClickHouse Keeper cluster must have a unique number (1, 2, 3, and so on). - `log_storage_path` — Path to coordination logs, better to store logs on the non-busy device (same for ZooKeeper). - `snapshot_storage_path` — Path to coordination snapshots. Other common parameters are inherited from the ClickHouse server config (`listen_host`, `logger`, and so on). Internal coordination settings are located in `.` section: - `operation_timeout_ms` — Timeout for a single client operation (ms) (default: 10000). - `min_session_timeout_ms` — Min timeout for client session (ms) (default: 10000). - `session_timeout_ms` — Max timeout for client session (ms) (default: 100000). - `dead_session_check_period_ms` — How often ClickHouse Keeper check dead sessions and remove them (ms) (default: 500). - `heart_beat_interval_ms` — How often a ClickHouse Keeper leader will send heartbeats to followers (ms) (default: 500). - `election_timeout_lower_bound_ms` — If the follower didn't receive heartbeats from the leader in this interval, then it can initiate leader election (default: 1000). - `election_timeout_upper_bound_ms` — If the follower didn't receive heartbeats from the leader in this interval, then it must initiate leader election (default: 2000). - `rotate_log_storage_interval` — How many log records to store in a single file (default: 100000). - `reserved_log_items` — How many coordination log records to store before compaction (default: 100000). - `snapshot_distance` — How often ClickHouse Keeper will create new snapshots (in the number of records in logs) (default: 100000). - `snapshots_to_keep` — How many snapshots to keep (default: 3). - `stale_log_gap` — Threshold when leader considers follower as stale and sends the snapshot to it instead of logs (default: 10000). - `fresh_log_gap` — When node became fresh (default: 200). - `max_requests_batch_size` - Max size of batch in requests count before it will be sent to RAFT (default: 100). - `force_sync` — Call `fsync` on each write to coordination log (default: true). - `quorum_reads` — Execute read requests as writes through whole RAFT consensus with similar speed (default: false). - `raft_logs_level` — Text logging level about coordination (trace, debug, and so on) (default: system default). - `auto_forwarding` — Allow to forward write requests from followers to the leader (default: true). - `shutdown_timeout` — Wait to finish internal connections and shutdown (ms) (default: 5000). - `startup_timeout` — If the server doesn't connect to other quorum participants in the specified timeout it will terminate (ms) (default: 30000). - `four_letter_word_allow_list` — Allow list of 4lw commands (default: "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro"). Quorum configuration is located in `.` section and contain servers description. The only parameter for the whole quorum is `secure`, which enables encrypted connection for communication between quorum participants. The parameter can be set `true` if SSL connection is required for internal communication between nodes, or left unspecified otherwise. The main parameters for each `` are: - `id` — Server identifier in a quorum. - `hostname` — Hostname where this server is placed. - `port` — Port where this server listens for connections. Examples of configuration for quorum with three nodes can be found in [integration tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/integration) with `test_keeper_` prefix. Example configuration for server #1: ```xml 2181 1 /var/lib/clickhouse/coordination/log /var/lib/clickhouse/coordination/snapshots 10000 30000 trace 1 zoo1 9444 2 zoo2 9444 3 zoo3 9444 ``` ## How to run {#how-to-run} ClickHouse Keeper is bundled into the ClickHouse server package, just add configuration of `` and start ClickHouse server as always. If you want to run standalone ClickHouse Keeper you can start it in a similar way with: ```bash clickhouse-keeper --config /etc/your_path_to_config/config.xml ``` If you don't have the symlink (`clickhouse-keeper`) you can create it or specify `keeper` as argument: ```bash clickhouse keeper --config /etc/your_path_to_config/config.xml ``` ## Four Letter Word Commands {#four-letter-word-commands} ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. The 4lw commands has a allow list configuration `four_letter_word_allow_list` which has default value "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro". You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. ``` echo mntr | nc localhost 9181 ``` Bellow is the detailed 4lw commands: - `ruok`: Tests if server is running in a non-error state. The server will respond with imok if it is running. Otherwise it will not respond at all. A response of "imok" does not necessarily indicate that the server has joined the quorum, just that the server process is active and bound to the specified client port. Use "stat" for details on state wrt quorum and client connection information. ``` imok ``` - `mntr`: Outputs a list of variables that could be used for monitoring the health of the cluster. ``` zk_version v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 zk_avg_latency 0 zk_max_latency 0 zk_min_latency 0 zk_packets_received 68 zk_packets_sent 68 zk_num_alive_connections 1 zk_outstanding_requests 0 zk_server_state leader zk_znode_count 4 zk_watch_count 1 zk_ephemerals_count 0 zk_approximate_data_size 723 zk_open_file_descriptor_count 310 zk_max_file_descriptor_count 10240 zk_followers 0 zk_synced_followers 0 ``` - `srvr`: Lists full details for the server. ``` ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 Latency min/avg/max: 0/0/0 Received: 2 Sent : 2 Connections: 1 Outstanding: 0 Zxid: 34 Mode: leader Node count: 4 ``` - `stat`: Lists brief details for the server and connected clients. ``` ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 Clients: 192.168.1.1:52852(recved=0,sent=0) 192.168.1.1:52042(recved=24,sent=48) Latency min/avg/max: 0/0/0 Received: 4 Sent : 4 Connections: 1 Outstanding: 0 Zxid: 36 Mode: leader Node count: 4 ``` - `srst`: Reset server statistics. The command will affect the result of `srvr`, `mntr` and `stat`. ``` Server stats reset. ``` - `conf`: Print details about serving configuration. ``` server_id=1 tcp_port=2181 four_letter_word_allow_list=* log_storage_path=./coordination/logs snapshot_storage_path=./coordination/snapshots max_requests_batch_size=100 session_timeout_ms=30000 operation_timeout_ms=10000 dead_session_check_period_ms=500 heart_beat_interval_ms=500 election_timeout_lower_bound_ms=1000 election_timeout_upper_bound_ms=2000 reserved_log_items=1000000000000000 snapshot_distance=10000 auto_forwarding=true shutdown_timeout=5000 startup_timeout=240000 raft_logs_level=information snapshots_to_keep=3 rotate_log_storage_interval=100000 stale_log_gap=10000 fresh_log_gap=200 max_requests_batch_size=100 quorum_reads=false force_sync=false compress_logs=true compress_snapshots_with_zstd_format=true configuration_change_tries_count=20 ``` - `cons`: List full connection/session details for all clients connected to this server. Includes information on numbers of packets received/sent, session id, operation latencies, last operation performed, etc... ``` 192.168.1.1:52163(recved=0,sent=0,sid=0xffffffffffffffff,lop=NA,est=1636454787393,to=30000,lzxid=0xffffffffffffffff,lresp=0,llat=0,minlat=0,avglat=0,maxlat=0) 192.168.1.1:52042(recved=9,sent=18,sid=0x0000000000000001,lop=List,est=1636454739887,to=30000,lcxid=0x0000000000000005,lzxid=0x0000000000000005,lresp=1636454739892,llat=0,minlat=0,avglat=0,maxlat=0) ``` - `crst`: Reset connection/session statistics for all connections. ``` Connection stats reset. ``` - `envi`: Print details about serving environment ``` Environment: clickhouse.keeper.version=v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 host.name=ZBMAC-C02D4054M.local os.name=Darwin os.arch=x86_64 os.version=19.6.0 cpu.count=12 user.name=root user.home=/Users/JackyWoo/ user.dir=/Users/JackyWoo/project/jd/clickhouse/cmake-build-debug/programs/ user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ ``` - `dirs`: Shows the total size of snapshot and log files in bytes ``` snapshot_dir_size: 0 log_dir_size: 3875 ``` - `isro`: Tests if server is running in read-only mode. The server will respond with "ro" if in read-only mode or "rw" if not in read-only mode. ``` rw ``` - `wchs`: Lists brief information on watches for the server. ``` 1 connections watching 1 paths Total watches:1 ``` - `wchc`: Lists detailed information on watches for the server, by session. This outputs a list of sessions (connections) with associated watches (paths). Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. ``` 0x0000000000000001 /clickhouse/task_queue/ddl ``` - `wchp`: Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (i. e. impact server performance), use it carefully. ``` /clickhouse/task_queue/ddl 0x0000000000000001 ``` - `dump`: Lists the outstanding sessions and ephemeral nodes. This only works on the leader. ``` Sessions dump (2): 0x0000000000000001 0x0000000000000002 Sessions with Ephemerals (1): 0x0000000000000001 /clickhouse/task_queue/ddl ``` ## [experimental] Migration from ZooKeeper {#migration-from-zookeeper} Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: 1. Stop all ZooKeeper nodes. 2. Optional, but recommended: find ZooKeeper leader node, start and stop it again. It will force ZooKeeper to create a consistent snapshot. 3. Run `clickhouse-keeper-converter` on a leader, for example: ```bash clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/version-2 --output-dir /path/to/clickhouse/keeper/snapshots ``` 4. Copy snapshot to ClickHouse server nodes with a configured `keeper` or start ClickHouse Keeper instead of ZooKeeper. The snapshot must persist on all nodes, otherwise, empty nodes can be faster and one of them can become a leader. [Original article](https://clickhouse.com/docs/en/operations/clickhouse-keeper/)