ClickHouse/tests/integration/test_restore_replica/test.py
Mike Kot 4c391f8e99
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 11:24:43 +03:00

157 lines
5.3 KiB
Python

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()