mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #23294 from ClickHouse/fix_create_drop_replica_race
Fix race between DROP/CREATE ReplicatedMergeTree
This commit is contained in:
commit
6c5ff3db7d
@ -585,42 +585,24 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr
|
||||
/// This is Ok because another replica is definitely going to drop the table.
|
||||
|
||||
LOG_WARNING(log, "Removing leftovers from table {} (this might take several minutes)", zookeeper_path);
|
||||
String drop_lock_path = zookeeper_path + "/dropped/lock";
|
||||
Coordination::Error code = zookeeper->tryCreate(drop_lock_path, "", zkutil::CreateMode::Ephemeral);
|
||||
|
||||
Strings children;
|
||||
Coordination::Error code = zookeeper->tryGetChildren(zookeeper_path, children);
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
LOG_WARNING(log, "Table {} is already finished removing by another replica right now", replica_path);
|
||||
LOG_WARNING(log, "The leftovers from table {} were removed by another replica", zookeeper_path);
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
throw Coordination::Exception(code, drop_lock_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & child : children)
|
||||
if (child != "dropped")
|
||||
zookeeper->tryRemoveRecursive(zookeeper_path + "/" + child);
|
||||
|
||||
Coordination::Requests ops;
|
||||
Coordination::Responses responses;
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/dropped", -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path, -1));
|
||||
code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *zookeeper);
|
||||
if (!removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, log))
|
||||
{
|
||||
LOG_WARNING(log, "Table {} is already finished removing by another replica right now", replica_path);
|
||||
}
|
||||
else if (code == Coordination::Error::ZNOTEMPTY)
|
||||
{
|
||||
throw Exception(fmt::format(
|
||||
"The old table was not completely removed from ZooKeeper, {} still exists and may contain some garbage. But it should never happen according to the logic of operations (it's a bug).", zookeeper_path), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
/// It is still possible that ZooKeeper session is expired or server is killed in the middle of the delete operation.
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(log, "The leftovers from table {} was successfully removed from ZooKeeper", zookeeper_path);
|
||||
/// Someone is recursively removing table right now, we cannot create new table until old one is removed
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -633,10 +615,6 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr
|
||||
Coordination::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent));
|
||||
|
||||
/// Check that the table is not being dropped right now.
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/dropped", "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/dropped", -1));
|
||||
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str,
|
||||
zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(),
|
||||
@ -824,10 +802,18 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con
|
||||
* because table creation is executed in single transaction that will conflict with remaining nodes.
|
||||
*/
|
||||
|
||||
/// Node /dropped works like a lock that protects from concurrent removal of old table and creation of new table.
|
||||
/// But recursive removal may fail in the middle of operation leaving some garbage in zookeeper_path, so
|
||||
/// we remove it on table creation if there is /dropped node. Creating thread may remove /dropped node created by
|
||||
/// removing thread, and it causes race condition if removing thread is not finished yet.
|
||||
/// To avoid this we also create ephemeral child before starting recursive removal.
|
||||
/// (The existence of child node does not allow to remove parent node).
|
||||
Coordination::Requests ops;
|
||||
Coordination::Responses responses;
|
||||
String drop_lock_path = zookeeper_path + "/dropped/lock";
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/replicas", -1));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/dropped", "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(drop_lock_path, "", zkutil::CreateMode::Ephemeral));
|
||||
Coordination::Error code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS)
|
||||
@ -844,48 +830,57 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con
|
||||
}
|
||||
else
|
||||
{
|
||||
auto metadata_drop_lock = zkutil::EphemeralNodeHolder::existing(drop_lock_path, *zookeeper);
|
||||
LOG_INFO(logger, "Removing table {} (this might take several minutes)", zookeeper_path);
|
||||
|
||||
Strings children;
|
||||
code = zookeeper->tryGetChildren(zookeeper_path, children);
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
LOG_WARNING(logger, "Table {} is already finished removing by another replica right now", remote_replica_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & child : children)
|
||||
if (child != "dropped")
|
||||
zookeeper->tryRemoveRecursive(zookeeper_path + "/" + child);
|
||||
|
||||
ops.clear();
|
||||
responses.clear();
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/dropped", -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path, -1));
|
||||
code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
LOG_WARNING(logger, "Table {} is already finished removing by another replica right now", remote_replica_path);
|
||||
}
|
||||
else if (code == Coordination::Error::ZNOTEMPTY)
|
||||
{
|
||||
LOG_ERROR(logger, "Table was not completely removed from ZooKeeper, {} still exists and may contain some garbage.",
|
||||
zookeeper_path);
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
/// It is still possible that ZooKeeper session is expired or server is killed in the middle of the delete operation.
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(logger, "Table {} was successfully removed from ZooKeeper", zookeeper_path);
|
||||
}
|
||||
}
|
||||
removeTableNodesFromZooKeeper(zookeeper, zookeeper_path, metadata_drop_lock, logger);
|
||||
}
|
||||
}
|
||||
|
||||
bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr zookeeper,
|
||||
const String & zookeeper_path, const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, Poco::Logger * logger)
|
||||
{
|
||||
bool completely_removed = false;
|
||||
Strings children;
|
||||
Coordination::Error code = zookeeper->tryGetChildren(zookeeper_path, children);
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal of replicated table. It's a bug");
|
||||
|
||||
|
||||
for (const auto & child : children)
|
||||
if (child != "dropped")
|
||||
zookeeper->tryRemoveRecursive(zookeeper_path + "/" + child);
|
||||
|
||||
Coordination::Requests ops;
|
||||
Coordination::Responses responses;
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/dropped", -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path, -1));
|
||||
code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "There is a race condition between creation and removal of replicated table. It's a bug");
|
||||
}
|
||||
else if (code == Coordination::Error::ZNOTEMPTY)
|
||||
{
|
||||
LOG_ERROR(logger, "Table was not completely removed from ZooKeeper, {} still exists and may contain some garbage,"
|
||||
"but someone is removing it right now.", zookeeper_path);
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
/// It is still possible that ZooKeeper session is expired or server is killed in the middle of the delete operation.
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
}
|
||||
else
|
||||
{
|
||||
metadata_drop_lock->setAlreadyRemoved();
|
||||
completely_removed = true;
|
||||
LOG_INFO(logger, "Table {} was successfully removed from ZooKeeper", zookeeper_path);
|
||||
}
|
||||
|
||||
return completely_removed;
|
||||
}
|
||||
|
||||
|
||||
/** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/metadata).
|
||||
* If not, throw an exception.
|
||||
|
@ -208,6 +208,10 @@ public:
|
||||
*/
|
||||
static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger);
|
||||
|
||||
/// Removes table from ZooKeeper after the last replica was dropped
|
||||
static bool removeTableNodesFromZooKeeper(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path,
|
||||
const zkutil::EphemeralNodeHolder::Ptr & metadata_drop_lock, Poco::Logger * logger);
|
||||
|
||||
/// Get job to execute in background pool (merge, mutate, drop range and so on)
|
||||
std::optional<JobAndPool> getDataProcessingJob() override;
|
||||
|
||||
|
@ -11,11 +11,10 @@ function thread()
|
||||
while true; do
|
||||
$CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1 SYNC;
|
||||
CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 |
|
||||
grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now|is already finished removing by another replica right now|Removing leftovers from table|Another replica was suddenly created|was successfully removed from ZooKeeper|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time'
|
||||
grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now| were removed by another replica|Removing leftovers from table|Another replica was suddenly created|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time'
|
||||
done
|
||||
}
|
||||
|
||||
|
||||
# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout
|
||||
export -f thread;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user