2024-10-02 15:53:16 +00:00
|
|
|
from os import path as p
|
2024-09-27 10:19:39 +00:00
|
|
|
|
2020-09-10 04:00:33 +00:00
|
|
|
import pytest
|
|
|
|
|
|
|
|
from helpers.client import QueryRuntimeException
|
2024-09-27 10:19:39 +00:00
|
|
|
from helpers.cluster import ClickHouseCluster
|
2024-10-02 15:53:16 +00:00
|
|
|
from helpers.keeper_utils import (
|
|
|
|
get_active_zk_connections,
|
|
|
|
replace_zookeeper_config,
|
|
|
|
reset_zookeeper_config,
|
|
|
|
)
|
2020-09-10 04:00:33 +00:00
|
|
|
from helpers.test_tools import assert_eq_with_retry
|
2024-10-02 15:53:16 +00:00
|
|
|
from helpers.utility import random_string
|
2020-09-10 04:00:33 +00:00
|
|
|
|
2024-10-02 15:53:16 +00:00
|
|
|
default_zk_config = p.join(p.dirname(p.realpath(__file__)), "configs/zookeeper.xml")
|
2020-09-10 04:00:33 +00:00
|
|
|
cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml")
|
2020-11-23 14:24:32 +00:00
|
|
|
node = cluster.add_instance("node", with_zookeeper=True)
|
2020-09-10 04:00:33 +00:00
|
|
|
|
|
|
|
|
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def start_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
|
|
|
|
|
|
|
yield cluster
|
|
|
|
finally:
|
|
|
|
cluster.shutdown()
|
|
|
|
|
|
|
|
|
|
|
|
def test_reload_zookeeper(start_cluster):
|
2024-10-02 15:53:16 +00:00
|
|
|
# random is used for flaky tests, where ZK is not fast enough to clear the node
|
|
|
|
node.query(
|
|
|
|
f"""
|
|
|
|
CREATE TABLE test_table(date Date, id UInt32)
|
|
|
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/shard1/{random_string(7)}/test_table', '1')
|
|
|
|
PARTITION BY toYYYYMM(date)
|
|
|
|
ORDER BY id
|
|
|
|
"""
|
|
|
|
)
|
2020-09-10 04:00:33 +00:00
|
|
|
node.query(
|
|
|
|
"INSERT INTO test_table(date, id) select today(), number FROM numbers(1000)"
|
|
|
|
)
|
|
|
|
|
|
|
|
## remove zoo2, zoo3 from configs
|
2020-11-03 11:40:45 +00:00
|
|
|
new_config = """
|
2021-09-25 04:08:34 +00:00
|
|
|
<clickhouse>
|
2020-09-10 04:00:33 +00:00
|
|
|
<zookeeper>
|
|
|
|
<node index="1">
|
|
|
|
<host>zoo1</host>
|
|
|
|
<port>2181</port>
|
|
|
|
</node>
|
|
|
|
<session_timeout_ms>2000</session_timeout_ms>
|
|
|
|
</zookeeper>
|
2021-09-25 17:07:48 +00:00
|
|
|
</clickhouse>
|
2020-09-10 04:00:33 +00:00
|
|
|
"""
|
2024-10-02 15:53:16 +00:00
|
|
|
replace_zookeeper_config(node, new_config)
|
2020-09-10 04:00:33 +00:00
|
|
|
## config reloads, but can still work
|
|
|
|
assert_eq_with_retry(
|
|
|
|
node, "SELECT COUNT() FROM test_table", "1000", retry_count=120, sleep_time=0.5
|
|
|
|
)
|
|
|
|
|
|
|
|
## stop all zookeepers, table will be readonly
|
|
|
|
cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"])
|
2020-11-23 14:24:32 +00:00
|
|
|
node.query("SELECT COUNT() FROM test_table")
|
2020-09-10 04:00:33 +00:00
|
|
|
with pytest.raises(QueryRuntimeException):
|
2020-11-23 14:24:32 +00:00
|
|
|
node.query(
|
|
|
|
"SELECT COUNT() FROM test_table",
|
|
|
|
settings={"select_sequential_consistency": 1},
|
|
|
|
)
|
2020-09-10 04:00:33 +00:00
|
|
|
|
|
|
|
## start zoo2, zoo3, table will be readonly too, because it only connect to zoo1
|
|
|
|
cluster.start_zookeeper_nodes(["zoo2", "zoo3"])
|
2024-10-02 15:09:11 +00:00
|
|
|
cluster.wait_zookeeper_nodes_to_start(["zoo2", "zoo3"])
|
2020-11-23 14:24:32 +00:00
|
|
|
node.query("SELECT COUNT() FROM test_table")
|
2020-09-10 04:00:33 +00:00
|
|
|
with pytest.raises(QueryRuntimeException):
|
2020-11-23 14:24:32 +00:00
|
|
|
node.query(
|
|
|
|
"SELECT COUNT() FROM test_table",
|
|
|
|
settings={"select_sequential_consistency": 1},
|
|
|
|
)
|
2020-09-10 04:00:33 +00:00
|
|
|
|
|
|
|
## set config to zoo2, server will be normal
|
2020-11-03 11:40:45 +00:00
|
|
|
new_config = """
|
2021-09-25 04:08:34 +00:00
|
|
|
<clickhouse>
|
2020-09-10 04:00:33 +00:00
|
|
|
<zookeeper>
|
|
|
|
<node index="1">
|
|
|
|
<host>zoo2</host>
|
|
|
|
<port>2181</port>
|
|
|
|
</node>
|
|
|
|
<session_timeout_ms>2000</session_timeout_ms>
|
|
|
|
</zookeeper>
|
2021-09-25 04:08:34 +00:00
|
|
|
</clickhouse>
|
2020-09-10 04:00:33 +00:00
|
|
|
"""
|
2024-10-02 15:53:16 +00:00
|
|
|
replace_zookeeper_config(node, new_config)
|
2020-11-03 09:32:31 +00:00
|
|
|
|
2024-10-01 22:54:11 +00:00
|
|
|
active_zk_connections = get_active_zk_connections(node)
|
2021-02-26 19:05:25 +00:00
|
|
|
assert (
|
2024-10-01 22:54:11 +00:00
|
|
|
len(active_zk_connections) == 1
|
2021-02-26 19:05:25 +00:00
|
|
|
), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections)
|
|
|
|
|
2020-09-10 04:00:33 +00:00
|
|
|
assert_eq_with_retry(
|
|
|
|
node, "SELECT COUNT() FROM test_table", "1000", retry_count=120, sleep_time=0.5
|
|
|
|
)
|
|
|
|
|
2024-10-01 22:54:11 +00:00
|
|
|
active_zk_connections = get_active_zk_connections(node)
|
2021-02-26 19:05:25 +00:00
|
|
|
assert (
|
2024-10-01 22:54:11 +00:00
|
|
|
len(active_zk_connections) == 1
|
2021-02-26 19:05:25 +00:00
|
|
|
), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections)
|
2024-10-02 15:53:16 +00:00
|
|
|
# Reset cluster state
|
|
|
|
cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"])
|
|
|
|
cluster.wait_zookeeper_nodes_to_start(["zoo1", "zoo2", "zoo3"])
|
|
|
|
reset_zookeeper_config(node, default_zk_config)
|
|
|
|
node.query("DROP TABLE test_table")
|