mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Remove log message "Table is in readonly mode" while dropping a table.
This commit is contained in:
parent
2b3098c641
commit
4aec023dcb
@ -316,9 +316,16 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperAndAssertNotReadonl
|
||||
/// There's a short period of time after connection loss when new session is created,
|
||||
/// but replication queue is not reinitialized. We must ensure that table is not readonly anymore
|
||||
/// before using new ZooKeeper session to write something (except maybe GET_PART) into replication log.
|
||||
auto res = getZooKeeper();
|
||||
assertNotReadonly();
|
||||
return res;
|
||||
return getZooKeeperIfNotReadonly(/* assert_if_readonly = */ true, /* assert_if_readonly_while_shutdown = */ true);
|
||||
}
|
||||
|
||||
zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperIfNotReadonly(bool assert_if_readonly, bool assert_if_readonly_while_shutdown) const
|
||||
{
|
||||
auto zookeeper = getZooKeeper();
|
||||
bool readonly = checkIfReadonly(assert_if_readonly, assert_if_readonly_while_shutdown);
|
||||
if (readonly)
|
||||
return nullptr;
|
||||
return zookeeper;
|
||||
}
|
||||
|
||||
String StorageReplicatedMergeTree::getEndpointName() const
|
||||
@ -3650,9 +3657,14 @@ void StorageReplicatedMergeTree::queueUpdatingTask()
|
||||
last_queue_update_start_time.store(time(nullptr));
|
||||
queue_update_in_progress = true;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
queue.pullLogsToQueue(getZooKeeperAndAssertNotReadonly(), queue_updating_task->getWatchCallback(), ReplicatedMergeTreeQueue::UPDATE);
|
||||
auto zookeeper = getZooKeeperIfNotReadonly(/* assert_if_readonly = */ true, /* assert_if_readonly_while_shutdown = */ false);
|
||||
if (!zookeeper)
|
||||
return; /// readonly because shutdown() was called
|
||||
|
||||
queue.pullLogsToQueue(zookeeper, queue_updating_task->getWatchCallback(), ReplicatedMergeTreeQueue::UPDATE);
|
||||
last_queue_update_finish_time.store(time(nullptr));
|
||||
queue_update_in_progress = false;
|
||||
}
|
||||
@ -3884,7 +3896,9 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
|
||||
/// (OPTIMIZE queries) can assign new merges.
|
||||
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
|
||||
|
||||
auto zookeeper = getZooKeeperAndAssertNotReadonly();
|
||||
auto zookeeper = getZooKeeperIfNotReadonly(/* assert_if_readonly = */ true, /* assert_if_readonly_while_shutdown = */ false);
|
||||
if (!zookeeper)
|
||||
return AttemptStatus::CannotSelect; /// readonly because shutdown() was called
|
||||
|
||||
std::optional<ReplicatedMergeTreeMergePredicate> merge_pred;
|
||||
|
||||
@ -4038,7 +4052,11 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask()
|
||||
|
||||
try
|
||||
{
|
||||
needs_reschedule = queue.tryFinalizeMutations(getZooKeeperAndAssertNotReadonly());
|
||||
auto zookeeper = getZooKeeperIfNotReadonly(/* assert_if_readonly = */ true, /* assert_if_readonly_while_shutdown = */ false);
|
||||
if (!zookeeper)
|
||||
return; /// readonly because shutdown() was called
|
||||
|
||||
needs_reschedule = queue.tryFinalizeMutations(zookeeper);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -5711,11 +5729,25 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalBytesUncompressed(const S
|
||||
}
|
||||
|
||||
void StorageReplicatedMergeTree::assertNotReadonly() const
|
||||
{
|
||||
checkIfReadonly(/* assert_if_readonly = */ true, /* assert_if_readonly_while_shutdown = */ true);
|
||||
}
|
||||
|
||||
bool StorageReplicatedMergeTree::checkIfReadonly(bool assert_if_readonly, bool assert_if_readonly_while_shutdown) const
|
||||
{
|
||||
if (is_readonly)
|
||||
{
|
||||
if (assert_if_readonly && (assert_if_readonly_while_shutdown || !(shutdown_called || partial_shutdown_called)))
|
||||
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {})", replica_path);
|
||||
return true; /// readonly
|
||||
}
|
||||
if (isStaticStorage())
|
||||
{
|
||||
if (assert_if_readonly && (assert_if_readonly_while_shutdown || !(shutdown_called || partial_shutdown_called)))
|
||||
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage");
|
||||
return true; /// readonly
|
||||
}
|
||||
return false; /// not readonly
|
||||
}
|
||||
|
||||
|
||||
|
@ -404,6 +404,7 @@ private:
|
||||
/// for table.
|
||||
zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const;
|
||||
zkutil::ZooKeeperPtr getZooKeeperAndAssertNotReadonly() const;
|
||||
zkutil::ZooKeeperPtr getZooKeeperIfNotReadonly(bool assert_if_readonly, bool assert_if_readonly_while_shutdown) const;
|
||||
void setZooKeeper();
|
||||
String getEndpointName() const;
|
||||
|
||||
@ -841,6 +842,7 @@ private:
|
||||
|
||||
/// Throw an exception if the table is readonly.
|
||||
void assertNotReadonly() const;
|
||||
bool checkIfReadonly(bool assert_if_readonly, bool assert_if_readonly_while_shutdown) const;
|
||||
|
||||
/// Produce an imaginary part info covering all parts in the specified partition (at the call moment).
|
||||
/// Returns false if the partition doesn't exist yet.
|
||||
|
Loading…
Reference in New Issue
Block a user