Merge pull request #45791 from ClickHouse/fix_drop_hung

Fix bug in tables drop which can lead to potential query hung
This commit is contained in:
Alexander Gololobov 2023-01-31 10:11:11 +01:00 committed by GitHub
commit f9133f3d85
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 100 additions and 3 deletions

View File

@ -294,6 +294,8 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp
void execute(MetadataTransactionPtr tx) override
{
/// Similar to DiskLocal and https://en.cppreference.com/w/cpp/filesystem/remove
if (metadata_storage.exists(path))
removeMetadataRecursive(tx, path);
}

View File

@ -2613,6 +2613,14 @@ void MergeTreeData::dropAllData()
if (disk->isBroken())
continue;
/// It can naturally happen if we cannot drop table from the first time
/// i.e. get exceptions after remove recursive
if (!disk->exists(relative_data_path))
{
LOG_INFO(log, "dropAllData: path {} is already removed from disk {}", relative_data_path, disk->getName());
continue;
}
LOG_INFO(log, "dropAllData: remove format_version.txt, detached, moving and write ahead logs");
disk->removeFileIfExists(fs::path(relative_data_path) / FORMAT_VERSION_FILE_NAME);

View File

@ -1156,7 +1156,15 @@ void StorageDistributed::drop()
auto disks = data_volume->getDisks();
for (const auto & disk : disks)
{
if (!disk->exists(relative_data_path))
{
LOG_INFO(log, "Path {} is already removed from disk {}", relative_data_path, disk->getName());
continue;
}
disk->removeRecursive(relative_data_path);
}
LOG_DEBUG(log, "Removed");
}

View File

@ -89,7 +89,11 @@ void StorageJoin::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPt
std::lock_guard mutate_lock(mutate_mutex);
TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context);
if (disk->exists(path))
disk->removeRecursive(path);
else
LOG_INFO(&Poco::Logger::get("StorageJoin"), "Path {} is already removed from disk {}", path, disk->getName());
disk->createDirectories(path);
disk->createDirectories(path + "tmp/");

View File

@ -162,7 +162,11 @@ std::optional<UInt64> StorageSet::totalBytes(const Settings &) const { return se
void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &)
{
if (disk->exists(path))
disk->removeRecursive(path);
else
LOG_INFO(&Poco::Logger::get("StorageSet"), "Path {} is already removed from disk {}", path, disk->getName());
disk->createDirectories(path);
disk->createDirectories(fs::path(path) / "tmp/");

View File

@ -0,0 +1 @@
#!/usr/bin/env python3

View File

@ -0,0 +1,12 @@
<clickhouse>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>instance</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</clickhouse>

View File

@ -0,0 +1,58 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance("instance", main_configs=["configs/remote_servers.xml"])
@pytest.fixture(scope="module")
def setup_nodes():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def drop_table_directory(table_name):
data_path = instance.query(
f"SELECT data_paths[1] FROM system.tables where name = '{table_name}'"
).strip()
print("Data path", data_path)
instance.exec_in_container(
["bash", "-c", f"rm -fr {data_path}"], privileged=True, user="root"
)
def test_drop_no_local_path(setup_nodes):
instance.query(
"CREATE TABLE merge_tree_table (key UInt64) ENGINE = MergeTree() ORDER BY tuple()"
)
instance.query("INSERT INTO merge_tree_table VALUES (1)")
drop_table_directory("merge_tree_table")
instance.query("DROP TABLE merge_tree_table SYNC", timeout=10)
instance.query(
"CREATE TABLE merge_tree_table (key UInt64) ENGINE = MergeTree() ORDER BY tuple()"
)
instance.query(
"CREATE TABLE distributed_table (key UInt64) ENGINE = Distributed(test_cluster, default, merge_tree_table, key)"
)
instance.query("INSERT INTO distributed_table VALUES(0)")
drop_table_directory("distributed_table")
instance.query("DROP TABLE distributed_table SYNC", timeout=10)
instance.query("DROP TABLE merge_tree_table SYNC", timeout=10)
instance.query(
"CREATE TABLE join_table(`id` UInt64, `val` String) ENGINE = Join(ANY, LEFT, id)"
)
instance.query("INSERT INTO join_table VALUES (1, 'a')")
drop_table_directory("join_table")
instance.query("TRUNCATE TABLE join_table", timeout=10)