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.

129 lines
3.9 KiB
Python
Raw Normal View History

import time
import pytest
import os
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
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):
def wait_zookeeper_node_to_start(zk_nodes, timeout=60):
start = time.time()
while time.time() - start < timeout:
try:
for instance in zk_nodes:
conn = start_cluster.get_kazoo_client(instance)
conn.get_children("/")
print("All instances of ZooKeeper started")
return
except Exception as ex:
2020-10-02 16:54:07 +00:00
print(("Can't connect to ZooKeeper " + str(ex)))
time.sleep(0.5)
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"])
wait_zookeeper_node_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},
)
def get_active_zk_connections():
return str(
node.exec_in_container(
[
"bash",
"-c",
"lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED | wc -l",
],
privileged=True,
user="root",
)
).strip()
## 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()
assert (
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()
assert (
active_zk_connections == "1"
), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections)