mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
a6f89c0546
* 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 tofd582a2
* revert tofd582a2
* 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 commit8ce961be21
. * Revert "fixed build" This reverts commit8ce961be21
. * changed usage of assertDatabaseExistsUnlocked() * fixed style check * style check * style check * Revert "style check" This reverts commit28a9ee85a0
. * 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 commitd45337d65c
. * 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 commitca6d4c17c8
. * fixed test * Revert "fixed test" This reverts commitfe6d0d9c86
, reversing changes made to22f4496704
. * 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>
232 lines
8.4 KiB
Python
232 lines
8.4 KiB
Python
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"]
|
|
)
|
|
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:
|
|
print(ex)
|
|
|
|
finally:
|
|
cluster.shutdown()
|
|
|
|
|
|
def check_exists(zk, path):
|
|
zk.sync(path)
|
|
return zk.exists(path)
|
|
|
|
|
|
def test_drop_replica(start_cluster):
|
|
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(
|
|
"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
|
|
)
|
|
)
|
|
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
|
|
)
|
|
)
|
|
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
|
|
)
|
|
)
|
|
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'"
|
|
)
|
|
|
|
node_1_1.query("DETACH DATABASE test")
|
|
for i in range(1, 5):
|
|
node_1_1.query("DETACH DATABASE test{}".format(i))
|
|
|
|
assert "does not exist" in node_1_3.query_and_get_error(
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table"
|
|
)
|
|
|
|
assert "does not exist" in node_1_3.query_and_get_error(
|
|
"SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test1"
|
|
)
|
|
|
|
node_1_3.query("SYSTEM DROP REPLICA 'node_1_1'")
|
|
exists_replica_1_1 = check_exists(
|
|
zk,
|
|
"/clickhouse/tables/test3/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
shard=1, replica="node_1_1"
|
|
),
|
|
)
|
|
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
|
|
)
|
|
)
|
|
exists_replica_1_1 = check_exists(
|
|
zk,
|
|
"/clickhouse/tables/test2/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
shard=1, replica="node_1_1"
|
|
),
|
|
)
|
|
assert exists_replica_1_1 == None
|
|
|
|
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table")
|
|
exists_replica_1_1 = check_exists(
|
|
zk,
|
|
"/clickhouse/tables/test/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
shard=1, replica="node_1_1"
|
|
),
|
|
)
|
|
assert exists_replica_1_1 == None
|
|
|
|
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test1")
|
|
exists_replica_1_1 = check_exists(
|
|
zk,
|
|
"/clickhouse/tables/test1/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
shard=1, replica="node_1_1"
|
|
),
|
|
)
|
|
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
|
|
)
|
|
)
|
|
exists_replica_1_1 = check_exists(
|
|
zk,
|
|
"/clickhouse/tables/test3/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
shard=1, replica="node_1_1"
|
|
),
|
|
)
|
|
assert exists_replica_1_1 == None
|
|
|
|
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1'")
|
|
exists_replica_1_1 = check_exists(
|
|
zk,
|
|
"/clickhouse/tables/test4/{shard}/replicated/test_table/replicas/{replica}".format(
|
|
shard=1, replica="node_1_1"
|
|
),
|
|
)
|
|
assert exists_replica_1_1 == None
|