2020-05-06 03:10:35 +00:00
|
|
|
import time
|
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
import pytest
|
2020-05-06 03:10:35 +00:00
|
|
|
from helpers.cluster import ClickHouseCluster
|
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
|
2020-05-06 03:10:35 +00:00
|
|
|
def fill_nodes(nodes, shard):
|
|
|
|
for node in nodes:
|
|
|
|
node.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
2020-09-16 04:26:10 +00:00
|
|
|
CREATE DATABASE test;
|
|
|
|
|
|
|
|
CREATE TABLE test.test_table(date Date, id UInt32)
|
2023-05-22 17:07:18 +00:00
|
|
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{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, cleanup_thread_preferred_points_per_iteration=0;
|
2020-09-16 04:26:10 +00:00
|
|
|
""".format(
|
|
|
|
shard=shard, replica=node.name
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-06-05 07:03:51 +00:00
|
|
|
|
|
|
|
node.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
2020-09-16 04:26:10 +00:00
|
|
|
CREATE DATABASE test1;
|
|
|
|
|
|
|
|
CREATE TABLE test1.test_table(date Date, id UInt32)
|
2023-05-22 17:07:18 +00:00
|
|
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test1/{shard}/replicated/test_table', '{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, cleanup_thread_preferred_points_per_iteration=0;
|
2020-09-16 04:26:10 +00:00
|
|
|
""".format(
|
|
|
|
shard=shard, replica=node.name
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-06-05 07:03:51 +00:00
|
|
|
|
|
|
|
node.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
2020-09-16 04:26:10 +00:00
|
|
|
CREATE DATABASE test2;
|
|
|
|
|
|
|
|
CREATE TABLE test2.test_table(date Date, id UInt32)
|
2023-05-22 17:07:18 +00:00
|
|
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test2/{shard}/replicated/test_table', '{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, cleanup_thread_preferred_points_per_iteration=0;
|
2020-09-16 04:26:10 +00:00
|
|
|
""".format(
|
|
|
|
shard=shard, replica=node.name
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-06-05 07:03:51 +00:00
|
|
|
|
|
|
|
node.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
2020-09-16 04:26:10 +00:00
|
|
|
CREATE DATABASE test3;
|
|
|
|
|
|
|
|
CREATE TABLE test3.test_table(date Date, id UInt32)
|
2023-05-22 17:07:18 +00:00
|
|
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test3/{shard}/replicated/test_table', '{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, cleanup_thread_preferred_points_per_iteration=0;
|
2020-09-16 04:26:10 +00:00
|
|
|
""".format(
|
|
|
|
shard=shard, replica=node.name
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-06-05 07:03:51 +00:00
|
|
|
|
|
|
|
node.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
2020-09-16 04:26:10 +00:00
|
|
|
CREATE DATABASE test4;
|
|
|
|
|
|
|
|
CREATE TABLE test4.test_table(date Date, id UInt32)
|
2023-05-22 17:07:18 +00:00
|
|
|
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test4/{shard}/replicated/test_table', '{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, cleanup_thread_preferred_points_per_iteration=0;
|
2020-09-16 04:26:10 +00:00
|
|
|
""".format(
|
|
|
|
shard=shard, replica=node.name
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-06-05 07:03:51 +00:00
|
|
|
|
2020-05-06 03:10:35 +00:00
|
|
|
|
|
|
|
cluster = ClickHouseCluster(__file__)
|
|
|
|
node_1_1 = cluster.add_instance(
|
|
|
|
"node_1_1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"]
|
|
|
|
)
|
|
|
|
node_1_2 = cluster.add_instance(
|
|
|
|
"node_1_2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"]
|
|
|
|
)
|
2020-05-17 12:44:22 +00:00
|
|
|
node_1_3 = cluster.add_instance(
|
|
|
|
"node_1_3", with_zookeeper=True, main_configs=["configs/remote_servers.xml"]
|
|
|
|
)
|
2020-05-06 03:10:35 +00:00
|
|
|
|
|
|
|
|
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def start_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
|
|
|
|
2020-06-05 07:03:51 +00:00
|
|
|
fill_nodes([node_1_1, node_1_2], 1)
|
2020-05-06 03:10:35 +00:00
|
|
|
|
|
|
|
yield cluster
|
|
|
|
|
|
|
|
except Exception as ex:
|
2020-10-02 16:54:07 +00:00
|
|
|
print(ex)
|
2020-05-06 03:10:35 +00:00
|
|
|
|
|
|
|
finally:
|
|
|
|
cluster.shutdown()
|
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
|
2023-03-30 22:42:16 +00:00
|
|
|
def check_exists(zk, path):
|
|
|
|
zk.sync(path)
|
|
|
|
return zk.exists(path)
|
|
|
|
|
|
|
|
|
2020-05-06 03:10:35 +00:00
|
|
|
def test_drop_replica(start_cluster):
|
2021-03-18 14:22:17 +00:00
|
|
|
node_1_1.query(
|
|
|
|
"INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)"
|
|
|
|
)
|
|
|
|
node_1_1.query(
|
|
|
|
"INSERT INTO test1.test_table SELECT number, toString(number) FROM numbers(100)"
|
|
|
|
)
|
|
|
|
node_1_1.query(
|
|
|
|
"INSERT INTO test2.test_table SELECT number, toString(number) FROM numbers(100)"
|
|
|
|
)
|
|
|
|
node_1_1.query(
|
|
|
|
"INSERT INTO test3.test_table SELECT number, toString(number) FROM numbers(100)"
|
|
|
|
)
|
|
|
|
node_1_1.query(
|
|
|
|
"INSERT INTO test4.test_table SELECT number, toString(number) FROM numbers(100)"
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
|
|
|
|
2020-05-06 03:10:35 +00:00
|
|
|
zk = cluster.get_kazoo_client("zoo1")
|
2020-09-16 04:26:10 +00:00
|
|
|
assert "can't drop local replica" in node_1_1.query_and_get_error(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1'"
|
|
|
|
)
|
|
|
|
assert "can't drop local replica" in node_1_1.query_and_get_error(
|
2021-03-18 16:59:51 +00:00
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test"
|
2020-09-16 04:26:10 +00:00
|
|
|
)
|
|
|
|
assert "can't drop local replica" in node_1_1.query_and_get_error(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table"
|
2020-06-05 07:03:51 +00:00
|
|
|
)
|
|
|
|
assert "it's active" in node_1_2.query_and_get_error(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1'"
|
|
|
|
)
|
|
|
|
assert "it's active" in node_1_2.query_and_get_error(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test"
|
|
|
|
)
|
|
|
|
assert "it's active" in node_1_2.query_and_get_error(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table"
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-09-16 04:26:10 +00:00
|
|
|
assert "it's active" in node_1_3.query_and_get_error(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format(
|
|
|
|
shard=1
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-06-23 12:01:51 +00:00
|
|
|
assert "There is a local table" in node_1_2.query_and_get_error(
|
2020-09-16 04:26:10 +00:00
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format(
|
|
|
|
shard=1
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-06-23 12:01:51 +00:00
|
|
|
assert "There is a local table" in node_1_1.query_and_get_error(
|
2020-09-16 04:26:10 +00:00
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format(
|
|
|
|
shard=1
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2020-06-23 12:01:51 +00:00
|
|
|
assert "does not look like a table path" in node_1_3.query_and_get_error(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test'"
|
|
|
|
)
|
2020-05-07 01:14:15 +00:00
|
|
|
|
2021-03-18 16:59:51 +00:00
|
|
|
node_1_1.query("DETACH DATABASE test")
|
|
|
|
for i in range(1, 5):
|
|
|
|
node_1_1.query("DETACH DATABASE test{}".format(i))
|
|
|
|
|
2023-08-11 10:24:16 +00:00
|
|
|
assert "does not exist" in node_1_3.query_and_get_error(
|
2021-03-18 16:59:51 +00:00
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table"
|
|
|
|
)
|
|
|
|
|
2023-08-11 10:24:16 +00:00
|
|
|
assert "does not exist" in node_1_3.query_and_get_error(
|
2021-03-18 16:59:51 +00:00
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test1"
|
|
|
|
)
|
|
|
|
|
|
|
|
node_1_3.query("SYSTEM DROP REPLICA 'node_1_1'")
|
2023-03-30 22:42:16 +00:00
|
|
|
exists_replica_1_1 = check_exists(
|
|
|
|
zk,
|
2021-03-18 16:59:51 +00:00
|
|
|
"/clickhouse/tables/test3/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
|
|
shard=1, replica="node_1_1"
|
2023-03-30 22:42:16 +00:00
|
|
|
),
|
2021-03-18 16:59:51 +00:00
|
|
|
)
|
|
|
|
assert exists_replica_1_1 != None
|
|
|
|
|
|
|
|
## If you want to drop a inactive/stale replicate table that does not have a local replica, you can following syntax(ZKPATH):
|
|
|
|
node_1_3.query(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test2/{shard}/replicated/test_table'".format(
|
|
|
|
shard=1
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2023-03-30 22:42:16 +00:00
|
|
|
exists_replica_1_1 = check_exists(
|
|
|
|
zk,
|
2021-03-18 16:59:51 +00:00
|
|
|
"/clickhouse/tables/test2/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
|
|
shard=1, replica="node_1_1"
|
2023-03-30 22:42:16 +00:00
|
|
|
),
|
2021-03-18 16:59:51 +00:00
|
|
|
)
|
|
|
|
assert exists_replica_1_1 == None
|
|
|
|
|
|
|
|
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table")
|
2023-03-30 22:42:16 +00:00
|
|
|
exists_replica_1_1 = check_exists(
|
|
|
|
zk,
|
2021-03-18 16:59:51 +00:00
|
|
|
"/clickhouse/tables/test/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
|
|
shard=1, replica="node_1_1"
|
2023-03-30 22:42:16 +00:00
|
|
|
),
|
2021-03-18 16:59:51 +00:00
|
|
|
)
|
|
|
|
assert exists_replica_1_1 == None
|
|
|
|
|
|
|
|
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test1")
|
2023-03-30 22:42:16 +00:00
|
|
|
exists_replica_1_1 = check_exists(
|
|
|
|
zk,
|
2021-03-18 16:59:51 +00:00
|
|
|
"/clickhouse/tables/test1/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
|
|
shard=1, replica="node_1_1"
|
2023-03-30 22:42:16 +00:00
|
|
|
),
|
2021-03-18 16:59:51 +00:00
|
|
|
)
|
|
|
|
assert exists_replica_1_1 == None
|
|
|
|
|
|
|
|
node_1_3.query(
|
|
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test3/{shard}/replicated/test_table'".format(
|
|
|
|
shard=1
|
|
|
|
)
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2023-03-30 22:42:16 +00:00
|
|
|
exists_replica_1_1 = check_exists(
|
|
|
|
zk,
|
2021-03-18 16:59:51 +00:00
|
|
|
"/clickhouse/tables/test3/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
|
|
shard=1, replica="node_1_1"
|
2023-03-30 22:42:16 +00:00
|
|
|
),
|
2021-03-18 16:59:51 +00:00
|
|
|
)
|
|
|
|
assert exists_replica_1_1 == None
|
|
|
|
|
|
|
|
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1'")
|
2023-03-30 22:42:16 +00:00
|
|
|
exists_replica_1_1 = check_exists(
|
|
|
|
zk,
|
2021-03-18 16:59:51 +00:00
|
|
|
"/clickhouse/tables/test4/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
|
|
shard=1, replica="node_1_1"
|
2023-03-30 22:42:16 +00:00
|
|
|
),
|
2021-03-18 16:59:51 +00:00
|
|
|
)
|
|
|
|
assert exists_replica_1_1 == None
|