ClickHouse Keeper (clickhouse-keeper)
このページは ClickHouse Cloud には適用されません。ここで説明している手順は、ClickHouse Cloud サービスでは自動化されています。
ClickHouse Keeper は、データのレプリケーションと分散 DDL クエリ実行のためのコーディネーションシステムを提供します。ClickHouse Keeper は ZooKeeper と互換性があります。
実装の詳細
ZooKeeper は、最初期によく知られるようになったオープンソースのコーディネーションシステムの 1 つです。Java で実装されており、シンプルかつ強力なデータモデルを備えています。ZooKeeper のコーディネーションアルゴリズムである ZooKeeper Atomic Broadcast (ZAB) は、各 ZooKeeper ノードがローカルで読み取りを処理するため、読み取りに対して線形化可能性を保証しません。ZooKeeper と異なり、ClickHouse Keeper は C++ で実装されており、RAFT アルゴリズムの実装を使用しています。このアルゴリズムは読み取りおよび書き込みの線形化可能性を実現し、複数の言語でオープンソース実装が提供されています。
デフォルトでは、ClickHouse Keeper は ZooKeeper と同じ保証、すなわち線形化可能な書き込みと線形化が保証されない読み取りを提供します。互換性のあるクライアント・サーバープロトコルを持つため、標準的な ZooKeeper クライアントで ClickHouse Keeper とやり取りできます。スナップショットとログは ZooKeeper と互換性のないフォーマットですが、clickhouse-keeper-converter ツールにより ZooKeeper のデータを ClickHouse Keeper のスナップショットに変換できます。ClickHouse Keeper のサーバー間プロトコルも ZooKeeper と互換性がないため、ZooKeeper / ClickHouse Keeper 混在クラスタは構成できません。
ClickHouse Keeper は、ZooKeeper と同じ方法で Access Control Lists (ACL) をサポートします。ClickHouse Keeper は同じ権限セットをサポートし、world、auth、digest という同一の組み込みスキームを持ちます。digest 認証スキームは username:password のペアを使用し、パスワードは Base64 でエンコードされます。
外部との連携はサポートされていません。
設定
ClickHouse Keeper は、ZooKeeper のスタンドアロン代替として、または ClickHouse サーバーの内部コンポーネントとして利用できます。どちらの場合も、設定はほぼ同じ .xml ファイルです。
Keeper の設定項目
ClickHouse Keeper の主な設定タグは <keeper_server> で、次のパラメータがあります。
| Parameter | Description | Default |
|---|---|---|
tcp_port | クライアントが接続するためのポート。 | 2181 |
tcp_port_secure | クライアントと keeper-server 間の SSL 接続用のセキュアポート。 | - |
server_id | 一意のサーバー ID。ClickHouse Keeper クラスターの各参加ノードは、(1, 2, 3, … のような) 一意の番号を持つ必要があります。 | - |
log_storage_path | 調停ログ (coordination logs) の保存パス。ZooKeeper と同様、ログは負荷の低いノード上に保存するのが望ましいです。 | - |
snapshot_storage_path | 調停スナップショットの保存パス。 | - |
enable_reconfiguration | reconfig による動的なクラスター再構成を有効にします。 | False |
max_memory_usage_soft_limit | Keeper の最大メモリ使用量に対するソフトリミット (バイト単位)。 | max_memory_usage_soft_limit_ratio * physical_memory_amount |
max_memory_usage_soft_limit_ratio | max_memory_usage_soft_limit が未設定、または 0 に設定されている場合、この値を使用してデフォルトのソフトリミットを定義します。 | 0.9 |
cgroups_memory_observer_wait_time | max_memory_usage_soft_limit が未設定、または 0 に設定されている場合、この間隔で物理メモリ量を監視します。メモリ量が変化した場合、max_memory_usage_soft_limit_ratio に基づいて Keeper のメモリのソフトリミットを再計算します。 | 15 |
http_control | HTTP control インターフェイスの設定。 | - |
digest_enabled | リアルタイムのデータ整合性チェックを有効にします。 | True |
create_snapshot_on_exit | シャットダウン時にスナップショットを作成します。 | - |
hostname_checks_enabled | クラスター設定に対するホスト名の妥当性チェックを有効にします (例:localhost がリモートエンドポイントと共に使用されている場合など)。 | True |
four_letter_word_white_list | 4lw コマンドのホワイトリスト。 | conf, cons, crst, envi, ruok, srst, srvr, stat, wchs, dirs, mntr, isro, rcvr, apiv, csnp, lgif, rqld, ydld |
enable_ipv6 | IPv6 を有効にします。 | True |
その他の一般的なパラメータは、ClickHouse サーバーの設定 (listen_host、logger など) から継承されます。
内部調停設定
内部調停設定は <keeper_server>.<coordination_settings> セクションに定義されており、次のパラメータを持ちます。
| Parameter | Description | Default |
|---|---|---|
operation_timeout_ms | 単一クライアント操作のタイムアウト (ms) | 10000 |
min_session_timeout_ms | クライアントセッションの最小タイムアウト (ms) | 10000 |
session_timeout_ms | クライアントセッションの最大タイムアウト (ms) | 100000 |
dead_session_check_period_ms | ClickHouse Keeper がデッドセッションを検出して削除する頻度 (ms) | 500 |
heart_beat_interval_ms | ClickHouse Keeper のリーダーがフォロワーにハートビートを送信する頻度 (ms) | 500 |
election_timeout_lower_bound_ms | フォロワーがこの間隔内にリーダーからのハートビートを受信しなかった場合、リーダー選出を開始できます。election_timeout_upper_bound_ms 以下である必要があります。理想的には両者は同一値にすべきではありません。 | 1000 |
election_timeout_upper_bound_ms | フォロワーがこの間隔内にリーダーからのハートビートを受信しなかった場合、リーダー選出を開始しなければなりません。 | 2000 |
rotate_log_storage_interval | 1 つのファイルに保存するログレコード数。 | 100000 |
reserved_log_items | コンパクション前に保持するコーディネーションログレコード数。 | 100000 |
snapshot_distance | ClickHouse Keeper が新しいスナップショットを作成する頻度 (ログ内のレコード数単位)。 | 100000 |
snapshots_to_keep | 保持するスナップショット数。 | 3 |
stale_log_gap | リーダーがフォロワーをステイルと見なし、ログではなくスナップショットを送信する際のしきい値。 | 10000 |
fresh_log_gap | ノードが最新と見なされるタイミング。 | 200 |
max_requests_batch_size | RAFT に送信される前のリクエストバッチの最大サイズ (リクエスト数)。 | 100 |
force_sync | 各コーディネーションログ書き込みごとに fsync を呼び出します。 | true |
quorum_reads | 読み取りリクエストを、書き込みと同様に RAFT コンセンサス全体を通して実行します (速度もほぼ同等です)。 | false |
raft_logs_level | コーディネーションに関するテキストログレベル (trace、debug など)。 | system default |
auto_forwarding | フォロワーからリーダーへの書き込みリクエストのフォワードを許可します。 | true |
shutdown_timeout | 内部接続の完了およびシャットダウンまで待機する時間 (ms)。 | 5000 |
startup_timeout | サーバーが指定されたタイムアウト内に他のクォーラム参加者に接続できない場合、終了します (ms)。 | 30000 |
async_replication | 非同期レプリケーションを有効にします。すべての書き込みおよび読み取りの保証を維持しつつ、パフォーマンスを向上させます。後方互換性を損なわないよう、デフォルトでは無効になっています。 | false |
latest_logs_cache_size_threshold | 最新ログエントリのインメモリキャッシュの合計最大サイズ | 1GiB |
commit_logs_cache_size_threshold | コミットに次に必要となるログエントリのインメモリキャッシュの合計最大サイズ | 500MiB |
disk_move_retries_wait_ms | ディスク間でファイルを移動中に発生した失敗後、再試行の間に待機する時間 | 1000 |
disk_move_retries_during_init | 初期化中にディスク間でファイルを移動している際に発生した失敗に対して行う再試行回数 | 100 |
experimental_use_rocksdb | バックエンドストレージとして RocksDB を使用するかどうか | 0 |
クォーラム構成は <keeper_server>.<raft_configuration> セクションにあり、サーバーの定義が含まれます。
クォーラム全体に対する唯一のパラメータは secure であり、クォーラム参加者間の通信に対する暗号化接続を有効にします。ノード間の内部通信で SSL 接続が必要な場合はこのパラメータを true に設定し、それ以外の場合は未指定のままにできます。
各 <server> に対する主なパラメータは次のとおりです。
id— クォーラム内のサーバー識別子。hostname— このサーバーが配置されているホスト名。port— このサーバーが接続を待ち受けるポート。can_become_leader— サーバーをlearnerとして設定するにはfalseを指定します。省略した場合の値はtrueです。
ClickHouse Keeper クラスターのトポロジーが変更される場合(例: サーバーの置き換え)、server_id と hostname の対応関係を必ず一貫して維持し、既存の server_id を別のサーバーで使い回したり順番を入れ替えたりしないようにしてください(特に、ClickHouse Keeper をデプロイするために自動化スクリプトに依存している場合に発生する可能性があります)。
Keeper インスタンスのホストが変更されうる場合は、生の IP アドレスではなくホスト名を定義して使用することを推奨します。ホスト名の変更はサーバーの削除と再追加に相当し、場合によっては実施できないことがあります(例: クォーラムを満たす Keeper インスタンス数が不足している場合)。
後方互換性を損なわないように、async_replication はデフォルトでは無効になっています。クラスター内のすべての Keeper インスタンスが async_replication をサポートするバージョン(v23.9 以降)で動作している場合は、パフォーマンスをデメリットなしに向上させられるため、有効化することを推奨します。
3 ノードでクォーラムを構成する場合の設定例は、test_keeper_ プレフィックスが付いた integration tests にあります。サーバー #1 の構成例は次のとおりです。
実行方法
ClickHouse Keeper は ClickHouse サーバーパッケージに同梱されています。<keeper_server> の設定を /etc/your_path_to_config/clickhouse-server/config.xml に追加し、通常どおり ClickHouse サーバーを起動してください。ClickHouse Keeper をスタンドアロンで実行したい場合は、同様の方法で次のように起動できます。
clickhouse-keeper というシンボリックリンクがない場合は、それを作成するか、clickhouse の引数として keeper を指定できます。
4 文字コマンド
ClickHouse Keeper は、ZooKeeper のものとほぼ同じ 4 文字コマンド (4lw) も提供します。各コマンドは mntr や stat などの 4 文字で構成されています。代表的なコマンドとして、stat はサーバーおよび接続クライアントに関する一般的な情報を返し、srvr と cons はそれぞれサーバーおよび接続に関する詳細情報を返します。
4lw コマンドには、four_letter_word_white_list というホワイトリスト設定があり、デフォルト値は conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,ydld です。
クライアントポートに対して、telnet または nc 経由で ClickHouse Keeper にこれらのコマンドを送信できます。
以下に 4lw コマンドの詳細を示します:
ruok: サーバーがエラーのない状態で動作しているかをテストします。サーバーが動作している場合はimokと応答します。そうでない場合は一切応答しません。imokという応答は、サーバーがクォーラムに参加していることを必ずしも意味せず、サーバープロセスが稼働しており、指定されたクライアントポートにバインドされていることだけを示します。クォーラムに関する状態やクライアント接続情報の詳細については「stat」を使用してください。
mntr: クラスターの健全性を監視するために使用できる変数のリストを出力します。
srvr: サーバーに関するすべての詳細を表示します。
stat: サーバーおよび接続中のクライアントに関する概要情報を一覧表示します。
srst: サーバーの統計情報をリセットします。このコマンドはsrvr、mntr、statの結果に影響を与えます。
conf: サービング設定の詳細を表示します。
cons: このサーバーに接続しているすべてのクライアントの接続およびセッションに関する詳細情報を一覧表示します。受信/送信パケット数、セッション ID、操作レイテンシー、最後に実行された操作などの情報が含まれます。
crst: すべての接続について、接続/セッションの統計情報をリセットします。
envi: 実行環境の詳細を表示
dirs: スナップショットファイルおよびログファイルの総サイズをバイト単位で表示します
isro: サーバーが読み取り専用モードで動作しているかを確認します。サーバーは、読み取り専用モードの場合はro、そうでない場合はrwを返します。
wchs: サーバー上のウォッチの概要情報を一覧表示します。
wchc: サーバー上のウォッチ情報をセッションごとに詳細表示します。これにより、各セッション(接続)と、それに紐づくウォッチ対象(パス)の一覧が出力されます。ウォッチの数によっては、この操作は高コストになり(サーバーのパフォーマンスに影響を与える可能性がある)ため、慎重に使用してください。
wchp: サーバー上のウォッチについて、パスごとの詳細情報を一覧表示します。セッション情報と関連付けられたパス(znode)のリストを出力します。ウォッチの数によっては、この操作は負荷の高い処理となりうる(サーバーのパフォーマンスに影響を与える可能性がある)ため、注意して使用してください。
dump: 未処理のセッションおよびエフェメラルノードを一覧表示します。これはリーダーノードでのみ有効です。
csnp: スナップショット作成タスクをスケジュールします。成功した場合は、スケジュールされたスナップショットの最新のコミット済みログインデックスを返し、失敗した場合はFailed to schedule snapshot creation task.を返します。lgifコマンドを使用すると、スナップショットが完了したかどうかを確認できます。
lgif: Keeper ログ情報。first_log_idx: ログストア内における自ノードの最初のログインデックス;first_log_term: 自ノードの最初のログターム;last_log_idx: ログストア内における自ノードの最後のログインデックス;last_log_term: 自ノードの最後のログターム;last_committed_log_idx: ステートマシンにおける自ノードの最後にコミットされたログインデックス;leader_committed_log_idx: 自ノードから見たリーダーのコミット済みログインデックス;target_committed_log_idx: コミットされるべき対象のログインデックス;last_snapshot_idx: 直近のスナップショットに含まれる最大のコミット済みログインデックス。
rqld: 自ノードを新しいリーダーにするよう要求します。要求が送信された場合はSent leadership request to leader.を返し、要求が送信されなかった場合はFailed to send leadership request to leader.を返します。ノードがすでにリーダーである場合でも、結果は要求が送信された場合と同じになります。
ftfl: すべてのフィーチャーフラグと、それぞれが Keeper インスタンスで有効かどうかを一覧表示します。
ydld: リーダー権限を放棄してフォロワーになるよう要求します。リクエストを受け取ったサーバーがリーダーの場合、まず書き込み操作を一時停止し、後継ノード(現在のリーダーが後継になることはありません)が最新ログのキャッチアップを完了するまで待機してから辞任します。後継ノードは自動的に選出されます。リクエストの送信に成功した場合はSent yield leadership request to leader.を返し、送信に失敗した場合はFailed to send yield leadership request to leader.を返します。ノードがすでにフォロワーである場合も、リクエストが送信された場合と同じ結果になります。
pfev: 収集されたすべてのイベントの値を返します。各イベントごとに、イベント名、イベント値、およびイベントの説明を返します。
HTTP 制御
ClickHouse Keeper は、レプリカがトラフィックを受信できる状態かどうかを確認するための HTTP インターフェイスを提供します。これは、Kubernetes のようなクラウド環境で使用できます。
/ready エンドポイントを有効化するための設定例:
フィーチャーフラグ
Keeper は ZooKeeper およびそのクライアントと完全互換ですが、ClickHouse クライアントから利用できる独自の機能やリクエスト種別も追加しています。
これらの機能は後方互換性のない変更を引き起こす可能性があるため、その多くはデフォルトで無効化されており、keeper_server.feature_flags 設定を使用して有効化できます。
すべての機能は個別に無効化することもできます。
Keeper クラスターで新しい機能を有効にしたい場合は、まずクラスター内のすべての Keeper インスタンスをその機能をサポートするバージョンに更新してから、その機能自体を有効にすることを推奨します。
multi_read を無効化し、check_not_exists を有効化するフィーチャーフラグ設定の例:
次の機能を利用できます:
| Feature | Description | Default |
|---|---|---|
multi_read | 複数読み取りリクエストのサポート | 1 |
filtered_list | ノードの種類(一時的か永続的か)で結果をフィルタリングする list リクエストのサポート | 1 |
check_not_exists | ノードが存在しないことを検証する CheckNotExists リクエストのサポート | 1 |
create_if_not_exists | ノードが存在しない場合に作成を試みる CreateIfNotExists リクエストのサポート。すでに存在する場合は変更は行われず、ZOK が返されます | 1 |
remove_recursive | 対象ノードとそのサブツリーを削除する RemoveRecursive リクエストのサポート | 1 |
一部の機能フラグはバージョン 25.7 からデフォルトで有効になっています。
Keeper を 25.7 以降にアップグレードする場合は、まずバージョン 24.9 以降にアップグレードしてから 25.7 以降に上げることを推奨します。
ZooKeeper からの移行
ZooKeeper から ClickHouse Keeper へのシームレスな移行はできません。ZooKeeper クラスターを停止し、データを変換してから ClickHouse Keeper を起動する必要があります。clickhouse-keeper-converter ツールを使用すると、ZooKeeper のログおよびスナップショットを ClickHouse Keeper のスナップショットに変換できます。このツールは ZooKeeper 3.4 より新しいバージョンでのみ動作します。移行の手順は次のとおりです。
-
すべての ZooKeeper ノードを停止します。
-
オプションですが推奨です。ZooKeeper のリーダーノードを特定し、それを起動してから再度停止します。これにより、ZooKeeper に整合性の取れたスナップショットを作成させることができます。
-
リーダー上で
clickhouse-keeper-converterを実行します。例:
keeperが設定されている ClickHouse サーバーノードにスナップショットをコピーするか、ZooKeeper の代わりに ClickHouse Keeper を起動します。スナップショットはすべてのノードに永続化されている必要があります。そうでないと、空のノードのほうが起動が速く、そのうちの 1 つがリーダーになってしまう可能性があります。
keeper-converter ツールは Keeper のスタンドアロンバイナリでは利用できません。
ClickHouse がインストールされている場合は、ClickHouse のバイナリを直接使用できます。
Otherwise, you can download the binary and run the tool as described above without installing ClickHouse.
クォーラム喪失後の復旧
ClickHouse Keeper は Raft を使用しているため、クラスタサイズに応じて一定数のノード障害を許容できます。
例えば、3 ノードクラスタの場合、1 ノードだけがクラッシュした状態であれば、正しく動作し続けます。
クラスタ構成は動的に変更できますが、いくつか制約があります。再構成も Raft に依存しているため、 クラスタにノードを追加 / 削除するにはクォーラムが必要です。クラスタ内で多数のノードを同時に失い、 再起動する見込みがまったくない場合、Raft は動作を停止し、通常の方法ではクラスタを再構成できなくなります。
ただし、ClickHouse Keeper にはリカバリーモードがあり、1 ノードだけでクラスタを強制的に再構成できます。 これは、ノードを再起動できない場合、あるいは同じエンドポイントで新しいインスタンスを起動できない場合の 最後の手段としてのみ実施してください。
続行する前に注意すべき重要な点:
- 障害が発生したノードが、再びクラスタに接続できないことを確認してください。
- 手順で明示されるまで、新しいノードはいずれも起動しないでください。
上記を確認したら、次の作業を行います:
- 新しいリーダーとする Keeper ノードを 1 つ選択します。そのノードのデータがクラスタ全体に使用されるため、状態が最も最新であるノードを選ぶことを推奨します。
- まず最初に、選択したノードの
log_storage_pathとsnapshot_storage_pathディレクトリをバックアップします。 - 使用するすべてのノード上でクラスタ構成を再設定します。
- 選択したノードに 4 文字コマンド
rcvrを送信して、そのノードをリカバリーモードに移行させるか、あるいはそのノード上の Keeper インスタンスを停止し、--force-recovery引数を付けて再起動します。 - 新しいノード上の Keeper インスタンスを 1 台ずつ起動し、次のノードを起動する前に、
mntrがzk_server_stateに対してfollowerを返すことを確認します。 - リカバリーモード中、リーダーノードは新しいノードとクォーラムを達成するまで
mntrコマンドに対してエラーメッセージを返し、クライアントおよびフォロワーからのすべてのリクエストを拒否します。 - クォーラム達成後、リーダーノードは通常の動作モードに戻り、すべてのリクエストを受け付けるようになります。
mntrを使用して Raft を検証すると、zk_server_stateに対してleaderが返されるはずです。
Keeper でのディスクの使用
Keeper は、スナップショット、ログファイル、および状態ファイルを保存するために、外部ディスク の一部をサポートしています。
サポートされているディスクの種類は次のとおりです。
- s3_plain
- s3
- local
以下は、設定ファイル内に含まれるディスク定義の例です。
ログ用にディスクを使用するには、keeper_server.log_storage_disk 設定にディスク名を指定します。
スナップショット用にディスクを使用するには、keeper_server.snapshot_storage_disk 設定にディスク名を指定します。
さらに、keeper_server.latest_log_storage_disk および keeper_server.latest_snapshot_storage_disk をそれぞれ使用することで、最新のログやスナップショットには別のディスクを使用できます。
その場合、新しいログやスナップショットが作成されると、Keeper はファイルを正しいディスクに自動的に移動します。
状態ファイル用にディスクを使用するには、keeper_server.state_storage_disk 設定にディスク名を指定します。
ディスク間でのファイル移動は安全であり、転送の途中で Keeper が停止してもデータを失うリスクはありません。 ファイルが新しいディスクに完全に移動されるまでは、古いディスクから削除されることはありません。
keeper_server.coordination_settings.force_sync を true(デフォルト値は true)に設定した Keeper は、すべての種類のディスクに対して一律の保証を満たすことはできません。
現時点では、local タイプのディスクのみが永続的な同期をサポートします。
force_sync を使用する場合、latest_log_storage_disk を使用していないなら、log_storage_disk は local ディスクでなければなりません。
latest_log_storage_disk を使用する場合、それは常に local ディスクである必要があります。
force_sync が無効な場合は、あらゆるタイプのディスクを任意の構成で使用できます。
Keeper インスタンスのストレージ構成の一例は、次のようになります。
このインスタンスは、最新のログ以外のすべてのログをディスク log_s3_plain に保存し、最新のログのみをディスク log_local に保存します。
同じロジックがスナップショットにも適用され、最新のスナップショット以外はすべて snapshot_s3_plain に保存され、最新のスナップショットのみがディスク snapshot_local に保存されます。
ディスク構成の変更
新しいディスク構成を適用する前に、すべての Keeper のログとスナップショットを手動でバックアップしてください。
階層化ディスク構成(最新ファイル用に個別のディスクを使用する構成)が定義されている場合、Keeper は起動時にファイルを正しいディスクへ自動的に移動しようとします。 以前と同じ保証が適用され、ファイルが新しいディスクへ完全に移動されるまでは古いディスクから削除されないため、複数回の再起動を安全に行うことができます。
ファイルをまったく新しいディスクへ移動する必要がある場合(または 2 ディスク構成から単一ディスク構成へ移行する場合)、keeper_server.old_snapshot_storage_disk と keeper_server.old_log_storage_disk を複数定義して使用することができます。
次の設定は、以前の 2 ディスク構成から、まったく新しい単一ディスク構成へ移行する方法を示しています。
起動時には、log_local および log_s3_plain 上のすべてのログファイルが log_local2 ディスクに移動されます。
また、snapshot_local および snapshot_s3_plain 上のすべてのスナップショットファイルが snapshot_local2 ディスクに移動されます。
ログキャッシュの設定
ディスクから読み取るデータ量を最小限に抑えるために、Keeper はログエントリをメモリにキャッシュします。 リクエストが大きい場合、ログエントリが多くのメモリを消費するため、キャッシュされるログの量には上限があります。 この上限は次の 2 つの設定で制御されます:
latest_logs_cache_size_threshold- キャッシュに保存される最新ログの合計サイズcommit_logs_cache_size_threshold- 次にコミットする必要がある後続ログの合計サイズ
デフォルト値が大きすぎる場合は、これら 2 つの設定値を小さくすることでメモリ使用量を削減できます。
pfev コマンドを使用して、それぞれのキャッシュおよびファイルから読み取られたログの量を確認できます。
Prometheus エンドポイントのメトリクスを使用して、両方のキャッシュの現在のサイズを追跡することもできます。
Prometheus
Keeper は Prometheus によるスクレイプ用のメトリクスデータを公開できます。
設定:
endpoint– Prometheus サーバーがメトリクスをスクレイプするための HTTP エンドポイント。先頭は '/' とします。port–endpoint用のポート。metrics– system.metrics テーブルからメトリクスを公開するかを制御するフラグ。events– system.events テーブルからメトリクスを公開するかを制御するフラグ。asynchronous_metrics– system.asynchronous_metrics テーブルから現在のメトリクス値を公開するかを制御するフラグ。
例
確認します(127.0.0.1 を ClickHouse サーバーの IP アドレスまたはホスト名に置き換えてください):
ClickHouse Cloud における Prometheus 連携 も参照してください。
ClickHouse Keeper ユーザーガイド
このガイドでは、ClickHouse Keeper を構成するためのシンプルで最小限の設定と、分散処理(分散操作)をテストする方法の例を示します。この例では、Linux 上の 3 ノードを使用します。
1. Keeper 設定でノードを構成する
-
3 つのホスト(
chnode1、chnode2、chnode3)に 3 つの ClickHouse インスタンスをインストールします。(ClickHouse のインストール方法の詳細は、クイックスタート を参照してください。) -
各ノードで、ネットワークインターフェイス経由の外部通信を許可するために、次のエントリを追加します。
-
次の ClickHouse Keeper 設定を 3 台すべてのサーバーに追加し、各サーバーに対して
<server_id>設定を更新します。chnode1は1、chnode2は2、というように設定します。上で使用した基本的な設定は次のとおりです。
Parameter Description Example tcp_port Keeper のクライアントが使用するポート 9181(ZooKeeper の 2181 と同等のデフォルト) server_id Raft 構成で使用される各 ClickHouse Keeper サーバーの識別子 1 coordination_settings タイムアウトなどのパラメータ用のセクション タイムアウト: 10000、ログレベル: trace server 参加するサーバーの定義 各サーバー定義の一覧 raft_configuration Keeper クラスター内の各サーバーの設定 各サーバーとその設定 id Keeper サービス用のサーバーの数値 ID 1 hostname Keeper クラスター内の各サーバーのホスト名、IP または FQDN chnode1.domain.comport サーバー間 Keeper 接続用のリッスンポート 9234 -
Zookeeper コンポーネントを有効化します。これは ClickHouse Keeper エンジンを使用します。
上で使用した基本的な設定は次のとおりです。
Parameter Description Example node ClickHouse Keeper への接続用ノードの一覧 各サーバーごとの設定エントリ host 各 ClickHouse Keeper ノードのホスト名、IP または FQDN chnode1.domain.comport ClickHouse Keeper のクライアント用ポート 9181 -
ClickHouse を再起動し、各 Keeper インスタンスが稼働していることを確認します。各サーバー上で次のコマンドを実行します。
ruokコマンドは、Keeper が稼働して健全な状態であればimokを返します。 -
systemデータベースには、ClickHouse Keeper インスタンスの詳細が含まれるzookeeperという名前のテーブルがあります。次のようにテーブルを参照します。
テーブルは次のとおりです。
2. ClickHouse でクラスタを構成する
-
2 つのノード上に、2 シャード・各 1 レプリカというシンプルなクラスタを構成します。3 台目のノードは、ClickHouse Keeper の要件であるクォーラムを満たすために使用します。
chnode1とchnode2の設定を更新します。次のクラスタ定義では、各ノードに 1 つずつシャードを配置し、合計 2 シャードとし、レプリケーションは行いません。この例では、データの一部は一方のノードに、残りはもう一方のノードに配置されます。Parameter Description Example shard クラスタ定義内のシャード(レプリカの集合) 各シャードごとのレプリカのリスト replica 各レプリカの設定 各レプリカの設定エントリ host レプリカシャードを配置するサーバのホスト名、IP、または FQDN chnode1.domain.comport ネイティブ TCP プロトコルで通信するために使用されるポート 9000 user クラスタインスタンスへの認証に使用されるユーザー名 default password クラスタインスタンスへの接続を許可するために定義されたユーザーのパスワード ClickHouse123! -
ClickHouse を再起動し、クラスタが作成されたことを確認します。
次のようにクラスタが表示されます。
3. 分散テーブルを作成してテストする
-
chnode1上の ClickHouse クライアントを使用して、新しいクラスタ上に新しいデータベースを作成します。ON CLUSTER句により、データベースは自動的に両方のノード上に作成されます。 -
db1データベース上に新しいテーブルを作成します。ここでも、ON CLUSTER句によって両方のノード上にテーブルが作成されます。 -
chnode1ノードで、いくつかの行を追加します: -
chnode2ノードでも、いくつかの行を追加します: -
各ノードで
SELECT文を実行しても、そのノード上のデータしか表示されないことが分かります。たとえば、chnode1では:chnode2では: -
-
2 つのシャード上のデータを表現するために
Distributedテーブルを作成できます。Distributedテーブルエンジンを使用するテーブル自体はデータを保持しませんが、複数サーバーにまたがる分散クエリ処理を可能にします。読み取りはすべてのシャードに対して行われ、書き込みはシャード間に分散できます。chnode1で次のクエリを実行します: -
dist_tableに対してクエリを実行すると、2 つのシャードから 4 行すべてのデータが返されることが分かります:
まとめ
このガイドでは、ClickHouse Keeper を使用してクラスタをセットアップする方法を説明しました。ClickHouse Keeper を使用すると、クラスタを構成し、シャード間でレプリケート可能な分散テーブルを定義できます。
一意のパスを使った ClickHouse Keeper の構成
このページは ClickHouse Cloud には適用されません。ここで説明している手順は、ClickHouse Cloud サービスでは自動化されています。
説明
この記事では、組み込みの {uuid} マクロ設定を使用して、
ClickHouse Keeper または ZooKeeper に一意のエントリを作成する方法を説明します。
テーブルを頻繁に作成・削除する場合、一意のパスを使用すると便利です。各パスを作成するたびに
そのパス内で新しい uuid が使用され、パスは再利用されないため、
Keeper のガベージコレクションがパスエントリを削除するまで数分待つ必要がなくなります。
サンプル環境
3 ノードのクラスタを使用し、3 つすべてのノードに ClickHouse Keeper を構成し、 そのうち 2 つのノードに ClickHouse を構成します。これにより、 ClickHouse Keeper 用に 3 ノード (タイブレーカーノードを含む) が用意され、 2 つのレプリカで構成された 1 つの ClickHouse シャードが構成されます。
| node | description |
|---|---|
chnode1.marsnet.local | データノード - クラスタ cluster_1S_2R |
chnode2.marsnet.local | データノード - クラスタ cluster_1S_2R |
chnode3.marsnet.local | ClickHouse Keeper タイブレーカーノード |
クラスタの設定例:
{uuid} を使用するためのテーブル設定手順
- 各サーバーでマクロを設定します サーバー 1 の例:
shard と replica についてはマクロを定義していますが、{uuid} はここでは定義されていない点に注意してください。これは組み込みのものなので、あらためて定義する必要はありません。
- データベースを作成する
- マクロと
{uuid}を使用してクラスター上にテーブルを作成する
┌─host──────────────────┬─port─┬─status─┬─error─┬─num_hosts_remaining─┬─num_hosts_active─┐ │ chnode1.marsnet.local │ 9440 │ 0 │ │ 1 │ 0 │ │ chnode2.marsnet.local │ 9440 │ 0 │ │ 0 │ 0 │ └───────────────────────┴──────┴────────┴───────┴─────────────────────┴──────────────────┘
テスト
- 最初のノード(例:
chnode1)にデータを挿入します
- 2番目のノード(例:
chnode2)にデータを挿入する
- 分散テーブルでレコードを表示する
代替案
デフォルトのレプリケーションパスは、マクロおよび {uuid} を使用して事前に定義できます。
- 各ノードでテーブル用のデフォルトを設定する
各ノードが特定のデータベース向けに使用される場合は、ノードごとにマクロ {database} を定義することもできます。
- 明示的なパラメーターを指定せずにテーブルを作成します:
クエリ ID: ab68cda9-ae41-4d6d-8d3b-20d8255774ee
┌─host──────────────────┬─port─┬─status─┬─error─┬─num_hosts_remaining─┬─num_hosts_active─┐ │ chnode2.marsnet.local │ 9440 │ 0 │ │ 1 │ 0 │ │ chnode1.marsnet.local │ 9440 │ 0 │ │ 0 │ 0 │ └───────────────────────┴──────┴────────┴───────┴─────────────────────┴──────────────────┘
2 行が返されました。経過時間: 1.175 秒。
トラブルシューティング
テーブル情報とUUIDを取得するためのコマンド例:
上記のテーブルに対応する UUID テーブルの情報を ZooKeeper から取得するためのコマンド例
データベースは Atomic である必要があります。以前のバージョンからアップグレードした場合、
default データベースはおそらく Ordinary タイプになっています。
確認するには:
例えば、
ClickHouse Keeper の動的再構成
このページは ClickHouse Cloud には適用されません。ここで説明している手順は、ClickHouse Cloud サービスでは自動化されています。
説明
ClickHouse Keeper は、keeper_server.enable_reconfiguration が有効になっている場合に、ZooKeeper の reconfig
コマンドによるクラスタの動的な再構成を部分的にサポートします。
この設定が無効になっている場合は、レプリカの raft_configuration
セクションを手動で変更することでクラスタを再構成できます。変更はリーダーのみが適用するため、必ずすべてのレプリカ上のファイルを編集してください。
あるいは、任意の ZooKeeper 互換クライアントを介して reconfig クエリを送信することもできます。
仮想ノード /keeper/config には、次の形式で直近にコミットされたクラスタ構成が格納されています。
- 各サーバーエントリは改行で区切られます。
server_typeはparticipantかlearnerのいずれかです(learnerはリーダー選出に参加しません)。server_priorityは、リーダー選出時にどのノードを優先すべきか を指定する 0 以上の整数です。 プライオリティが 0 の場合、そのサーバーがリーダーになることはありません。
例:
reconfig コマンドを使用すると、新しいサーバーの追加、既存サーバーの削除、および既存サーバーの優先順位の変更が行えます。次に、clickhouse-keeper-client を使用した例を示します:
また、kazoo の例は次のとおりです。
既存サーバーの優先度を 8 に変更
reconfig(joining="server.5=localhost:5123;participant;8", leaving=None)
単一ノードの keeper をクラスタに変換する
実験用の単一ノード keeper をクラスタに拡張したい場合があります。3 ノードのクラスタの場合の手順を次に示します。
- 重要: 新しいノードは、現在のクォーラムより少ない台数のグループで追加する必要があります。そうしないと、新ノード同士でリーダーが選出されてしまいます。この例では 1 台ずつ追加します。
- 既存の keeper ノードでは、設定パラメータ
keeper_server.enable_reconfigurationを有効にしておく必要があります。 - keeper クラスタの新しい完全な設定を用いて 2 台目のノードを起動します。
- 起動後、
reconfigを使ってノード 1 にこのノードを追加します。 - 続いて 3 台目のノードを起動し、
reconfigを使って追加します。 clickhouse-serverの設定を更新し、新しい keeper ノードを追加してから、変更を反映するために再起動します。- ノード 1 の Raft の設定を更新し、必要に応じて再起動します。
この手順に慣れるための sandbox リポジトリ を用意しています。
未サポートの機能
ClickHouse Keeper は ZooKeeper との完全な互換性を目指していますが、現時点では一部の機能が未実装となっています(開発は進行中です):
createはStatオブジェクトを返すことをサポートしていませんcreateは TTL をサポートしていませんaddWatchはPERSISTENTウォッチでは動作しませんremoveWatchおよびremoveAllWatchesはサポートされていませんsetWatchesはサポートされていませんCONTAINER型の znode の作成はサポートされていませんSASL authenticationはサポートされていません