Merge pull request #21119 from fastio/bugfix_drop_metadata_from_auxiliary

fix the metadata leak when the Replicated*MergeTree with custom ZooKeeper cluster is dropped
This commit is contained in:
alexey-milovidov 2021-02-25 19:24:58 +03:00 committed by GitHub
commit 6a5ab40988
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 30 additions and 3 deletions

View File

@ -749,8 +749,12 @@ void StorageReplicatedMergeTree::drop()
if (has_metadata_in_zookeeper)
{
/// Table can be shut down, restarting thread is not active
/// and calling StorageReplicatedMergeTree::getZooKeeper() won't suffice.
auto zookeeper = global_context.getZooKeeper();
/// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice.
zkutil::ZooKeeperPtr zookeeper;
if (zookeeper_name == default_zookeeper_name)
zookeeper = global_context.getZooKeeper();
else
zookeeper = global_context.getAuxiliaryZooKeeper(zookeeper_name);
/// If probably there is metadata in ZooKeeper, we don't allow to drop the table.
if (!zookeeper)

View File

@ -6,7 +6,6 @@ from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance("node1", main_configs=["configs/zookeeper_config.xml", "configs/remote_servers.xml"], with_zookeeper=True)
node2 = cluster.add_instance("node2", main_configs=["configs/zookeeper_config.xml", "configs/remote_servers.xml"], with_zookeeper=True)
@ -78,3 +77,27 @@ def test_create_replicated_merge_tree_with_not_exists_auxiliary_zookeeper(starte
ENGINE = ReplicatedMergeTree('zookeeper_not_exits:/clickhouse/tables/test/test_auxiliary_zookeeper', '{replica}')
ORDER BY a;
'''.format(replica=node1.name))
# Drop table with auxiliary zookeeper.
def test_drop_replicated_merge_tree_with_auxiliary_zookeeper(started_cluster):
drop_table([node1, node2], "test_auxiliary_zookeeper")
for node in [node1, node2]:
node.query(
'''
CREATE TABLE test_auxiliary_zookeeper(a Int32)
ENGINE = ReplicatedMergeTree('zookeeper2:/clickhouse/tables/test/test_auxiliary_zookeeper', '{replica}')
ORDER BY a;
'''.format(replica=node.name))
# Insert data into node1, and query it from node2.
node1.query("INSERT INTO test_auxiliary_zookeeper VALUES (1)")
time.sleep(5)
expected = "1\n"
assert TSV(node1.query("SELECT a FROM test_auxiliary_zookeeper")) == TSV(expected)
assert TSV(node2.query("SELECT a FROM test_auxiliary_zookeeper")) == TSV(expected)
zk = cluster.get_kazoo_client('zoo1')
assert zk.exists('/clickhouse/tables/test/test_auxiliary_zookeeper')
drop_table([node1, node2], "test_auxiliary_zookeeper")
assert zk.exists('/clickhouse/tables/test/test_auxiliary_zookeeper') is None