mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #33847 from ClickHouse/fix_33806
Fix logical error on RESTORE REPLICA
This commit is contained in:
commit
da9a38655b
@ -56,7 +56,6 @@ namespace ErrorCodes
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int TIMEOUT_EXCEEDED;
|
||||
extern const int TABLE_WAS_NOT_DROPPED;
|
||||
extern const int NO_ZOOKEEPER;
|
||||
}
|
||||
|
||||
|
||||
@ -472,12 +471,6 @@ void InterpreterSystemQuery::restoreReplica()
|
||||
{
|
||||
getContext()->checkAccess(AccessType::SYSTEM_RESTORE_REPLICA, table_id);
|
||||
|
||||
const zkutil::ZooKeeperPtr & zookeeper = getContext()->getZooKeeper();
|
||||
|
||||
if (zookeeper->expired())
|
||||
throw Exception(ErrorCodes::NO_ZOOKEEPER,
|
||||
"Cannot restore table metadata because ZooKeeper session has expired");
|
||||
|
||||
const StoragePtr table_ptr = DatabaseCatalog::instance().getTable(table_id, getContext());
|
||||
|
||||
auto * const table_replicated_ptr = dynamic_cast<StorageReplicatedMergeTree *>(table_ptr.get());
|
||||
@ -485,24 +478,7 @@ void InterpreterSystemQuery::restoreReplica()
|
||||
if (table_replicated_ptr == nullptr)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, table_is_not_replicated.data(), table_id.getNameForLogs());
|
||||
|
||||
auto & table_replicated = *table_replicated_ptr;
|
||||
|
||||
StorageReplicatedMergeTree::Status status;
|
||||
table_replicated.getStatus(status);
|
||||
|
||||
if (!status.is_readonly)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica must be readonly");
|
||||
|
||||
const String replica_name = table_replicated.getReplicaName();
|
||||
const String& zk_root_path = status.zookeeper_path;
|
||||
|
||||
if (String replica_path = zk_root_path + "replicas/" + replica_name; zookeeper->exists(replica_path))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Replica path is present at {} -- nothing to restore. "
|
||||
"If you are sure that metadata it lost and replica path contain some garbage, "
|
||||
"then use SYSTEM DROP REPLICA query first.", replica_path);
|
||||
|
||||
table_replicated.restoreMetadataInZooKeeper();
|
||||
table_replicated_ptr->restoreMetadataInZooKeeper();
|
||||
}
|
||||
|
||||
StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextMutablePtr system_context, bool need_ddl_guard)
|
||||
|
@ -774,7 +774,8 @@ void StorageReplicatedMergeTree::drop()
|
||||
/// or metadata of staled replica were removed manually,
|
||||
/// in this case, has_metadata_in_zookeeper = false, and we also permit to drop the table.
|
||||
|
||||
if (has_metadata_in_zookeeper)
|
||||
bool maybe_has_metadata_in_zookeeper = !has_metadata_in_zookeeper.has_value() || *has_metadata_in_zookeeper;
|
||||
if (maybe_has_metadata_in_zookeeper)
|
||||
{
|
||||
/// Table can be shut down, restarting thread is not active
|
||||
/// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice.
|
||||
@ -4811,12 +4812,22 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St
|
||||
void StorageReplicatedMergeTree::restoreMetadataInZooKeeper()
|
||||
{
|
||||
LOG_INFO(log, "Restoring replica metadata");
|
||||
if (!is_readonly)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica must be readonly");
|
||||
|
||||
if (!is_readonly || has_metadata_in_zookeeper)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: replica is not readonly");
|
||||
if (getZooKeeper()->exists(replica_path))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Replica path is present at {} - nothing to restore. "
|
||||
"If you are sure that metadata is lost and that replica path contains some garbage, "
|
||||
"then use SYSTEM DROP REPLICA query first.", replica_path);
|
||||
|
||||
if (has_metadata_in_zookeeper.has_value() && *has_metadata_in_zookeeper)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica has metadata in ZooKeeper: "
|
||||
"it's either a bug or it's a result of manual intervention to ZooKeeper");
|
||||
|
||||
if (are_restoring_replica.exchange(true))
|
||||
throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Replica restoration in progress");
|
||||
SCOPE_EXIT({ are_restoring_replica.store(false); });
|
||||
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
@ -4857,8 +4868,6 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper()
|
||||
LOG_INFO(log, "Attached all partitions, starting table");
|
||||
|
||||
startup();
|
||||
|
||||
are_restoring_replica.store(false);
|
||||
}
|
||||
|
||||
void StorageReplicatedMergeTree::dropPartNoWaitNoThrow(const String & part_name)
|
||||
|
@ -322,8 +322,9 @@ private:
|
||||
|
||||
/// If true, the table is offline and can not be written to it.
|
||||
std::atomic_bool is_readonly {false};
|
||||
/// If nullopt - ZooKeeper is not available, so we don't know if there is table metadata.
|
||||
/// If false - ZooKeeper is available, but there is no table metadata. It's safe to drop table in this case.
|
||||
bool has_metadata_in_zookeeper = true;
|
||||
std::optional<bool> has_metadata_in_zookeeper;
|
||||
|
||||
static constexpr auto default_zookeeper_name = "default";
|
||||
String zookeeper_name;
|
||||
|
Loading…
Reference in New Issue
Block a user