mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #71911 from ianton-ru/auxiliary_autodicovery
Auxiliary autodiscovery
This commit is contained in:
commit
59fe7e1951
@ -129,9 +129,11 @@ ClusterDiscovery::ClusterDiscovery(
|
||||
if (!config.has(cluster_config_prefix))
|
||||
continue;
|
||||
|
||||
String zk_root = config.getString(cluster_config_prefix + ".path");
|
||||
if (zk_root.empty())
|
||||
String zk_name_and_root = config.getString(cluster_config_prefix + ".path");
|
||||
if (zk_name_and_root.empty())
|
||||
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "ZooKeeper path for cluster '{}' is empty", key);
|
||||
String zk_root = zkutil::extractZooKeeperPath(zk_name_and_root, true);
|
||||
String zk_name = zkutil::extractZooKeeperName(zk_name_and_root);
|
||||
|
||||
const auto & password = config.getString(cluster_config_prefix + ".password", "");
|
||||
const auto & cluster_secret = config.getString(cluster_config_prefix + ".secret", "");
|
||||
@ -142,6 +144,7 @@ ClusterDiscovery::ClusterDiscovery(
|
||||
key,
|
||||
ClusterInfo(
|
||||
/* name_= */ key,
|
||||
/* zk_name_= */ zk_name,
|
||||
/* zk_root_= */ zk_root,
|
||||
/* host_name= */ config.getString(cluster_config_prefix + ".my_hostname", getFQDNOrHostName()),
|
||||
/* username= */ config.getString(cluster_config_prefix + ".user", context->getUserName()),
|
||||
@ -288,7 +291,7 @@ bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info)
|
||||
{
|
||||
LOG_DEBUG(log, "Updating cluster '{}'", cluster_info.name);
|
||||
|
||||
auto zk = context->getZooKeeper();
|
||||
auto zk = context->getDefaultOrAuxiliaryZooKeeper(cluster_info.zk_name);
|
||||
|
||||
int start_version;
|
||||
Strings node_uuids = getNodeNames(zk, cluster_info.zk_root, cluster_info.name, &start_version, false);
|
||||
@ -381,9 +384,9 @@ void ClusterDiscovery::initialUpdate()
|
||||
throw Exception(ErrorCodes::KEEPER_EXCEPTION, "Failpoint cluster_discovery_faults is triggered");
|
||||
});
|
||||
|
||||
auto zk = context->getZooKeeper();
|
||||
for (auto & [_, info] : clusters_info)
|
||||
{
|
||||
auto zk = context->getDefaultOrAuxiliaryZooKeeper(info.zk_name);
|
||||
registerInZk(zk, info);
|
||||
if (!updateCluster(info))
|
||||
{
|
||||
|
@ -67,6 +67,7 @@ private:
|
||||
struct ClusterInfo
|
||||
{
|
||||
const String name;
|
||||
const String zk_name;
|
||||
const String zk_root;
|
||||
NodesInfo nodes_info;
|
||||
|
||||
@ -88,6 +89,7 @@ private:
|
||||
String cluster_secret;
|
||||
|
||||
ClusterInfo(const String & name_,
|
||||
const String & zk_name_,
|
||||
const String & zk_root_,
|
||||
const String & host_name,
|
||||
const String & username_,
|
||||
@ -99,6 +101,7 @@ private:
|
||||
bool observer_mode,
|
||||
bool invisible)
|
||||
: name(name_)
|
||||
, zk_name(zk_name_)
|
||||
, zk_root(zk_root_)
|
||||
, current_node(host_name + ":" + toString(port), secure, shard_id)
|
||||
, current_node_is_observer(observer_mode)
|
||||
|
@ -1,11 +1,6 @@
|
||||
<clickhouse>
|
||||
<allow_experimental_cluster_discovery>1</allow_experimental_cluster_discovery>
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>/clickhouse/discovery/test_auto_cluster</path>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
<two_shards>
|
||||
<!-- just to check that there's no conflict between automatic and manual clusters -->
|
||||
<shard>
|
||||
|
@ -0,0 +1,9 @@
|
||||
<clickhouse>
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>/clickhouse/discovery/test_auto_cluster</path>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
@ -0,0 +1,9 @@
|
||||
<clickhouse>
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>zookeeper2:/clickhouse/discovery/test_auto_cluster</path>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
@ -0,0 +1,24 @@
|
||||
<clickhouse>
|
||||
<zookeeper>
|
||||
<node index="1">
|
||||
<host>zoo1</host>
|
||||
<port>2181</port>
|
||||
</node>
|
||||
<node index="2">
|
||||
<host>zoo2</host>
|
||||
<port>2181</port>
|
||||
</node>
|
||||
<node index="3">
|
||||
<host>zoo3</host>
|
||||
<port>2181</port>
|
||||
</node>
|
||||
</zookeeper>
|
||||
<auxiliary_zookeepers>
|
||||
<zookeeper2>
|
||||
<node index="1">
|
||||
<host>zoo1</host>
|
||||
<port>2181</port>
|
||||
</node>
|
||||
</zookeeper2>
|
||||
</auxiliary_zookeepers>
|
||||
</clickhouse>
|
@ -3,7 +3,6 @@
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>/clickhouse/discovery/test_auto_cluster</path>
|
||||
<observer/>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
|
@ -3,7 +3,6 @@
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>/clickhouse/discovery/test_auto_cluster</path>
|
||||
<shard>1</shard>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
|
@ -3,7 +3,6 @@
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>/clickhouse/discovery/test_auto_cluster</path>
|
||||
<shard>3</shard>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
|
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
<macros>
|
||||
<shard>shard0</shard>
|
||||
<replica>replica0</replica>
|
||||
</macros>
|
||||
</clickhouse>
|
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
<macros>
|
||||
<shard>shard1</shard>
|
||||
<replica>replica1</replica>
|
||||
</macros>
|
||||
</clickhouse>
|
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
<macros>
|
||||
<shard>shard2</shard>
|
||||
<replica>replica2</replica>
|
||||
</macros>
|
||||
</clickhouse>
|
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
<macros>
|
||||
<shard>shard3</shard>
|
||||
<replica>replica3</replica>
|
||||
</macros>
|
||||
</clickhouse>
|
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
<macros>
|
||||
<shard>shard4</shard>
|
||||
<replica>replica4</replica>
|
||||
</macros>
|
||||
</clickhouse>
|
@ -0,0 +1,6 @@
|
||||
<clickhouse>
|
||||
<macros>
|
||||
<shard>shard_o</shard>
|
||||
<replica>replica_o</replica>
|
||||
</macros>
|
||||
</clickhouse>
|
@ -20,7 +20,7 @@ shard_configs = {
|
||||
nodes = {
|
||||
node_name: cluster.add_instance(
|
||||
node_name,
|
||||
main_configs=[shard_config],
|
||||
main_configs=[shard_config, "config/config_discovery_path.xml"],
|
||||
stay_alive=True,
|
||||
with_zookeeper=True,
|
||||
)
|
||||
@ -119,3 +119,6 @@ def test_cluster_discovery_startup_and_stop(start_cluster):
|
||||
check_nodes_count(
|
||||
[nodes["node1"], nodes["node2"]], 2, cluster_name="two_shards", retries=1
|
||||
)
|
||||
|
||||
# cleanup
|
||||
nodes["node0"].query("DROP TABLE tbl ON CLUSTER 'test_auto_cluster' SYNC")
|
||||
|
@ -0,0 +1,127 @@
|
||||
import functools
|
||||
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
from .common import check_on_cluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
shard_configs = {
|
||||
"node0": ["config/config.xml", "config/macros0.xml"],
|
||||
"node1": ["config/config_shard1.xml", "config/macros1.xml"],
|
||||
"node2": ["config/config.xml", "config/macros2.xml"],
|
||||
"node3": ["config/config_shard3.xml", "config/macros3.xml"],
|
||||
"node4": ["config/config.xml", "config/macros4.xml"],
|
||||
"node_observer": ["config/config_observer.xml", "config/macros_o.xml"],
|
||||
}
|
||||
|
||||
nodes = {
|
||||
node_name: cluster.add_instance(
|
||||
node_name,
|
||||
main_configs=shard_config
|
||||
+ [
|
||||
"config/config_discovery_path_auxiliary_keeper.xml",
|
||||
"config/config_keepers.xml",
|
||||
],
|
||||
stay_alive=True,
|
||||
with_zookeeper=True,
|
||||
)
|
||||
for node_name, shard_config in shard_configs.items()
|
||||
}
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def test_cluster_discovery_with_auxiliary_keeper_startup_and_stop(start_cluster):
|
||||
"""
|
||||
Start cluster, check nodes count in system.clusters,
|
||||
then stop/start some nodes and check that it (dis)appeared in cluster.
|
||||
"""
|
||||
|
||||
check_nodes_count = functools.partial(
|
||||
check_on_cluster, what="count()", msg="Wrong nodes count in cluster"
|
||||
)
|
||||
check_shard_num = functools.partial(
|
||||
check_on_cluster,
|
||||
what="count(DISTINCT shard_num)",
|
||||
msg="Wrong shard_num count in cluster",
|
||||
)
|
||||
|
||||
total_shards = 3
|
||||
total_nodes = 5
|
||||
|
||||
check_nodes_count(
|
||||
[nodes["node0"], nodes["node2"], nodes["node_observer"]], total_nodes
|
||||
)
|
||||
check_shard_num(
|
||||
[nodes["node0"], nodes["node2"], nodes["node_observer"]], total_shards
|
||||
)
|
||||
|
||||
# test ON CLUSTER query
|
||||
nodes["node0"].query(
|
||||
"CREATE TABLE tbl ON CLUSTER 'test_auto_cluster' (x UInt64) ENGINE = ReplicatedMergeTree('zookeeper2:/clickhouse/{shard}/tbl', '{replica}') ORDER BY x"
|
||||
)
|
||||
nodes["node0"].query("INSERT INTO tbl VALUES (1)")
|
||||
nodes["node1"].query("INSERT INTO tbl VALUES (2)")
|
||||
|
||||
assert (
|
||||
int(
|
||||
nodes["node_observer"]
|
||||
.query(
|
||||
"SELECT sum(x) FROM clusterAllReplicas(test_auto_cluster, default.tbl)"
|
||||
)
|
||||
.strip()
|
||||
)
|
||||
== 3
|
||||
)
|
||||
|
||||
# Query SYSTEM DROP DNS CACHE may reload cluster configuration
|
||||
# check that it does not affect cluster discovery
|
||||
nodes["node1"].query("SYSTEM DROP DNS CACHE")
|
||||
nodes["node0"].query("SYSTEM DROP DNS CACHE")
|
||||
|
||||
check_shard_num(
|
||||
[nodes["node0"], nodes["node2"], nodes["node_observer"]], total_shards
|
||||
)
|
||||
|
||||
nodes["node1"].stop_clickhouse(kill=True)
|
||||
check_nodes_count(
|
||||
[nodes["node0"], nodes["node2"], nodes["node_observer"]], total_nodes - 1
|
||||
)
|
||||
|
||||
# node1 was the only node in shard '1'
|
||||
check_shard_num(
|
||||
[nodes["node0"], nodes["node2"], nodes["node_observer"]], total_shards - 1
|
||||
)
|
||||
|
||||
nodes["node3"].stop_clickhouse()
|
||||
check_nodes_count(
|
||||
[nodes["node0"], nodes["node2"], nodes["node_observer"]], total_nodes - 2
|
||||
)
|
||||
|
||||
nodes["node1"].start_clickhouse()
|
||||
check_nodes_count(
|
||||
[nodes["node0"], nodes["node2"], nodes["node_observer"]], total_nodes - 1
|
||||
)
|
||||
|
||||
nodes["node3"].start_clickhouse()
|
||||
check_nodes_count(
|
||||
[nodes["node0"], nodes["node2"], nodes["node_observer"]], total_nodes
|
||||
)
|
||||
|
||||
# regular cluster is not affected
|
||||
check_nodes_count(
|
||||
[nodes["node1"], nodes["node2"]], 2, cluster_name="two_shards", retries=1
|
||||
)
|
||||
|
||||
# cleanup
|
||||
nodes["node0"].query("DROP TABLE tbl ON CLUSTER 'test_auto_cluster' SYNC")
|
Loading…
Reference in New Issue
Block a user