mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-13 09:52:38 +00:00
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:
commit
f086cb47f3
@ -323,6 +323,13 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperAndAssertNotReadonl
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperAndAssertNotStaticStorage() const
|
||||||
|
{
|
||||||
|
auto res = getZooKeeper();
|
||||||
|
assertNotStaticStorage();
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
String StorageReplicatedMergeTree::getEndpointName() const
|
String StorageReplicatedMergeTree::getEndpointName() const
|
||||||
{
|
{
|
||||||
const MergeTreeSettings & settings = getContext()->getReplicatedMergeTreeSettings();
|
const MergeTreeSettings & settings = getContext()->getReplicatedMergeTreeSettings();
|
||||||
@ -3656,9 +3663,20 @@ void StorageReplicatedMergeTree::queueUpdatingTask()
|
|||||||
last_queue_update_start_time.store(time(nullptr));
|
last_queue_update_start_time.store(time(nullptr));
|
||||||
queue_update_in_progress = true;
|
queue_update_in_progress = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
try
|
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));
|
last_queue_update_finish_time.store(time(nullptr));
|
||||||
queue_update_in_progress = false;
|
queue_update_in_progress = false;
|
||||||
}
|
}
|
||||||
@ -3890,7 +3908,15 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
|
|||||||
/// (OPTIMIZE queries) can assign new merges.
|
/// (OPTIMIZE queries) can assign new merges.
|
||||||
std::lock_guard merge_selecting_lock(merge_selecting_mutex);
|
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;
|
std::optional<ReplicatedMergeTreeMergePredicate> merge_pred;
|
||||||
|
|
||||||
@ -4044,7 +4070,17 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask()
|
|||||||
|
|
||||||
try
|
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 (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
@ -5727,6 +5763,11 @@ void StorageReplicatedMergeTree::assertNotReadonly() const
|
|||||||
{
|
{
|
||||||
if (is_readonly)
|
if (is_readonly)
|
||||||
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {})", replica_path);
|
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode (replica path: {})", replica_path);
|
||||||
|
assertNotStaticStorage();
|
||||||
|
}
|
||||||
|
|
||||||
|
void StorageReplicatedMergeTree::assertNotStaticStorage() const
|
||||||
|
{
|
||||||
if (isStaticStorage())
|
if (isStaticStorage())
|
||||||
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage");
|
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to static storage");
|
||||||
}
|
}
|
||||||
|
@ -404,6 +404,7 @@ private:
|
|||||||
/// for table.
|
/// for table.
|
||||||
zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const;
|
zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const;
|
||||||
zkutil::ZooKeeperPtr getZooKeeperAndAssertNotReadonly() const;
|
zkutil::ZooKeeperPtr getZooKeeperAndAssertNotReadonly() const;
|
||||||
|
zkutil::ZooKeeperPtr getZooKeeperAndAssertNotStaticStorage() const;
|
||||||
void setZooKeeper();
|
void setZooKeeper();
|
||||||
String getEndpointName() const;
|
String getEndpointName() const;
|
||||||
|
|
||||||
@ -842,6 +843,9 @@ private:
|
|||||||
/// Throw an exception if the table is readonly.
|
/// Throw an exception if the table is readonly.
|
||||||
void assertNotReadonly() const;
|
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).
|
/// 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.
|
/// Returns false if the partition doesn't exist yet.
|
||||||
/// Caller must hold delimiting_block_lock until creation of drop/replace entry in log.
|
/// Caller must hold delimiting_block_lock until creation of drop/replace entry in log.
|
||||||
|
Loading…
Reference in New Issue
Block a user