Merge pull request #72192 from vitlibar/fix-logging-table-is-readonly

Remove log message "Table is in readonly mode" while dropping a table.
This commit is contained in:
Vitaly Baranov 2024-12-04 01:22:26 +00:00 committed by GitHub
commit f086cb47f3
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 48 additions and 3 deletions

View File

@ -323,6 +323,13 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperAndAssertNotReadonl
return res;
}
zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperAndAssertNotStaticStorage() const
{
auto res = getZooKeeper();
assertNotStaticStorage();
return res;
}
String StorageReplicatedMergeTree::getEndpointName() const
{
const MergeTreeSettings & settings = getContext()->getReplicatedMergeTreeSettings();
@ -3656,9 +3663,20 @@ 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 = getZooKeeperAndAssertNotStaticStorage();
if (is_readonly)
{
/// Note that we need to check shutdown_prepared_called, not shutdown_called, since the table will be marked as readonly
/// after calling StorageReplicatedMergeTree::flushAndPrepareForShutdown().
if (shutdown_prepared_called)
return;
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {}), cannot update queue", replica_path);
}
queue.pullLogsToQueue(zookeeper, queue_updating_task->getWatchCallback(), ReplicatedMergeTreeQueue::UPDATE);
last_queue_update_finish_time.store(time(nullptr));
queue_update_in_progress = false;
}
@ -3890,7 +3908,15 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
/// (OPTIMIZE queries) can assign new merges.
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
auto zookeeper = getZooKeeperAndAssertNotReadonly();
auto zookeeper = getZooKeeperAndAssertNotStaticStorage();
if (is_readonly)
{
/// Note that we need to check shutdown_prepared_called, not shutdown_called, since the table will be marked as readonly
/// after calling StorageReplicatedMergeTree::flushAndPrepareForShutdown().
if (shutdown_prepared_called)
return AttemptStatus::CannotSelect;
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {}), cannot assign new merges", replica_path);
}
std::optional<ReplicatedMergeTreeMergePredicate> merge_pred;
@ -4044,7 +4070,17 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask()
try
{
needs_reschedule = queue.tryFinalizeMutations(getZooKeeperAndAssertNotReadonly());
auto zookeeper = getZooKeeperAndAssertNotStaticStorage();
if (is_readonly)
{
/// Note that we need to check shutdown_prepared_called, not shutdown_called, since the table will be marked as readonly
/// after calling StorageReplicatedMergeTree::flushAndPrepareForShutdown().
if (shutdown_prepared_called)
return;
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {}), cannot finalize mutations", replica_path);
}
needs_reschedule = queue.tryFinalizeMutations(zookeeper);
}
catch (...)
{
@ -5727,6 +5763,11 @@ void StorageReplicatedMergeTree::assertNotReadonly() const
{
if (is_readonly)
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {})", replica_path);
assertNotStaticStorage();
}
void StorageReplicatedMergeTree::assertNotStaticStorage() const
{
if (isStaticStorage())
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage");
}

View File

@ -404,6 +404,7 @@ private:
/// for table.
zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const;
zkutil::ZooKeeperPtr getZooKeeperAndAssertNotReadonly() const;
zkutil::ZooKeeperPtr getZooKeeperAndAssertNotStaticStorage() const;
void setZooKeeper();
String getEndpointName() const;
@ -842,6 +843,9 @@ private:
/// Throw an exception if the table is readonly.
void assertNotReadonly() const;
/// Throw an exception if the table is readonly because it's a static storage.
void assertNotStaticStorage() 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.
/// Caller must hold delimiting_block_lock until creation of drop/replace entry in log.