Get rid of some deadlocks

This commit is contained in:
alesapin 2020-10-15 11:29:18 +03:00
parent c5419083e3
commit 583d7042e9
3 changed files with 5 additions and 18 deletions

View File

@ -117,7 +117,6 @@ catch (...) /// Exception while we looking for a task
void IBackgroundJobExecutor::start()
{
std::lock_guard lock(task_mutex);
if (!scheduling_task)
{
scheduling_task = global_context.getSchedulePool().createTask(
@ -129,7 +128,6 @@ void IBackgroundJobExecutor::start()
void IBackgroundJobExecutor::finish()
{
std::lock_guard lock(task_mutex);
if (scheduling_task)
{
scheduling_task->deactivate();
@ -140,7 +138,6 @@ void IBackgroundJobExecutor::finish()
void IBackgroundJobExecutor::triggerTask()
{
std::lock_guard lock(task_mutex);
if (scheduling_task)
scheduling_task->schedule();
}

View File

@ -65,7 +65,6 @@ private:
std::unordered_map<PoolType, PoolConfig> pools_configs;
BackgroundSchedulePool::TaskHolder scheduling_task;
std::mutex task_mutex;
public:
IBackgroundJobExecutor(

View File

@ -3482,10 +3482,7 @@ void StorageReplicatedMergeTree::startup()
/// If we don't separate create/start steps, race condition will happen
/// between the assignment of queue_task_handle and queueTask that use the queue_task_handle.
{
auto lock = queue.lockQueue();
background_executor.start();
}
background_executor.start();
if (areBackgroundMovesNeeded())
background_moves_executor.start();
@ -3518,13 +3515,10 @@ void StorageReplicatedMergeTree::shutdown()
parts_mover.moves_blocker.cancelForever();
restarting_thread.shutdown();
background_executor.finish();
{
/// Queue can trigger queue_task_handle itself. So we ensure that all
/// queue processes finished and after that reset queue_task_handle.
auto lock = queue.lockQueue();
background_executor.finish();
/// Cancel logs pulling after background task were cancelled. It's still
/// required because we can trigger pullLogsToQueue during manual OPTIMIZE,
/// MUTATE, etc. query.
@ -5866,12 +5860,9 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI
/// Let's fetch new log entries firstly
queue.pullLogsToQueue(getZooKeeper());
{
auto lock = queue.lockQueue();
background_executor.triggerTask();
/// This is significant, because the execution of this task could be delayed at BackgroundPool.
/// And we force it to be executed.
}
/// This is significant, because the execution of this task could be delayed at BackgroundPool.
/// And we force it to be executed.
background_executor.triggerTask();
Poco::Event target_size_event;
auto callback = [&target_size_event, queue_size] (size_t new_queue_size)