ClickHouse/tests/integration/test_drop_replica/test.py

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

232 lines
8.4 KiB
Python
Raw Normal View History

import time
import pytest
from helpers.cluster import ClickHouseCluster
def fill_nodes(nodes, shard):
for node in nodes:
node.query(
"""
CREATE DATABASE test;
CREATE TABLE test.test_table(date Date, id UInt32)
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;
""".format(
shard=shard, replica=node.name
)
)
node.query(
"""
CREATE DATABASE test1;
CREATE TABLE test1.test_table(date Date, id UInt32)
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;
""".format(
shard=shard, replica=node.name
)
)
node.query(
"""
CREATE DATABASE test2;
CREATE TABLE test2.test_table(date Date, id UInt32)
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;
""".format(
shard=shard, replica=node.name
)
)
node.query(
"""
CREATE DATABASE test3;
CREATE TABLE test3.test_table(date Date, id UInt32)
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;
""".format(
shard=shard, replica=node.name
)
)
node.query(
"""
CREATE DATABASE test4;
CREATE TABLE test4.test_table(date Date, id UInt32)
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;
""".format(
shard=shard, replica=node.name
)
)
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"]
)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
fill_nodes([node_1_1, node_1_2], 1)
yield cluster
except Exception as ex:
2020-10-02 16:54:07 +00:00
print(ex)
finally:
cluster.shutdown()
2023-03-30 22:42:16 +00:00
def check_exists(zk, path):
zk.sync(path)
return zk.exists(path)
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)"
)
zk = cluster.get_kazoo_client("zoo1")
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"
)
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"
)
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"
)
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
)
)
2020-06-23 12:01:51 +00:00
assert "There is a local table" in node_1_2.query_and_get_error(
"SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format(
shard=1
)
)
2020-06-23 12:01:51 +00:00
assert "There is a local table" in node_1_1.query_and_get_error(
"SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format(
shard=1
)
)
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'"
)
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))
Added suggestions for mistyped names for db and tables with different… (#49801) * Added suggestions for mistyped names for db and tables with different scenarios commented * fixed bugs * fixed style check * fixed errors * fixed errors * fixed error with exceptions * fixed exceptions * fixed exceptions * fixed exceptions * added test and fixed bugs * fixed style check * fixed style check * fixed style check * fixed check black * Update test.py * Fixed server crash * Fixed server crash and style check * Fixed style check * Fixed style check * Fixed style check * Fixed bugs with drop_db * fixed fast test * added tests * fixed style check * fixed style check * fixed bug with lock_db * fixed bug with lock_db and fixed reviews * fixed bug with lock_db and fixed reviews * fixed style check * fixed fast test * fixed fast test * revert to fd582a2 * revert to fd582a2 * Removed unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Remove unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * resolved arguments issue in assertDatabaseExists * fixing fast test * fixed fast test * fixed stateless test for default db * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Fixing tests. * resolved problem with mutex * Fixed mutex in assertDatabaseExists * changes about assertDatabaseExists * fixed bugs with file types * fixed string types * fixed fast test * fixed mutex * fixed mutex * fixed style check * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * fixed build * added -unlocked versions of functions * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * changed usage of assertDatabaseExistsUnlocked() * fixed style check * style check * style check * Revert "style check" This reverts commit 28a9ee85a0a801438f05ca8c6604d67b8c132e69. * Merge branch 'master' into hints-for-wrong-db-or-table-name * Changed AssertDatabaseExists and unified exception output * resolved proposed changes and modified tests * Revert "resolved proposed changes and modified tests" This reverts commit d45337d65c16bd55b6bbb5c29023a1d0e5e1a38e. * resolved requested changes * fixed tests * fixed tests * fixed check black * Update include brackets Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update suggested changes * Update suggested changes * Fixed style check * Added test to analyzer_integration_broken * Update DatabaseCatalog.cpp * Update test.py * fixed test * Revert "fixed test" This reverts commit ca6d4c17c85b2bebba56ebc24dceac0e425afa0e. * fixed test * Revert "fixed test" This reverts commit fe6d0d9c865fcfe030ea63f370d2930a3bb3b53a, reversing changes made to 22f44967041fbd32f5f8884daa6f8736247f6fd8. * Update test.py * fixed black check * Update test.py * fixed long_log_tinylog_deadlock_race * Update DatabaseCatalog.cpp * Update test.py * style * Update DatabaseCatalog.cpp * Fixed test * implemented for IDatabase * Style check * removed const_cast * Update DatabaseCatalog.h * Update DatabaseCatalog.h * Update DatabaseCatalog.cpp * Update DatabaseCatalog.cpp * Added db name to hints * Update 00002_log_and_exception_messages_formatting.sql * Update 00002_log_and_exception_messages_formatting.sql --------- Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com>
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"
)
Added suggestions for mistyped names for db and tables with different… (#49801) * Added suggestions for mistyped names for db and tables with different scenarios commented * fixed bugs * fixed style check * fixed errors * fixed errors * fixed error with exceptions * fixed exceptions * fixed exceptions * fixed exceptions * added test and fixed bugs * fixed style check * fixed style check * fixed style check * fixed check black * Update test.py * Fixed server crash * Fixed server crash and style check * Fixed style check * Fixed style check * Fixed style check * Fixed bugs with drop_db * fixed fast test * added tests * fixed style check * fixed style check * fixed bug with lock_db * fixed bug with lock_db and fixed reviews * fixed bug with lock_db and fixed reviews * fixed style check * fixed fast test * fixed fast test * revert to fd582a2 * revert to fd582a2 * Removed unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Remove unused parameters Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * resolved arguments issue in assertDatabaseExists * fixing fast test * fixed fast test * fixed stateless test for default db * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Fixing tests. * resolved problem with mutex * Fixed mutex in assertDatabaseExists * changes about assertDatabaseExists * fixed bugs with file types * fixed string types * fixed fast test * fixed mutex * fixed mutex * fixed style check * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update src/Interpreters/DatabaseCatalog.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * fixed build * added -unlocked versions of functions * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * Revert "fixed build" This reverts commit 8ce961be210c3cfbca35e0d5fb9f86b2ec42ff78. * changed usage of assertDatabaseExistsUnlocked() * fixed style check * style check * style check * Revert "style check" This reverts commit 28a9ee85a0a801438f05ca8c6604d67b8c132e69. * Merge branch 'master' into hints-for-wrong-db-or-table-name * Changed AssertDatabaseExists and unified exception output * resolved proposed changes and modified tests * Revert "resolved proposed changes and modified tests" This reverts commit d45337d65c16bd55b6bbb5c29023a1d0e5e1a38e. * resolved requested changes * fixed tests * fixed tests * fixed check black * Update include brackets Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> * Update suggested changes * Update suggested changes * Fixed style check * Added test to analyzer_integration_broken * Update DatabaseCatalog.cpp * Update test.py * fixed test * Revert "fixed test" This reverts commit ca6d4c17c85b2bebba56ebc24dceac0e425afa0e. * fixed test * Revert "fixed test" This reverts commit fe6d0d9c865fcfe030ea63f370d2930a3bb3b53a, reversing changes made to 22f44967041fbd32f5f8884daa6f8736247f6fd8. * Update test.py * fixed black check * Update test.py * fixed long_log_tinylog_deadlock_race * Update DatabaseCatalog.cpp * Update test.py * style * Update DatabaseCatalog.cpp * Fixed test * implemented for IDatabase * Style check * removed const_cast * Update DatabaseCatalog.h * Update DatabaseCatalog.h * Update DatabaseCatalog.cpp * Update DatabaseCatalog.cpp * Added db name to hints * Update 00002_log_and_exception_messages_formatting.sql * Update 00002_log_and_exception_messages_formatting.sql --------- Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com>
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
)
)
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
)
)
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