ClickHouse/tests/integration/test_reload_zookeeper/test.py

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

104 lines
3.1 KiB
Python
Raw Normal View History

2024-09-27 10:19:39 +00:00
import os
import time
2024-09-27 10:19:39 +00:00
import pytest
from helpers.client import QueryRuntimeException
2024-09-27 10:19:39 +00:00
from helpers.cluster import ClickHouseCluster
from helpers.keeper_utils import get_active_zk_connections
from helpers.test_tools import assert_eq_with_retry
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)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
node.query(
"""
CREATE TABLE test_table(date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/shard1/test/test_table', '1')
PARTITION BY toYYYYMM(date)
ORDER BY id
"""
)
yield cluster
finally:
cluster.shutdown()
def test_reload_zookeeper(start_cluster):
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 = """
<clickhouse>
<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-11-03 09:32:31 +00:00
node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config)
2020-11-27 11:05:59 +00:00
node.query("SYSTEM RELOAD CONFIG")
## 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")
with pytest.raises(QueryRuntimeException):
2020-11-23 14:24:32 +00:00
node.query(
"SELECT COUNT() FROM test_table",
settings={"select_sequential_consistency": 1},
)
## start zoo2, zoo3, table will be readonly too, because it only connect to zoo1
cluster.start_zookeeper_nodes(["zoo2", "zoo3"])
cluster.wait_zookeeper_nodes_to_start(["zoo2", "zoo3"])
2020-11-23 14:24:32 +00:00
node.query("SELECT COUNT() FROM test_table")
with pytest.raises(QueryRuntimeException):
2020-11-23 14:24:32 +00:00
node.query(
"SELECT COUNT() FROM test_table",
settings={"select_sequential_consistency": 1},
)
## set config to zoo2, server will be normal
2020-11-03 11:40:45 +00:00
new_config = """
<clickhouse>
<zookeeper>
<node index="1">
<host>zoo2</host>
<port>2181</port>
</node>
<session_timeout_ms>2000</session_timeout_ms>
</zookeeper>
</clickhouse>
"""
2020-11-03 09:32:31 +00:00
node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config)
2020-11-27 11:05:59 +00:00
node.query("SYSTEM RELOAD CONFIG")
2020-11-03 09:32:31 +00:00
active_zk_connections = get_active_zk_connections(node)
assert (
len(active_zk_connections) == 1
), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections)
assert_eq_with_retry(
node, "SELECT COUNT() FROM test_table", "1000", retry_count=120, sleep_time=0.5
)
active_zk_connections = get_active_zk_connections(node)
assert (
len(active_zk_connections) == 1
), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections)