mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Deactivate mutationsFinalizingTask until startup
This commit is contained in:
parent
45c6974f7d
commit
ab8da5a539
@ -294,12 +294,17 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
|||||||
merge_selecting_task = getContext()->getSchedulePool().createTask(
|
merge_selecting_task = getContext()->getSchedulePool().createTask(
|
||||||
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); });
|
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); });
|
||||||
|
|
||||||
/// Will be activated if we win leader election.
|
/// Will be activated if we will achieve leader state.
|
||||||
merge_selecting_task->deactivate();
|
merge_selecting_task->deactivate();
|
||||||
|
|
||||||
mutations_finalizing_task = getContext()->getSchedulePool().createTask(
|
mutations_finalizing_task = getContext()->getSchedulePool().createTask(
|
||||||
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); });
|
getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); });
|
||||||
|
|
||||||
|
/// This task can be scheduled by different parts of code even when storage is readonly.
|
||||||
|
/// This can lead to redundant exceptions during startup.
|
||||||
|
/// Will be activated by restarting thread.
|
||||||
|
mutations_finalizing_task->deactivate();
|
||||||
|
|
||||||
bool has_zookeeper = getContext()->hasZooKeeper() || getContext()->hasAuxiliaryZooKeeper(zookeeper_name);
|
bool has_zookeeper = getContext()->hasZooKeeper() || getContext()->hasAuxiliaryZooKeeper(zookeeper_name);
|
||||||
if (has_zookeeper)
|
if (has_zookeeper)
|
||||||
{
|
{
|
||||||
|
Loading…
Reference in New Issue
Block a user