ClickHouse/tests/integration/test_recovery_replica/test.py

77 lines
2.8 KiB
Python
Raw Normal View History

2018-08-29 10:04:41 +00:00
import time
import pytest
2018-08-29 10:04:41 +00:00
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
2018-08-29 10:04:41 +00:00
2018-08-29 10:04:41 +00:00
def fill_nodes(nodes, shard):
for node in nodes:
node.query(
'''
CREATE TABLE test_table(date Date, id UInt32)
2020-10-06 20:05:28 +00:00
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/replicated', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
'''.format(shard=shard, replica=node.name))
2018-08-29 10:04:41 +00:00
cluster = ClickHouseCluster(__file__)
2020-10-06 20:05:28 +00:00
node1 = cluster.add_instance('node1', with_zookeeper=True)
node2 = cluster.add_instance('node2', with_zookeeper=True)
node3 = cluster.add_instance('node3', with_zookeeper=True)
2018-08-29 10:04:41 +00:00
2018-08-29 10:04:41 +00:00
@pytest.fixture(scope="module")
2018-08-29 13:47:48 +00:00
def start_cluster():
2018-08-29 10:04:41 +00:00
try:
cluster.start()
2020-10-06 20:05:28 +00:00
fill_nodes([node1, node2, node3], 1)
2018-08-29 10:04:41 +00:00
yield cluster
2018-08-29 13:47:48 +00:00
except Exception as ex:
2020-10-02 16:54:07 +00:00
print(ex)
2018-08-29 13:47:48 +00:00
2018-08-29 10:04:41 +00:00
finally:
cluster.shutdown()
2018-08-29 13:47:48 +00:00
def test_recovery(start_cluster):
2018-08-29 10:04:41 +00:00
node1.query("INSERT INTO test_table VALUES (1, 1)")
time.sleep(1)
node2.query("DETACH TABLE test_table")
for i in range(100):
node1.query("INSERT INTO test_table VALUES (1, {})".format(i))
node2.query_with_retry("ATTACH TABLE test_table",
check_callback=lambda x: len(node2.query("select * from test_table")) > 0)
2018-08-29 10:04:41 +00:00
assert_eq_with_retry(node2, "SELECT count(*) FROM test_table", node1.query("SELECT count(*) FROM test_table"))
2020-10-06 20:05:28 +00:00
lost_marker = "Will mark replica node2 as lost"
assert node1.contains_in_log(lost_marker) or node3.contains_in_log(lost_marker)
def test_choose_source_replica(start_cluster):
node3.query("INSERT INTO test_table VALUES (2, 1)")
time.sleep(1)
node2.query("DETACH TABLE test_table")
node1.query("SYSTEM STOP FETCHES test_table") # node1 will have many entries in queue, so node2 will clone node3
for i in range(100):
node3.query("INSERT INTO test_table VALUES (2, {})".format(i))
node2.query_with_retry("ATTACH TABLE test_table",
check_callback=lambda x: len(node2.query("select * from test_table")) > 0)
node1.query("SYSTEM START FETCHES test_table")
node1.query("SYSTEM SYNC REPLICA test_table")
node2.query("SYSTEM SYNC REPLICA test_table")
assert node1.query("SELECT count(*) FROM test_table") == node3.query("SELECT count(*) FROM test_table")
assert node2.query("SELECT count(*) FROM test_table") == node3.query("SELECT count(*) FROM test_table")
lost_marker = "Will mark replica node2 as lost"
assert node1.contains_in_log(lost_marker) or node3.contains_in_log(lost_marker)
assert node2.contains_in_log("Will mimic node3")