Even more stable

This commit is contained in:
alesapin 2021-02-22 10:33:27 +03:00
parent 1aa0c0bff0
commit 74a07e406b

View File

@ -14,6 +14,18 @@ node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3
from kazoo.client import KazooClient, KazooState
"""
In this test, we blockade RAFT leader and check that the whole system is
able to recover. It's not a good test because we use ClickHouse's replicated
tables to check connectivity, but they may require special operations (or a long
wait) after session expiration. We don't use kazoo, because this client pretends
to be very smart: SUSPEND sessions, try to recover them, and so on. The test
will be even less predictable than with ClickHouse tables.
TODO find (or write) not so smart python client.
TODO remove this when jepsen tests will be written.
"""
@pytest.fixture(scope="module")
def started_cluster():
try:
@ -66,10 +78,16 @@ def get_fake_zk(nodename, timeout=30.0):
# in extremely rare case it can take more than 5 minutes in debug build with sanitizer
@pytest.mark.timeout(600)
def test_blocade_leader(started_cluster):
wait_nodes()
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary")
node.query("CREATE TABLE ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1))
for i in range(100):
wait_nodes()
try:
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary")
node.query("CREATE TABLE IF NOT EXISTS ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1))
break
except Exception as ex:
print("Got exception from node", smaller_exception(ex))
time.sleep(0.1)
node2.query("INSERT INTO ordinary.t1 SELECT number FROM numbers(10)")
@ -209,10 +227,16 @@ def restart_replica_for_sure(node, table_name, zk_replica_path):
# in extremely rare case it can take more than 5 minutes in debug build with sanitizer
@pytest.mark.timeout(600)
def test_blocade_leader_twice(started_cluster):
wait_nodes()
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary")
node.query("CREATE TABLE ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1))
for i in range(100):
wait_nodes()
try:
for i, node in enumerate([node1, node2, node3]):
node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary")
node.query("CREATE TABLE IF NOT EXISTS ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1))
break
except Exception as ex:
print("Got exception from node", smaller_exception(ex))
time.sleep(0.1)
node2.query("INSERT INTO ordinary.t2 SELECT number FROM numbers(10)")