ClickHouse/tests/integration/test_restore_replica/test.py

157 lines
5.3 KiB
Python
Raw Normal View History

SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.cluster import ClickHouseKiller
from helpers.test_tools import assert_eq_with_retry
from helpers.network import PartitionManager
def fill_nodes(nodes):
for node in nodes:
node.query(
'''
CREATE TABLE test(n UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/', '{replica}')
ORDER BY n PARTITION BY n % 10;
'''.format(replica=node.name))
cluster = ClickHouseCluster(__file__)
configs =["configs/remote_servers.xml"]
node_1 = cluster.add_instance('replica1', with_zookeeper=True, main_configs=configs)
node_2 = cluster.add_instance('replica2', with_zookeeper=True, main_configs=configs)
node_3 = cluster.add_instance('replica3', with_zookeeper=True, main_configs=configs)
nodes = [node_1, node_2, node_3]
def fill_table():
node_1.query("TRUNCATE TABLE test")
for node in nodes:
node.query("SYSTEM SYNC REPLICA test")
check_data(0, 0)
# it will create multiple parts in each partition and probably cause merges
node_1.query("INSERT INTO test SELECT number + 0 FROM numbers(200)")
node_1.query("INSERT INTO test SELECT number + 200 FROM numbers(200)")
node_1.query("INSERT INTO test SELECT number + 400 FROM numbers(200)")
node_1.query("INSERT INTO test SELECT number + 600 FROM numbers(200)")
node_1.query("INSERT INTO test SELECT number + 800 FROM numbers(200)")
check_data(499500, 1000)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
fill_nodes(nodes)
yield cluster
except Exception as ex:
print(ex)
finally:
cluster.shutdown()
def check_data(_sum: int, count: int) -> None:
res = "{}\t{}\n".format(_sum, count)
assert_eq_with_retry(node_1, "SELECT sum(n), count() FROM test", res)
assert_eq_with_retry(node_2, "SELECT sum(n), count() FROM test", res)
assert_eq_with_retry(node_3, "SELECT sum(n), count() FROM test", res)
def check_after_restoration():
check_data(1999000, 2000)
for node in nodes:
node.query_and_get_error("SYSTEM RESTORE REPLICA test")
def test_restore_replica_invalid_tables(start_cluster):
print("Checking the invocation on non-existent and non-replicated tables")
node_1.query_and_get_error("SYSTEM RESTORE REPLICA i_dont_exist_42")
node_1.query_and_get_error("SYSTEM RESTORE REPLICA no_db.i_dont_exist_42")
node_1.query_and_get_error("SYSTEM RESTORE REPLICA system.numbers")
def test_restore_replica_sequential(start_cluster):
zk = cluster.get_kazoo_client('zoo1')
fill_table()
print("Deleting root ZK path metadata")
zk.delete("/clickhouse/tables/test", recursive=True)
assert zk.exists("/clickhouse/tables/test") is None
node_1.query("SYSTEM RESTART REPLICA test")
node_1.query_and_get_error("INSERT INTO test SELECT number AS num FROM numbers(1000,2000) WHERE num % 2 = 0")
print("Restoring replica1")
node_1.query("SYSTEM RESTORE REPLICA test")
assert zk.exists("/clickhouse/tables/test")
check_data(499500, 1000)
node_1.query("INSERT INTO test SELECT number + 1000 FROM numbers(1000)")
print("Restoring other replicas")
node_2.query("SYSTEM RESTART REPLICA test")
node_2.query("SYSTEM RESTORE REPLICA test")
node_3.query("SYSTEM RESTART REPLICA test")
node_3.query("SYSTEM RESTORE REPLICA test")
node_2.query("SYSTEM SYNC REPLICA test")
node_3.query("SYSTEM SYNC REPLICA test")
check_after_restoration()
def test_restore_replica_parallel(start_cluster):
zk = cluster.get_kazoo_client('zoo1')
fill_table()
print("Deleting root ZK path metadata")
zk.delete("/clickhouse/tables/test", recursive=True)
assert zk.exists("/clickhouse/tables/test") is None
node_1.query("SYSTEM RESTART REPLICA test")
node_1.query_and_get_error("INSERT INTO test SELECT number AS num FROM numbers(1000,2000) WHERE num % 2 = 0")
print("Restoring replicas in parallel")
node_2.query("SYSTEM RESTART REPLICA test")
node_3.query("SYSTEM RESTART REPLICA test")
node_1.query("SYSTEM RESTORE REPLICA test ON CLUSTER test_cluster")
assert zk.exists("/clickhouse/tables/test")
check_data(499500, 1000)
node_1.query("INSERT INTO test SELECT number + 1000 FROM numbers(1000)")
check_after_restoration()
def test_restore_replica_alive_replicas(start_cluster):
zk = cluster.get_kazoo_client('zoo1')
fill_table()
print("Deleting replica2 path, trying to restore replica1")
zk.delete("/clickhouse/tables/test/replicas/replica2", recursive=True)
assert zk.exists("/clickhouse/tables/test/replicas/replica2") is None
node_1.query_and_get_error("SYSTEM RESTORE REPLICA test")
print("Deleting replica1 path, trying to restore replica1")
zk.delete("/clickhouse/tables/test/replicas/replica1", recursive=True)
assert zk.exists("/clickhouse/tables/test/replicas/replica1") is None
node_1.query("SYSTEM RESTART REPLICA test")
node_1.query("SYSTEM RESTORE REPLICA test")
node_2.query("SYSTEM RESTART REPLICA test")
node_2.query("SYSTEM RESTORE REPLICA test")
check_data(499500, 1000)
node_1.query("INSERT INTO test SELECT number + 1000 FROM numbers(1000)")
node_2.query("SYSTEM SYNC REPLICA test")
node_3.query("SYSTEM SYNC REPLICA test")
check_after_restoration()