ClickHouse/tests/integration/test_drop_replica/test.py
Yarik Briukhovetskyi a6f89c0546
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 8ce961be21.

* Revert "fixed build"

This reverts commit 8ce961be21.

* changed usage of assertDatabaseExistsUnlocked()

* fixed style check

* style check

* style check

* Revert "style check"

This reverts commit 28a9ee85a0.

* 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 d45337d65c.

* 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 ca6d4c17c8.

* fixed test

* Revert "fixed test"

This reverts commit fe6d0d9c86, reversing
changes made to 22f4496704.

* 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 13:24:16 +03:00

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