mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
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:
commit
f9133f3d85
@ -294,7 +294,9 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp
|
||||
|
||||
void execute(MetadataTransactionPtr tx) override
|
||||
{
|
||||
removeMetadataRecursive(tx, path);
|
||||
/// Similar to DiskLocal and https://en.cppreference.com/w/cpp/filesystem/remove
|
||||
if (metadata_storage.exists(path))
|
||||
removeMetadataRecursive(tx, path);
|
||||
}
|
||||
|
||||
void undo() override
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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");
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
disk->removeRecursive(path);
|
||||
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/");
|
||||
|
||||
|
@ -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 &)
|
||||
{
|
||||
disk->removeRecursive(path);
|
||||
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/");
|
||||
|
||||
|
1
tests/integration/test_drop_no_local_path/__init__.py
Normal file
1
tests/integration/test_drop_no_local_path/__init__.py
Normal file
@ -0,0 +1 @@
|
||||
#!/usr/bin/env python3
|
@ -0,0 +1,12 @@
|
||||
<clickhouse>
|
||||
<remote_servers>
|
||||
<test_cluster>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>instance</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
58
tests/integration/test_drop_no_local_path/test.py
Normal file
58
tests/integration/test_drop_no_local_path/test.py
Normal 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)
|
Loading…
Reference in New Issue
Block a user