From 7036d82e3623af250415a6d0a67cf18f8c5efe0a Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 15:32:09 +0800 Subject: [PATCH 01/19] for task --- .../Schedule/Events/tests/gtest_event.cpp | 14 +++--- .../Pipeline/Schedule/Reactor/WaitReactor.cpp | 11 +++-- .../Reactor/tests/gtest_wait_task_list.cpp | 2 +- .../TaskQueues/tests/bench_task_queue.cpp | 2 +- .../TaskQueues/tests/gtest_io_priority.cpp | 2 +- .../Schedule/TaskQueues/tests/gtest_mlfq.cpp | 2 +- .../tests/gtest_resource_control_queue.cpp | 2 +- .../Tasks/AggregateFinalSpillTask.cpp | 2 +- .../Schedule/Tasks/AggregateFinalSpillTask.h | 2 +- .../Pipeline/Schedule/Tasks/ExecTaskStatus.h | 41 ++++++++++++++++ .../Pipeline/Schedule/Tasks/IOEventTask.h | 4 +- .../Schedule/Tasks/LoadBucketTask.cpp | 2 +- .../Pipeline/Schedule/Tasks/LoadBucketTask.h | 2 +- .../Pipeline/Schedule/Tasks/NotifyFuture.h | 28 +++++++++++ .../Schedule/Tasks/PipeConditionVariable.h | 44 +++++++++++++++++ .../Pipeline/Schedule/Tasks/PipelineTask.h | 6 +-- .../Schedule/Tasks/PipelineTaskBase.h | 6 +-- .../Pipeline/Schedule/Tasks/RFWaitTask.h | 4 +- .../Schedule/Tasks/SimplePipelineTask.h | 6 +-- .../Schedule/Tasks/StreamRestoreTask.cpp | 8 +-- .../Schedule/Tasks/StreamRestoreTask.h | 6 +-- .../Flash/Pipeline/Schedule/Tasks/Task.cpp | 12 ++--- dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 49 +++++++++---------- .../Schedule/ThreadPool/TaskThreadPool.cpp | 15 ++++-- .../Schedule/ThreadPool/TaskThreadPoolImpl.h | 4 +- .../Schedule/tests/gtest_task_scheduler.cpp | 22 ++++----- .../DeltaMerge/ReadThread/WorkQueue.h | 14 ++++++ .../Storages/DeltaMerge/SegmentReadTaskPool.h | 7 ++- 28 files changed, 228 insertions(+), 91 deletions(-) create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/ExecTaskStatus.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp index dda48c8b632..a3cf9f88b97 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp @@ -34,7 +34,7 @@ class BaseTask : public EventTask {} protected: - ExecTaskStatus executeImpl() override + ReturnStatus executeImpl() override { --counter; return ExecTaskStatus::FINISHED; @@ -75,7 +75,7 @@ class RunTask : public EventTask {} protected: - ExecTaskStatus executeImpl() override + ReturnStatus executeImpl() override { while ((--loop_count) > 0) return ExecTaskStatus::RUNNING; @@ -116,7 +116,7 @@ class DeadLoopTask : public EventTask {} protected: - ExecTaskStatus executeImpl() override + ReturnStatus executeImpl() override { std::this_thread::sleep_for(std::chrono::milliseconds(1)); return ExecTaskStatus::RUNNING; @@ -185,7 +185,7 @@ class ThrowExceptionTask : public EventTask {} protected: - ExecTaskStatus executeImpl() override { throw Exception("throw exception in doExecuteImpl"); } + ReturnStatus executeImpl() override { throw Exception("throw exception in doExecuteImpl"); } }; class ThrowExceptionEvent : public Event @@ -291,7 +291,7 @@ class TestPorfileTask : public EventTask protected: // executeImpl min_time ==> executeIOImpl min_time ==> awaitImpl min_time. - ExecTaskStatus executeImpl() override + ReturnStatus executeImpl() override { if (cpu_execute_time < min_time) { @@ -302,7 +302,7 @@ class TestPorfileTask : public EventTask return ExecTaskStatus::IO_IN; } - ExecTaskStatus executeIOImpl() override + ReturnStatus executeIOImpl() override { if (io_execute_time < min_time) { @@ -313,7 +313,7 @@ class TestPorfileTask : public EventTask return ExecTaskStatus::WAITING; } - ExecTaskStatus awaitImpl() override + ReturnStatus awaitImpl() override { if unlikely (!wait_stopwatch) wait_stopwatch.emplace(CLOCK_MONOTONIC_COARSE); diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp index b018e72f394..aa53dda7162 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -36,8 +37,8 @@ bool WaitReactor::awaitAndCollectReadyTask(WaitingTask && task) { assert(task.first); auto * task_ptr = task.second; - auto status = task_ptr->await(); - switch (status) + auto return_status = task_ptr->await(); + switch (return_status.status) { case ExecTaskStatus::WAITING: return false; @@ -50,6 +51,10 @@ bool WaitReactor::awaitAndCollectReadyTask(WaitingTask && task) task_ptr->profile_info.elapsedAwaitTime(); io_tasks.push_back(std::move(task.first)); return true; + case ExecTaskStatus::WAIT_FOR_NOTIFY: + assert(return_status.future); + return_status.future->registerTask(std::move(task.first)); + return true; case FINISH_STATUS: task_ptr->profile_info.elapsedAwaitTime(); task_ptr->startTraceMemory(); @@ -58,7 +63,7 @@ bool WaitReactor::awaitAndCollectReadyTask(WaitingTask && task) task.first.reset(); return true; default: - UNEXPECTED_STATUS(logger, status); + UNEXPECTED_STATUS(logger, return_status.status); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/tests/gtest_wait_task_list.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/tests/gtest_wait_task_list.cpp index ad51a729214..62ccd9d1044 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/tests/gtest_wait_task_list.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/tests/gtest_wait_task_list.cpp @@ -30,7 +30,7 @@ class PlainTask : public Task : Task(exec_context_) {} - ExecTaskStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } + ReturnStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } }; } // namespace diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp index 03397bed9d0..c44cf9e5758 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp @@ -33,7 +33,7 @@ namespace tests : Task(exec_context) \ {} \ \ - ExecTaskStatus executeImpl() override \ + ReturnStatus executeImpl() override \ { \ if (task_exec_cur_count <= task_exec_total_count) \ { \ diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp index 398f5a4dff3..2bac2eec02a 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp @@ -30,7 +30,7 @@ class MockIOTask : public Task : Task(exec_context_, "", is_io_in ? ExecTaskStatus::IO_IN : ExecTaskStatus::IO_OUT) {} - ExecTaskStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } + ReturnStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } }; } // namespace diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp index 6d004f956a4..d5ce456a4a6 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp @@ -30,7 +30,7 @@ class PlainTask : public Task : Task(exec_context_) {} - ExecTaskStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } + ReturnStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } }; } // namespace diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index 5003117c91c..9d242eb4d40 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -39,7 +39,7 @@ class SimpleTask : public Task ~SimpleTask() override = default; - ExecTaskStatus executeImpl() noexcept override + ReturnStatus executeImpl() noexcept override { if (exec_time_counter < total_exec_times) { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.cpp index d3367abd39a..2d7b918aedf 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.cpp @@ -35,7 +35,7 @@ void AggregateFinalSpillTask::doFinalizeImpl() agg_context.reset(); } -ExecTaskStatus AggregateFinalSpillTask::executeIOImpl() +ReturnStatus AggregateFinalSpillTask::executeIOImpl() { agg_context->spillData(index); return ExecTaskStatus::FINISHED; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.h index 1fa9af3ddf3..846f5ca1c31 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.h @@ -32,7 +32,7 @@ class AggregateFinalSpillTask : public OutputIOEventTask size_t index_); protected: - ExecTaskStatus executeIOImpl() override; + ReturnStatus executeIOImpl() override; void doFinalizeImpl() override; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/ExecTaskStatus.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/ExecTaskStatus.h new file mode 100644 index 00000000000..14bf4864275 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/ExecTaskStatus.h @@ -0,0 +1,41 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +namespace DB +{ +/** + * CANCELLED/ERROR/FINISHED + * ▲ + * │ + * ┌───────────────────────────────────────────────┐ + * │ ┌──►RUNNING◄──┐ │ + * INIT───►│ │ │ │ + * │ ▼ ▼ │ + * │ WAIT_FOR_NOTIFY/WATITING◄────────►IO_IN/OUT │ + * └───────────────────────────────────────────────┘ + */ +enum class ExecTaskStatus +{ + WAIT_FOR_NOTIFY, + WAITING, + RUNNING, + IO_IN, + IO_OUT, + FINISHED, + ERROR, + CANCELLED, +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/IOEventTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/IOEventTask.h index 7fa435bef22..6e206e9bcfa 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/IOEventTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/IOEventTask.h @@ -27,7 +27,7 @@ class IOEventTask : public EventTask {} private: - ExecTaskStatus executeImpl() final + ReturnStatus executeImpl() final { if constexpr (is_input) return ExecTaskStatus::IO_IN; @@ -35,7 +35,7 @@ class IOEventTask : public EventTask return ExecTaskStatus::IO_OUT; } - ExecTaskStatus awaitImpl() final + ReturnStatus awaitImpl() final { if constexpr (is_input) return ExecTaskStatus::IO_IN; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.cpp index 7a17ed5f4e8..74e748c9a3c 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.cpp @@ -17,7 +17,7 @@ namespace DB { -ExecTaskStatus LoadBucketTask::executeIOImpl() +ReturnStatus LoadBucketTask::executeIOImpl() { input.load(); return ExecTaskStatus::FINISHED; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.h index f4d452f35ec..2d317eca8ce 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.h @@ -33,7 +33,7 @@ class LoadBucketTask : public InputIOEventTask {} private: - ExecTaskStatus executeIOImpl() override; + ReturnStatus executeIOImpl() override; private: SpilledBucketInput & input; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h new file mode 100644 index 00000000000..408150a1566 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h @@ -0,0 +1,28 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +namespace DB +{ +struct NotifyFuture +{ + virtual ~NotifyFuture() = default; + virtual void registerTask(TaskPtr && task); +}; +using NotifyFuturePtr = std::shared_ptr; + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h new file mode 100644 index 00000000000..df233d0b851 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -0,0 +1,44 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include + +namespace DB +{ +// Must have lock to use this class +class PipeConditionVariable +{ +public: + void registerTask(TaskPtr && task) + { + tasks.push_back(std::move(task)); + } + + void notifyOne() + { + if (!tasks.empty()) + { + auto task = std::move(tasks.back()); + tasks.pop_back(); + TaskScheduler::instance->submitToCPUTaskThreadPool(std::move(task)); + } + } + +private: + std::vector tasks; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h index 23f2e4ac9c8..fcc2ff804ff 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h @@ -35,11 +35,11 @@ class PipelineTask {} protected: - ExecTaskStatus executeImpl() override { return runExecute(); } + ReturnStatus executeImpl() override { return runExecute(); } - ExecTaskStatus executeIOImpl() override { return runExecuteIO(); } + ReturnStatus executeIOImpl() override { return runExecuteIO(); } - ExecTaskStatus awaitImpl() override { return runAwait(); } + ReturnStatus awaitImpl() override { return runAwait(); } void doFinalizeImpl() override { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h index e9b1cd657f9..b25a0d4c188 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h @@ -66,7 +66,7 @@ class PipelineTaskBase } protected: - ExecTaskStatus runExecute() + ReturnStatus runExecute() { assert(pipeline_exec); auto op_status = (pipeline_exec)->execute(); @@ -82,7 +82,7 @@ class PipelineTaskBase } } - ExecTaskStatus runExecuteIO() + ReturnStatus runExecuteIO() { assert(pipeline_exec); auto op_status = (pipeline_exec)->executeIO(); @@ -101,7 +101,7 @@ class PipelineTaskBase } } - ExecTaskStatus runAwait() + ReturnStatus runAwait() { assert(pipeline_exec); auto op_status = (pipeline_exec)->await(); diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/RFWaitTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/RFWaitTask.h index 245d45f8e30..b2f23653f98 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/RFWaitTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/RFWaitTask.h @@ -75,9 +75,9 @@ class RFWaitTask : public Task } private: - ExecTaskStatus executeImpl() override { return ExecTaskStatus::WAITING; } + ReturnStatus executeImpl() override { return ExecTaskStatus::WAITING; } - ExecTaskStatus awaitImpl() override + ReturnStatus awaitImpl() override { filterAndMoveReadyRfs(waiting_rf_list, ready_rf_list); if (waiting_rf_list.empty() || stopwatch.elapsed() >= max_wait_time_ns) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/SimplePipelineTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/SimplePipelineTask.h index c713bfe95f5..6552fb712bf 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/SimplePipelineTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/SimplePipelineTask.h @@ -34,11 +34,11 @@ class SimplePipelineTask {} protected: - ExecTaskStatus executeImpl() override { return runExecute(); } + ReturnStatus executeImpl() override { return runExecute(); } - ExecTaskStatus executeIOImpl() override { return runExecuteIO(); } + ReturnStatus executeIOImpl() override { return runExecuteIO(); } - ExecTaskStatus awaitImpl() override { return runAwait(); } + ReturnStatus awaitImpl() override { return runAwait(); } void finalizeImpl() override { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.cpp index d075b22cd16..5710371eed7 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.cpp @@ -20,7 +20,7 @@ namespace DB { namespace { -ALWAYS_INLINE ExecTaskStatus tryPushBlock(const ResultQueuePtr & result_queue, Block & block) +ALWAYS_INLINE ReturnStatus tryPushBlock(const ResultQueuePtr & result_queue, Block & block) { assert(block); auto ret = result_queue->tryPush(std::move(block)); @@ -53,12 +53,12 @@ StreamRestoreTask::StreamRestoreTask( assert(result_queue); } -ExecTaskStatus StreamRestoreTask::executeImpl() +ReturnStatus StreamRestoreTask::executeImpl() { return is_done ? ExecTaskStatus::FINISHED : ExecTaskStatus::IO_IN; } -ExecTaskStatus StreamRestoreTask::awaitImpl() +ReturnStatus StreamRestoreTask::awaitImpl() { if (unlikely(is_done)) return ExecTaskStatus::FINISHED; @@ -68,7 +68,7 @@ ExecTaskStatus StreamRestoreTask::awaitImpl() return tryPushBlock(result_queue, block); } -ExecTaskStatus StreamRestoreTask::executeIOImpl() +ReturnStatus StreamRestoreTask::executeIOImpl() { if (!block) { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.h index 2d6aa7f08d7..f3054f330b6 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.h @@ -33,11 +33,11 @@ class StreamRestoreTask : public Task const ResultQueuePtr & result_queue_); protected: - ExecTaskStatus executeImpl() override; + ReturnStatus executeImpl() override; - ExecTaskStatus awaitImpl() override; + ReturnStatus awaitImpl() override; - ExecTaskStatus executeIOImpl() override; + ReturnStatus executeIOImpl() override; void finalizeImpl() override; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index cf56b1267a1..e47e1811dc9 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -112,11 +112,11 @@ Task::~Task() } \ try \ { \ - auto status = (function()); \ + auto return_status = (function()); \ FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_task_run_failpoint); \ FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_query_run); \ - switchStatus(status); \ - return task_status; \ + switchStatus(return_status.status); \ + return return_status; \ } \ catch (...) \ { \ @@ -126,21 +126,21 @@ Task::~Task() return task_status; \ } -ExecTaskStatus Task::execute() +ReturnStatus Task::execute() { assert(mem_tracker_ptr == current_memory_tracker); assert(task_status == ExecTaskStatus::RUNNING); EXECUTE(executeImpl); } -ExecTaskStatus Task::executeIO() +ReturnStatus Task::executeIO() { assert(mem_tracker_ptr == current_memory_tracker); assert(task_status == ExecTaskStatus::IO_IN || task_status == ExecTaskStatus::IO_OUT); EXECUTE(executeIOImpl); } -ExecTaskStatus Task::await() +ReturnStatus Task::await() { // Because await only performs polling checks and does not involve computing/memory tracker memory allocation, // await will not invoke MemoryTracker, so current_memory_tracker must be nullptr here. diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 57c8bf2f408..70dbe0ed11d 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -17,34 +17,29 @@ #include #include #include +#include #include #include namespace DB { -/** - * CANCELLED/ERROR/FINISHED - * ▲ - * │ - * ┌───────────────────────────────┐ - * │ ┌──►RUNNING◄──┐ │ - * INIT───►│ │ │ │ - * │ ▼ ▼ │ - * │ WATITING◄────────►IO_IN/OUT │ - * └───────────────────────────────┘ - */ -enum class ExecTaskStatus +class PipelineExecutorContext; + +struct NotifyFuture; +using NotifyFuturePtr = std::shared_ptr; + +struct ReturnStatus { - WAITING, - RUNNING, - IO_IN, - IO_OUT, - FINISHED, - ERROR, - CANCELLED, -}; + ReturnStatus(ExecTaskStatus status_) // NOLINT(google-explicit-constructor) + : status(status_) + , future(nullptr) + { + assert(status != ExecTaskStatus::WAIT_FOR_NOTIFY); + } -class PipelineExecutorContext; + ExecTaskStatus status; + NotifyFuturePtr future{nullptr}; +}; class Task { @@ -61,11 +56,11 @@ class Task ExecTaskStatus getStatus() const { return task_status; } - ExecTaskStatus execute(); + ReturnStatus execute(); - ExecTaskStatus executeIO(); + ReturnStatus executeIO(); - ExecTaskStatus await(); + ReturnStatus await(); // `finalize` must be called before destructuring. // `TaskHelper::FINALIZE_TASK` can help this. @@ -95,10 +90,10 @@ class Task LoggerPtr log; protected: - virtual ExecTaskStatus executeImpl() = 0; - virtual ExecTaskStatus executeIOImpl() { return ExecTaskStatus::RUNNING; } + virtual ReturnStatus executeImpl() = 0; + virtual ReturnStatus executeIOImpl() { return ExecTaskStatus::RUNNING; } // Avoid allocating memory in `await` if possible. - virtual ExecTaskStatus awaitImpl() { return ExecTaskStatus::RUNNING; } + virtual ReturnStatus awaitImpl() { return ExecTaskStatus::RUNNING; } // Used to release held resources, just like `Event::finishImpl`. virtual void finalizeImpl() {} diff --git a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp index 2e645fca2c3..8d246ef8049 100644 --- a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -96,20 +97,20 @@ void TaskThreadPool::handleTask(TaskPtr & task) metrics.elapsedPendingTime(task); ExecTaskStatus status_before_exec = task->getStatus(); - ExecTaskStatus status_after_exec = status_before_exec; + ReturnStatus return_status_after_exec = status_before_exec; UInt64 total_time_spent = 0; while (true) { - status_after_exec = Impl::exec(task); + return_status_after_exec = Impl::exec(task); total_time_spent += task->profile_info.elapsedFromPrev(); // The executing task should yield if it takes more than `YIELD_MAX_TIME_SPENT_NS`. - if (!Impl::isTargetStatus(status_after_exec) || total_time_spent >= YIELD_MAX_TIME_SPENT_NS) + if (!Impl::isTargetStatus(return_status_after_exec.status) || total_time_spent >= YIELD_MAX_TIME_SPENT_NS) break; } task_queue->updateStatistics(task, status_before_exec, total_time_spent); metrics.addExecuteTime(task, total_time_spent); metrics.decExecutingTask(); - switch (status_after_exec) + switch (return_status_after_exec.status) { case ExecTaskStatus::RUNNING: task->endTraceMemory(); @@ -124,13 +125,17 @@ void TaskThreadPool::handleTask(TaskPtr & task) task->endTraceMemory(); scheduler.submitToWaitReactor(std::move(task)); break; + case ExecTaskStatus::WAIT_FOR_NOTIFY: + assert(return_status_after_exec.future); + return_status_after_exec.future->registerTask(std::move(task)); + break;; case FINISH_STATUS: task->finalize(); task->endTraceMemory(); task.reset(); break; default: - UNEXPECTED_STATUS(task->log, status_after_exec); + UNEXPECTED_STATUS(task->log, return_status_after_exec.status); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h index 5c7e1984148..13923a6e8c9 100644 --- a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h +++ b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h @@ -29,7 +29,7 @@ struct CPUImpl static bool isTargetStatus(ExecTaskStatus status) { return status == ExecTaskStatus::RUNNING; } - static ExecTaskStatus exec(TaskPtr & task) { return task->execute(); } + static ReturnStatus exec(TaskPtr & task) { return task->execute(); } static TaskQueuePtr newTaskQueue(TaskQueueType type); }; @@ -45,7 +45,7 @@ struct IOImpl return status == ExecTaskStatus::IO_IN || status == ExecTaskStatus::IO_OUT; } - static ExecTaskStatus exec(TaskPtr & task) { return task->executeIO(); } + static ReturnStatus exec(TaskPtr & task) { return task->executeIO(); } static TaskQueuePtr newTaskQueue(TaskQueueType type); }; diff --git a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp index c69a84b3a30..e915a55effc 100644 --- a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp @@ -32,7 +32,7 @@ class SimpleTask : public Task {} protected: - ExecTaskStatus executeImpl() noexcept override + ReturnStatus executeImpl() noexcept override { while ((--loop_count) > 0) return ExecTaskStatus::RUNNING; @@ -51,7 +51,7 @@ class SimpleWaitingTask : public Task {} protected: - ExecTaskStatus executeImpl() noexcept override + ReturnStatus executeImpl() noexcept override { if (loop_count > 0) { @@ -66,7 +66,7 @@ class SimpleWaitingTask : public Task return ExecTaskStatus::FINISHED; } - ExecTaskStatus awaitImpl() noexcept override + ReturnStatus awaitImpl() noexcept override { if (loop_count > 0) { @@ -93,7 +93,7 @@ class SimpleBlockedTask : public Task {} protected: - ExecTaskStatus executeImpl() override + ReturnStatus executeImpl() override { if (loop_count > 0) { @@ -108,7 +108,7 @@ class SimpleBlockedTask : public Task return ExecTaskStatus::FINISHED; } - ExecTaskStatus executeIOImpl() override + ReturnStatus executeIOImpl() override { if (loop_count > 0) { @@ -140,19 +140,19 @@ class MemoryTraceTask : public Task static constexpr Int64 MEMORY_TRACER_SUBMIT_THRESHOLD = 1024 * 1024; // 1 MiB protected: - ExecTaskStatus executeImpl() noexcept override + ReturnStatus executeImpl() noexcept override { CurrentMemoryTracker::alloc(MEMORY_TRACER_SUBMIT_THRESHOLD - 10); return ExecTaskStatus::IO_IN; } - ExecTaskStatus executeIOImpl() override + ReturnStatus executeIOImpl() override { CurrentMemoryTracker::alloc(MEMORY_TRACER_SUBMIT_THRESHOLD + 10); return ExecTaskStatus::WAITING; } - ExecTaskStatus awaitImpl() override + ReturnStatus awaitImpl() override { // await wouldn't call MemoryTracker. return ExecTaskStatus::FINISHED; @@ -167,11 +167,11 @@ class DeadLoopTask : public Task {} protected: - ExecTaskStatus executeImpl() override { return ExecTaskStatus::WAITING; } + ReturnStatus executeImpl() override { return ExecTaskStatus::WAITING; } - ExecTaskStatus awaitImpl() override { return ExecTaskStatus::IO_IN; } + ReturnStatus awaitImpl() override { return ExecTaskStatus::IO_IN; } - ExecTaskStatus executeIOImpl() override { return ExecTaskStatus::RUNNING; } + ReturnStatus executeIOImpl() override { return ExecTaskStatus::RUNNING; } }; } // namespace diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h index 1b7f3bfd603..ac2ad29f024 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h @@ -20,6 +20,8 @@ #include #include +#include + namespace DB::DM { template @@ -30,6 +32,7 @@ class WorkQueue std::condition_variable reader_cv; std::condition_variable writer_cv; std::condition_variable finish_cv; + PipeConditionVariable pipe_cv; std::queue queue; bool done; std::size_t max_size; @@ -61,6 +64,15 @@ class WorkQueue , pop_times(0) , pop_empty_times(0) {} + + void registerPipeTask(TaskPtr && task) + { + std::lock_guard lock(mu); + pipe_cv.registerTask(std::move(task)); + if (!queue.empty() || done) + pipe_cv.notifyOne(); + } + /** * Push an item onto the work queue. Notify a single thread that work is * available. If `finish()` has been called, do nothing and return false. @@ -81,6 +93,7 @@ class WorkQueue } if (done) { + pipe_cv.notifyOne(); return false; } queue.push(std::forward(item)); @@ -89,6 +102,7 @@ class WorkQueue { *size = queue.size(); } + pipe_cv.notifyOne(); } reader_cv.notify_one(); return true; diff --git a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h index a695fd9846a..1f200cab82e 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h @@ -167,6 +167,11 @@ class SegmentReadTaskPool : private boost::noncopyable std::once_flag & addToSchedulerFlag() { return add_to_scheduler; } + void registerPipeTask(TaskPtr && task) + { + q.registerPipeTask(std::move(task)); + } + public: const uint64_t pool_id; @@ -175,7 +180,7 @@ class SegmentReadTaskPool : private boost::noncopyable ColumnDefines & getColumnToRead() { return columns_to_read; } - void appendRSOperator(RSOperatorPtr & new_filter) + void appendRSOperator(RSOperatorPtr & new_filter) const { if (filter->rs_operator == DM::EMPTY_RS_OPERATOR) { From c0321fb4523d82c3a3896ae3426163255da2e21e Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 16:17:27 +0800 Subject: [PATCH 02/19] operator --- dbms/src/Common/TiFlashMetrics.h | 1 + dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp | 40 +++++----- dbms/src/Flash/Pipeline/Exec/PipelineExec.h | 14 ++-- .../Exec/tests/gtest_simple_operator.cpp | 6 +- .../Pipeline/Schedule/Reactor/WaitReactor.cpp | 2 +- .../TaskQueues/tests/bench_task_queue.cpp | 2 +- .../Schedule/Tasks/PipeConditionVariable.h | 4 +- .../Schedule/Tasks/PipelineTaskBase.h | 62 ++++++++-------- .../Flash/Pipeline/Schedule/Tasks/Task.cpp | 7 ++ dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 9 +++ .../Schedule/ThreadPool/TaskThreadPool.cpp | 5 +- .../AddExtraTableIDColumnTransformOp.cpp | 2 +- .../AddExtraTableIDColumnTransformOp.h | 2 +- dbms/src/Operators/AggregateBuildSinkOp.cpp | 6 +- dbms/src/Operators/AggregateBuildSinkOp.h | 6 +- .../Operators/AggregateConvergentSourceOp.cpp | 2 +- .../Operators/AggregateConvergentSourceOp.h | 2 +- .../Operators/AggregateRestoreSourceOp.cpp | 4 +- dbms/src/Operators/AggregateRestoreSourceOp.h | 4 +- .../Operators/BlockInputStreamSourceOp.cpp | 2 +- dbms/src/Operators/BlockInputStreamSourceOp.h | 2 +- dbms/src/Operators/ConcatSourceOp.h | 14 ++-- .../Operators/CoprocessorReaderSourceOp.cpp | 6 +- .../src/Operators/CoprocessorReaderSourceOp.h | 4 +- .../src/Operators/DMSegmentThreadSourceOp.cpp | 4 +- dbms/src/Operators/DMSegmentThreadSourceOp.h | 4 +- .../Operators/ExchangeReceiverSourceOp.cpp | 6 +- dbms/src/Operators/ExchangeReceiverSourceOp.h | 4 +- dbms/src/Operators/ExchangeSenderSinkOp.cpp | 6 +- dbms/src/Operators/ExchangeSenderSinkOp.h | 6 +- dbms/src/Operators/Expand2TransformOp.cpp | 4 +- dbms/src/Operators/Expand2TransformOp.h | 4 +- dbms/src/Operators/ExpressionTransformOp.cpp | 2 +- dbms/src/Operators/ExpressionTransformOp.h | 2 +- dbms/src/Operators/FilterTransformOp.cpp | 2 +- dbms/src/Operators/FilterTransformOp.h | 2 +- .../GeneratedColumnPlaceHolderTransformOp.cpp | 2 +- .../GeneratedColumnPlaceHolderTransformOp.h | 2 +- dbms/src/Operators/GetResultSinkOp.cpp | 6 +- dbms/src/Operators/GetResultSinkOp.h | 6 +- dbms/src/Operators/HashJoinBuildSink.cpp | 6 +- dbms/src/Operators/HashJoinBuildSink.h | 6 +- .../Operators/HashJoinProbeTransformOp.cpp | 16 ++-- dbms/src/Operators/HashJoinProbeTransformOp.h | 10 +-- dbms/src/Operators/HashProbeTransformExec.cpp | 6 +- dbms/src/Operators/HashProbeTransformExec.h | 6 +- .../Operators/IOBlockInputStreamSourceOp.h | 4 +- dbms/src/Operators/LimitTransformOp.cpp | 2 +- dbms/src/Operators/LimitTransformOp.h | 2 +- .../src/Operators/LocalAggregateTransform.cpp | 10 +-- dbms/src/Operators/LocalAggregateTransform.h | 10 +-- dbms/src/Operators/MergeSortTransformOp.cpp | 14 ++-- dbms/src/Operators/MergeSortTransformOp.h | 14 ++-- dbms/src/Operators/NullSourceOp.h | 2 +- dbms/src/Operators/Operator.cpp | 74 ++++++++++--------- dbms/src/Operators/Operator.h | 50 +++++++++---- dbms/src/Operators/OperatorHelper.cpp | 1 + dbms/src/Operators/PartialSortTransformOp.cpp | 2 +- dbms/src/Operators/PartialSortTransformOp.h | 2 +- dbms/src/Operators/SharedQueue.cpp | 12 +-- dbms/src/Operators/SharedQueue.h | 10 +-- dbms/src/Operators/UnorderedSourceOp.cpp | 6 +- dbms/src/Operators/UnorderedSourceOp.h | 4 +- dbms/src/Operators/WindowTransformOp.cpp | 4 +- dbms/src/Operators/WindowTransformOp.h | 4 +- .../Operators/tests/gtest_concat_source.cpp | 2 +- .../DeltaMerge/ReadThread/WorkQueue.h | 3 +- .../DeltaMerge/Remote/RNSegmentSourceOp.cpp | 8 +- .../DeltaMerge/Remote/RNSegmentSourceOp.h | 8 +- .../Storages/DeltaMerge/SegmentReadTaskPool.h | 5 +- 70 files changed, 312 insertions(+), 261 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 27ed6fa60ca..36ae631b072 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -745,6 +745,7 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva "pipeline task change to status", \ Counter, \ F(type_to_waiting, {"type", "to_waiting"}), \ + F(type_to_wait_for_notify, {"type", "to_wait_for_notify"}), \ F(type_to_running, {"type", "to_running"}), \ F(type_to_io, {"type", "to_io"}), \ F(type_to_finished, {"type", "to_finished"}), \ diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp index b939713e77a..9b13b7a9227 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp @@ -25,7 +25,7 @@ extern const char random_pipeline_model_execute_suffix_failpoint[]; } // namespace FailPoints #define HANDLE_OP_STATUS(op, op_status, expect_status) \ - switch (op_status) \ + switch ((op_status).status) \ { \ /* For the expected status, it will not return here, */ \ /* but instead return control to the macro caller, */ \ @@ -41,14 +41,14 @@ extern const char random_pipeline_model_execute_suffix_failpoint[]; case OperatorStatus::WAITING: \ fillAwaitable((op).get()); \ return (op_status); \ - /* For unexpected status, an immediate return is required. */ \ + /* For other status, an immediate return is required. */ \ default: \ return (op_status); \ } #define HANDLE_LAST_OP_STATUS(op, op_status) \ assert(op); \ - switch (op_status) \ + switch ((op_status).status) \ { \ /* For the io status, the operator needs to be filled in io_op for later use in executeIO. */ \ case OperatorStatus::IO_IN: \ @@ -89,12 +89,12 @@ void PipelineExec::executeSuffix() source_op->operateSuffix(); } -OperatorStatus PipelineExec::execute() +ReturnOpStatus PipelineExec::execute() { auto op_status = executeImpl(); #ifndef NDEBUG // `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute`. - assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT}); + assertOperatorStatus(op_status.status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT}); #endif return op_status; } @@ -105,13 +105,13 @@ OperatorStatus PipelineExec::execute() * │ block * write◄────transform◄─── ... ◄───transform◄────────────┘ */ -OperatorStatus PipelineExec::executeImpl() +ReturnOpStatus PipelineExec::executeImpl() { Block block; size_t start_transform_op_index = 0; auto op_status = fetchBlock(block, start_transform_op_index); // If the status `fetchBlock` returns isn't `HAS_OUTPUT`, it means that `fetchBlock` did not return a block. - if (op_status != OperatorStatus::HAS_OUTPUT) + if (op_status.status != OperatorStatus::HAS_OUTPUT) return op_status; // start from the next transform op after fetched block transform op. @@ -127,7 +127,7 @@ OperatorStatus PipelineExec::executeImpl() } // try fetch block from transform_ops and source_op. -OperatorStatus PipelineExec::fetchBlock(Block & block, size_t & start_transform_op_index) +ReturnOpStatus PipelineExec::fetchBlock(Block & block, size_t & start_transform_op_index) { auto op_status = sink_op->prepare(); HANDLE_OP_STATUS(sink_op, op_status, OperatorStatus::NEED_INPUT); @@ -144,44 +144,48 @@ OperatorStatus PipelineExec::fetchBlock(Block & block, size_t & start_transform_ HANDLE_LAST_OP_STATUS(source_op, op_status); } -OperatorStatus PipelineExec::executeIO() +ReturnOpStatus PipelineExec::executeIO() { auto op_status = executeIOImpl(); #ifndef NDEBUG // `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute`. // `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute`. - assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::HAS_OUTPUT, OperatorStatus::NEED_INPUT}); + assertOperatorStatus( + op_status.status, + {OperatorStatus::FINISHED, OperatorStatus::HAS_OUTPUT, OperatorStatus::NEED_INPUT}); #endif return op_status; } -OperatorStatus PipelineExec::executeIOImpl() +ReturnOpStatus PipelineExec::executeIOImpl() { assert(io_op); auto op_status = io_op->executeIO(); - if (op_status == OperatorStatus::WAITING) + if (op_status.status == OperatorStatus::WAITING) fillAwaitable(io_op); - if (op_status != OperatorStatus::IO_IN && op_status != OperatorStatus::IO_OUT) + if (op_status.status != OperatorStatus::IO_IN && op_status.status != OperatorStatus::IO_OUT) io_op = nullptr; return op_status; } -OperatorStatus PipelineExec::await() +ReturnOpStatus PipelineExec::await() { auto op_status = awaitImpl(); #ifndef NDEBUG // `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute`. // `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute`. - assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::HAS_OUTPUT, OperatorStatus::NEED_INPUT}); + assertOperatorStatus( + op_status.status, + {OperatorStatus::FINISHED, OperatorStatus::HAS_OUTPUT, OperatorStatus::NEED_INPUT}); #endif return op_status; } -OperatorStatus PipelineExec::awaitImpl() +ReturnOpStatus PipelineExec::awaitImpl() { assert(awaitable); auto op_status = awaitable->await(); - if (op_status == OperatorStatus::IO_IN || op_status == OperatorStatus::IO_OUT) + if (op_status.status == OperatorStatus::IO_IN || op_status.status == OperatorStatus::IO_OUT) fillIOOp(awaitable); - if (op_status != OperatorStatus::WAITING) + if (op_status.status != OperatorStatus::WAITING) awaitable = nullptr; return op_status; } diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.h b/dbms/src/Flash/Pipeline/Exec/PipelineExec.h index 275c019cc97..645de8e6925 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExec.h +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.h @@ -31,22 +31,22 @@ class PipelineExec : private boost::noncopyable void executePrefix(); void executeSuffix(); - OperatorStatus execute(); + ReturnOpStatus execute(); - OperatorStatus executeIO(); + ReturnOpStatus executeIO(); - OperatorStatus await(); + ReturnOpStatus await(); void finalizeProfileInfo(UInt64 extra_time); private: - inline OperatorStatus executeImpl(); + inline ReturnOpStatus executeImpl(); - inline OperatorStatus executeIOImpl(); + inline ReturnOpStatus executeIOImpl(); - inline OperatorStatus awaitImpl(); + inline ReturnOpStatus awaitImpl(); - inline OperatorStatus fetchBlock(Block & block, size_t & start_transform_op_index); + inline ReturnOpStatus fetchBlock(Block & block, size_t & start_transform_op_index); ALWAYS_INLINE void fillAwaitable(Operator * op) { diff --git a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp index 45b8ab09a3e..d026ddab77f 100644 --- a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp +++ b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp @@ -39,7 +39,7 @@ class SimpleGetResultSinkOp : public SinkOp String getName() const override { return "SimpleGetResultSinkOp"; } protected: - OperatorStatus writeImpl(Block && block) override + ReturnOpStatus writeImpl(Block && block) override { if (!block) return OperatorStatus::FINISHED; @@ -121,7 +121,7 @@ class SimpleOperatorTestRunner : public DB::tests::ExecutorTest }}; PipelineExecutorContext exec_context; auto op_pipeline = build(request, result_handler, exec_context); - while (op_pipeline->execute() != OperatorStatus::FINISHED) {} + while (op_pipeline->execute().status != OperatorStatus::FINISHED) {} ASSERT_COLUMNS_EQ_UR(expect_columns, vstackBlocks(std::move(blocks)).getColumnsWithTypeAndName()); } }; @@ -140,7 +140,7 @@ try PipelineExecutorContext exec_context; auto op_pipeline = build(request, result_handler, exec_context); exec_context.cancel(); - ASSERT_EQ(op_pipeline->execute(), OperatorStatus::CANCELLED); + ASSERT_EQ(op_pipeline->execute().status, OperatorStatus::CANCELLED); } CATCH diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp index aa53dda7162..dde678d37ef 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp @@ -18,8 +18,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp index c44cf9e5758..bf6b62dc066 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp @@ -33,7 +33,7 @@ namespace tests : Task(exec_context) \ {} \ \ - ReturnStatus executeImpl() override \ + ReturnStatus executeImpl() override \ { \ if (task_exec_cur_count <= task_exec_total_count) \ { \ diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index df233d0b851..143a627dbc9 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -14,8 +14,8 @@ #pragma once -#include #include +#include namespace DB { @@ -25,6 +25,7 @@ class PipeConditionVariable public: void registerTask(TaskPtr && task) { + assert(task->getStatus() == ExecTaskStatus::WAIT_FOR_NOTIFY); tasks.push_back(std::move(task)); } @@ -34,6 +35,7 @@ class PipeConditionVariable { auto task = std::move(tasks.back()); tasks.pop_back(); + task->notify(); TaskScheduler::instance->submitToCPUTaskThreadPool(std::move(task)); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h index b25a0d4c188..95fc498d918 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h @@ -29,26 +29,30 @@ namespace DB /// - OperatorStatus::WAITING ==> ExecTaskStatus::WAITING /// - OperatorStatus::NEED_INPUT/HAS_OUTPUT ==> ExecTaskStatus::RUNNING -#define MAP_NOT_RUNNING_TASK_STATUS \ - case OperatorStatus::FINISHED: \ - { \ - return ExecTaskStatus::FINISHED; \ - } \ - case OperatorStatus::CANCELLED: \ - { \ - return ExecTaskStatus::CANCELLED; \ - } \ - case OperatorStatus::IO_IN: \ - { \ - return ExecTaskStatus::IO_IN; \ - } \ - case OperatorStatus::IO_OUT: \ - { \ - return ExecTaskStatus::IO_OUT; \ - } \ - case OperatorStatus::WAITING: \ - { \ - return ExecTaskStatus::WAITING; \ +#define MAP_NOT_RUNNING_TASK_STATUS(op_status) \ + case OperatorStatus::FINISHED: \ + { \ + return ExecTaskStatus::FINISHED; \ + } \ + case OperatorStatus::CANCELLED: \ + { \ + return ExecTaskStatus::CANCELLED; \ + } \ + case OperatorStatus::IO_IN: \ + { \ + return ExecTaskStatus::IO_IN; \ + } \ + case OperatorStatus::IO_OUT: \ + { \ + return ExecTaskStatus::IO_OUT; \ + } \ + case OperatorStatus::WAITING: \ + { \ + return ExecTaskStatus::WAITING; \ + } \ + case OperatorStatus::WAIT_FOR_NOTIFY: \ + { \ + return std::move((op_status).future); \ } #define UNEXPECTED_OP_STATUS(op_status, function_name) \ @@ -70,15 +74,15 @@ class PipelineTaskBase { assert(pipeline_exec); auto op_status = (pipeline_exec)->execute(); - switch (op_status) + switch (op_status.status) { - MAP_NOT_RUNNING_TASK_STATUS + MAP_NOT_RUNNING_TASK_STATUS(op_status) /* After `pipeline_exec->execute`, `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute` */ /* And other states are unexpected. */ case OperatorStatus::NEED_INPUT: return ExecTaskStatus::RUNNING; default: - UNEXPECTED_OP_STATUS(op_status, "PipelineExec::execute"); + UNEXPECTED_OP_STATUS(op_status.status, "PipelineExec::execute"); } } @@ -86,9 +90,9 @@ class PipelineTaskBase { assert(pipeline_exec); auto op_status = (pipeline_exec)->executeIO(); - switch (op_status) + switch (op_status.status) { - MAP_NOT_RUNNING_TASK_STATUS + MAP_NOT_RUNNING_TASK_STATUS(op_status) /* After `pipeline_exec->executeIO`, */ /* - `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute` */ /* - `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute` */ @@ -97,7 +101,7 @@ class PipelineTaskBase case OperatorStatus::HAS_OUTPUT: return ExecTaskStatus::RUNNING; default: - UNEXPECTED_OP_STATUS(op_status, "PipelineExec::execute"); + UNEXPECTED_OP_STATUS(op_status.status, "PipelineExec::execute"); } } @@ -105,9 +109,9 @@ class PipelineTaskBase { assert(pipeline_exec); auto op_status = (pipeline_exec)->await(); - switch (op_status) + switch (op_status.status) { - MAP_NOT_RUNNING_TASK_STATUS + MAP_NOT_RUNNING_TASK_STATUS(op_status) /* After `pipeline_exec->await`, */ /* - `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute` */ /* - `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute` */ @@ -116,7 +120,7 @@ class PipelineTaskBase case OperatorStatus::HAS_OUTPUT: return ExecTaskStatus::RUNNING; default: - UNEXPECTED_OP_STATUS(op_status, "PipelineExec::await"); + UNEXPECTED_OP_STATUS(op_status.status, "PipelineExec::await"); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index e47e1811dc9..c9f4e130874 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -56,6 +56,7 @@ ALWAYS_INLINE void addToStatusMetrics(ExecTaskStatus to) switch (to) { M(ExecTaskStatus::WAITING, type_to_waiting) + M(ExecTaskStatus::WAIT_FOR_NOTIFY, type_to_wait_for_notify) M(ExecTaskStatus::RUNNING, type_to_running) M(ExecTaskStatus::IO_IN, type_to_io) M(ExecTaskStatus::IO_OUT, type_to_io) @@ -151,6 +152,12 @@ ReturnStatus Task::await() #undef EXECUTE +void Task::notify() +{ + assert(task_status == ExecTaskStatus::WAIT_FOR_NOTIFY); + switchStatus(ExecTaskStatus::RUNNING); +} + void Task::finalize() { // To make sure that `finalize` only called once. diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 70dbe0ed11d..04bace9add7 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -37,6 +37,13 @@ struct ReturnStatus assert(status != ExecTaskStatus::WAIT_FOR_NOTIFY); } + ReturnStatus(NotifyFuturePtr && furture_) // NOLINT(google-explicit-constructor) + : status(ExecTaskStatus::WAIT_FOR_NOTIFY) + , future(std::move(furture_)) + { + assert(future != nullptr); + } + ExecTaskStatus status; NotifyFuturePtr future{nullptr}; }; @@ -62,6 +69,8 @@ class Task ReturnStatus await(); + void notify(); + // `finalize` must be called before destructuring. // `TaskHelper::FINALIZE_TASK` can help this. void finalize(); diff --git a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp index 8d246ef8049..42620c623cd 100644 --- a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp @@ -17,10 +17,10 @@ #include #include #include +#include #include #include #include -#include #include #include @@ -128,7 +128,8 @@ void TaskThreadPool::handleTask(TaskPtr & task) case ExecTaskStatus::WAIT_FOR_NOTIFY: assert(return_status_after_exec.future); return_status_after_exec.future->registerTask(std::move(task)); - break;; + break; + ; case FINISH_STATUS: task->finalize(); task->endTraceMemory(); diff --git a/dbms/src/Operators/AddExtraTableIDColumnTransformOp.cpp b/dbms/src/Operators/AddExtraTableIDColumnTransformOp.cpp index 67ab63da3bc..1374f9c60ff 100644 --- a/dbms/src/Operators/AddExtraTableIDColumnTransformOp.cpp +++ b/dbms/src/Operators/AddExtraTableIDColumnTransformOp.cpp @@ -17,7 +17,7 @@ namespace DB { -OperatorStatus AddExtraTableIDColumnTransformOp::transformImpl(Block & block) +ReturnOpStatus AddExtraTableIDColumnTransformOp::transformImpl(Block & block) { if (!action.transform(block, physical_table_id)) block = {}; diff --git a/dbms/src/Operators/AddExtraTableIDColumnTransformOp.h b/dbms/src/Operators/AddExtraTableIDColumnTransformOp.h index 57905629bff..e790186b729 100644 --- a/dbms/src/Operators/AddExtraTableIDColumnTransformOp.h +++ b/dbms/src/Operators/AddExtraTableIDColumnTransformOp.h @@ -42,7 +42,7 @@ class AddExtraTableIDColumnTransformOp : public TransformOp IOProfileInfoPtr getIOProfileInfo() const override { return IOProfileInfo::createForLocal(profile_info_ptr); } protected: - OperatorStatus transformImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/AggregateBuildSinkOp.cpp b/dbms/src/Operators/AggregateBuildSinkOp.cpp index f3bc51d21b1..af027867742 100644 --- a/dbms/src/Operators/AggregateBuildSinkOp.cpp +++ b/dbms/src/Operators/AggregateBuildSinkOp.cpp @@ -17,7 +17,7 @@ namespace DB { -OperatorStatus AggregateBuildSinkOp::prepareImpl() +ReturnOpStatus AggregateBuildSinkOp::prepareImpl() { while (agg_context->hasLocalDataToBuild(index)) { @@ -28,7 +28,7 @@ OperatorStatus AggregateBuildSinkOp::prepareImpl() return agg_context->isTaskMarkedForSpill(index) ? OperatorStatus::IO_OUT : OperatorStatus::NEED_INPUT; } -OperatorStatus AggregateBuildSinkOp::writeImpl(Block && block) +ReturnOpStatus AggregateBuildSinkOp::writeImpl(Block && block) { if (unlikely(!block)) { @@ -44,7 +44,7 @@ OperatorStatus AggregateBuildSinkOp::writeImpl(Block && block) return agg_context->needSpill(index) ? OperatorStatus::IO_OUT : OperatorStatus::NEED_INPUT; } -OperatorStatus AggregateBuildSinkOp::executeIOImpl() +ReturnOpStatus AggregateBuildSinkOp::executeIOImpl() { agg_context->spillData(index); return is_final_spill ? OperatorStatus::FINISHED : OperatorStatus::NEED_INPUT; diff --git a/dbms/src/Operators/AggregateBuildSinkOp.h b/dbms/src/Operators/AggregateBuildSinkOp.h index 5bed80c5f68..0d08106e362 100644 --- a/dbms/src/Operators/AggregateBuildSinkOp.h +++ b/dbms/src/Operators/AggregateBuildSinkOp.h @@ -39,11 +39,11 @@ class AggregateBuildSinkOp : public SinkOp protected: void operateSuffixImpl() override; - OperatorStatus prepareImpl() override; + ReturnOpStatus prepareImpl() override; - OperatorStatus writeImpl(Block && block) override; + ReturnOpStatus writeImpl(Block && block) override; - OperatorStatus executeIOImpl() override; + ReturnOpStatus executeIOImpl() override; private: size_t index{}; diff --git a/dbms/src/Operators/AggregateConvergentSourceOp.cpp b/dbms/src/Operators/AggregateConvergentSourceOp.cpp index 053e28dbb4d..3e88efe9dcc 100644 --- a/dbms/src/Operators/AggregateConvergentSourceOp.cpp +++ b/dbms/src/Operators/AggregateConvergentSourceOp.cpp @@ -29,7 +29,7 @@ AggregateConvergentSourceOp::AggregateConvergentSourceOp( setHeader(agg_context->getHeader()); } -OperatorStatus AggregateConvergentSourceOp::readImpl(Block & block) +ReturnOpStatus AggregateConvergentSourceOp::readImpl(Block & block) { block = agg_context->readForConvergent(index); total_rows += block.rows(); diff --git a/dbms/src/Operators/AggregateConvergentSourceOp.h b/dbms/src/Operators/AggregateConvergentSourceOp.h index 9d7e6032a36..fbf888a7983 100644 --- a/dbms/src/Operators/AggregateConvergentSourceOp.h +++ b/dbms/src/Operators/AggregateConvergentSourceOp.h @@ -35,7 +35,7 @@ class AggregateConvergentSourceOp : public SourceOp protected: void operateSuffixImpl() override; - OperatorStatus readImpl(Block & block) override; + ReturnOpStatus readImpl(Block & block) override; private: AggregateContextPtr agg_context; diff --git a/dbms/src/Operators/AggregateRestoreSourceOp.cpp b/dbms/src/Operators/AggregateRestoreSourceOp.cpp index a1e29c0ffb4..e02ccabd062 100644 --- a/dbms/src/Operators/AggregateRestoreSourceOp.cpp +++ b/dbms/src/Operators/AggregateRestoreSourceOp.cpp @@ -30,12 +30,12 @@ AggregateRestoreSourceOp::AggregateRestoreSourceOp( setHeader(agg_context->getHeader()); } -OperatorStatus AggregateRestoreSourceOp::readImpl(Block & block) +ReturnOpStatus AggregateRestoreSourceOp::readImpl(Block & block) { return restorer->tryPop(block) ? OperatorStatus::HAS_OUTPUT : OperatorStatus::WAITING; } -OperatorStatus AggregateRestoreSourceOp::awaitImpl() +ReturnOpStatus AggregateRestoreSourceOp::awaitImpl() { return restorer->tryLoadBucketData() == SharedLoadResult::RETRY ? OperatorStatus::WAITING : OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/AggregateRestoreSourceOp.h b/dbms/src/Operators/AggregateRestoreSourceOp.h index 3fc474f0961..e05f6b4ad94 100644 --- a/dbms/src/Operators/AggregateRestoreSourceOp.h +++ b/dbms/src/Operators/AggregateRestoreSourceOp.h @@ -34,9 +34,9 @@ class AggregateRestoreSourceOp : public SourceOp String getName() const override { return "AggregateRestoreSourceOp"; } protected: - OperatorStatus readImpl(Block & block) override; + ReturnOpStatus readImpl(Block & block) override; - OperatorStatus awaitImpl() override; + ReturnOpStatus awaitImpl() override; private: AggregateContextPtr agg_context; diff --git a/dbms/src/Operators/BlockInputStreamSourceOp.cpp b/dbms/src/Operators/BlockInputStreamSourceOp.cpp index a47a12e4bb5..ece8d9133fe 100644 --- a/dbms/src/Operators/BlockInputStreamSourceOp.cpp +++ b/dbms/src/Operators/BlockInputStreamSourceOp.cpp @@ -38,7 +38,7 @@ void BlockInputStreamSourceOp::operateSuffixImpl() impl->readSuffix(); } -OperatorStatus BlockInputStreamSourceOp::readImpl(Block & block) +ReturnOpStatus BlockInputStreamSourceOp::readImpl(Block & block) { block = impl->read(); return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/BlockInputStreamSourceOp.h b/dbms/src/Operators/BlockInputStreamSourceOp.h index 22136c558c0..437ee8b7d12 100644 --- a/dbms/src/Operators/BlockInputStreamSourceOp.h +++ b/dbms/src/Operators/BlockInputStreamSourceOp.h @@ -38,7 +38,7 @@ class BlockInputStreamSourceOp : public SourceOp void operatePrefixImpl() override; void operateSuffixImpl() override; - OperatorStatus readImpl(Block & block) override; + ReturnOpStatus readImpl(Block & block) override; private: BlockInputStreamPtr impl; diff --git a/dbms/src/Operators/ConcatSourceOp.h b/dbms/src/Operators/ConcatSourceOp.h index 29f47be70e8..e96ece88750 100644 --- a/dbms/src/Operators/ConcatSourceOp.h +++ b/dbms/src/Operators/ConcatSourceOp.h @@ -34,7 +34,7 @@ class SetBlockSinkOp : public SinkOp String getName() const override { return "SetBlockSinkOp"; } protected: - OperatorStatus writeImpl(Block && block) override + ReturnOpStatus writeImpl(Block && block) override { if unlikely (!block) return OperatorStatus::FINISHED; @@ -89,7 +89,7 @@ class ConcatSourceOp : public SourceOp exec_pool.clear(); } - OperatorStatus readImpl(Block & block) override + ReturnOpStatus readImpl(Block & block) override { if unlikely (done) return OperatorStatus::HAS_OUTPUT; @@ -104,7 +104,7 @@ class ConcatSourceOp : public SourceOp { assert(cur_exec); auto status = cur_exec->execute(); - switch (status) + switch (status.status) { case OperatorStatus::NEED_INPUT: assert(res); @@ -125,25 +125,25 @@ class ConcatSourceOp : public SourceOp } } - OperatorStatus executeIOImpl() override + ReturnOpStatus executeIOImpl() override { if unlikely (done || res) return OperatorStatus::HAS_OUTPUT; assert(cur_exec); auto status = cur_exec->executeIO(); - assert(status != OperatorStatus::FINISHED); + assert(status.status != OperatorStatus::FINISHED); return status; } - OperatorStatus awaitImpl() override + ReturnOpStatus awaitImpl() override { if unlikely (done || res) return OperatorStatus::HAS_OUTPUT; assert(cur_exec); auto status = cur_exec->await(); - assert(status != OperatorStatus::FINISHED); + assert(status.status != OperatorStatus::FINISHED); return status; } diff --git a/dbms/src/Operators/CoprocessorReaderSourceOp.cpp b/dbms/src/Operators/CoprocessorReaderSourceOp.cpp index 45d6f3d32fb..0ce661271c9 100644 --- a/dbms/src/Operators/CoprocessorReaderSourceOp.cpp +++ b/dbms/src/Operators/CoprocessorReaderSourceOp.cpp @@ -54,7 +54,7 @@ Block CoprocessorReaderSourceOp::popFromBlockQueue() return block; } -OperatorStatus CoprocessorReaderSourceOp::readImpl(Block & block) +ReturnOpStatus CoprocessorReaderSourceOp::readImpl(Block & block) { if (!block_queue.empty()) { @@ -66,7 +66,7 @@ OperatorStatus CoprocessorReaderSourceOp::readImpl(Block & block) { assert(block_queue.empty()); auto await_status = awaitImpl(); - if (await_status == OperatorStatus::HAS_OUTPUT) + if (await_status.status == OperatorStatus::HAS_OUTPUT) { assert(reader_res); assert(reader_res->second || reader_res->first.finished); @@ -112,7 +112,7 @@ OperatorStatus CoprocessorReaderSourceOp::readImpl(Block & block) return await_status; } } -OperatorStatus CoprocessorReaderSourceOp::awaitImpl() +ReturnOpStatus CoprocessorReaderSourceOp::awaitImpl() { if unlikely (!block_queue.empty()) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/CoprocessorReaderSourceOp.h b/dbms/src/Operators/CoprocessorReaderSourceOp.h index 8acb6f255ed..520b9f84fca 100644 --- a/dbms/src/Operators/CoprocessorReaderSourceOp.h +++ b/dbms/src/Operators/CoprocessorReaderSourceOp.h @@ -36,8 +36,8 @@ class CoprocessorReaderSourceOp : public SourceOp void operatePrefixImpl() override; void operateSuffixImpl() override; - OperatorStatus readImpl(Block & block) override; - OperatorStatus awaitImpl() override; + ReturnOpStatus readImpl(Block & block) override; + ReturnOpStatus awaitImpl() override; IOProfileInfoPtr getIOProfileInfo() const override { return io_profile_info; } diff --git a/dbms/src/Operators/DMSegmentThreadSourceOp.cpp b/dbms/src/Operators/DMSegmentThreadSourceOp.cpp index 05d7a9d07aa..3248044f026 100644 --- a/dbms/src/Operators/DMSegmentThreadSourceOp.cpp +++ b/dbms/src/Operators/DMSegmentThreadSourceOp.cpp @@ -57,7 +57,7 @@ void DMSegmentThreadSourceOp::operateSuffixImpl() LOG_DEBUG(log, "Finish read {} rows from storage", total_rows); } -OperatorStatus DMSegmentThreadSourceOp::readImpl(Block & block) +ReturnOpStatus DMSegmentThreadSourceOp::readImpl(Block & block) { if (done) { @@ -74,7 +74,7 @@ OperatorStatus DMSegmentThreadSourceOp::readImpl(Block & block) return OperatorStatus::IO_IN; } -OperatorStatus DMSegmentThreadSourceOp::executeIOImpl() +ReturnOpStatus DMSegmentThreadSourceOp::executeIOImpl() { if (unlikely(done)) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/DMSegmentThreadSourceOp.h b/dbms/src/Operators/DMSegmentThreadSourceOp.h index 3e89c01fcce..6548c3be62c 100644 --- a/dbms/src/Operators/DMSegmentThreadSourceOp.h +++ b/dbms/src/Operators/DMSegmentThreadSourceOp.h @@ -49,9 +49,9 @@ class DMSegmentThreadSourceOp : public SourceOp protected: void operateSuffixImpl() override; - OperatorStatus readImpl(Block & block) override; + ReturnOpStatus readImpl(Block & block) override; - OperatorStatus executeIOImpl() override; + ReturnOpStatus executeIOImpl() override; private: DM::DMContextPtr dm_context; diff --git a/dbms/src/Operators/ExchangeReceiverSourceOp.cpp b/dbms/src/Operators/ExchangeReceiverSourceOp.cpp index 8e017fa497c..9e2fd624708 100644 --- a/dbms/src/Operators/ExchangeReceiverSourceOp.cpp +++ b/dbms/src/Operators/ExchangeReceiverSourceOp.cpp @@ -29,7 +29,7 @@ Block ExchangeReceiverSourceOp::popFromBlockQueue() return block; } -OperatorStatus ExchangeReceiverSourceOp::readImpl(Block & block) +ReturnOpStatus ExchangeReceiverSourceOp::readImpl(Block & block) { if (!block_queue.empty()) { @@ -41,7 +41,7 @@ OperatorStatus ExchangeReceiverSourceOp::readImpl(Block & block) { assert(block_queue.empty()); auto await_status = awaitImpl(); - if (await_status == OperatorStatus::HAS_OUTPUT) + if (await_status.status == OperatorStatus::HAS_OUTPUT) { assert(receive_status != ReceiveStatus::empty); auto result @@ -93,7 +93,7 @@ OperatorStatus ExchangeReceiverSourceOp::readImpl(Block & block) } } -OperatorStatus ExchangeReceiverSourceOp::awaitImpl() +ReturnOpStatus ExchangeReceiverSourceOp::awaitImpl() { if unlikely (!block_queue.empty()) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/ExchangeReceiverSourceOp.h b/dbms/src/Operators/ExchangeReceiverSourceOp.h index f7f1e86c99f..002ecf26cc5 100644 --- a/dbms/src/Operators/ExchangeReceiverSourceOp.h +++ b/dbms/src/Operators/ExchangeReceiverSourceOp.h @@ -46,9 +46,9 @@ class ExchangeReceiverSourceOp : public SourceOp protected: void operateSuffixImpl() override; - OperatorStatus readImpl(Block & block) override; + ReturnOpStatus readImpl(Block & block) override; - OperatorStatus awaitImpl() override; + ReturnOpStatus awaitImpl() override; private: Block popFromBlockQueue(); diff --git a/dbms/src/Operators/ExchangeSenderSinkOp.cpp b/dbms/src/Operators/ExchangeSenderSinkOp.cpp index 664e22e3cc5..c6d085a9ee8 100644 --- a/dbms/src/Operators/ExchangeSenderSinkOp.cpp +++ b/dbms/src/Operators/ExchangeSenderSinkOp.cpp @@ -26,7 +26,7 @@ void ExchangeSenderSinkOp::operateSuffixImpl() LOG_DEBUG(log, "finish write with {} rows", total_rows); } -OperatorStatus ExchangeSenderSinkOp::writeImpl(Block && block) +ReturnOpStatus ExchangeSenderSinkOp::writeImpl(Block && block) { if (!block) { @@ -39,12 +39,12 @@ OperatorStatus ExchangeSenderSinkOp::writeImpl(Block && block) return OperatorStatus::NEED_INPUT; } -OperatorStatus ExchangeSenderSinkOp::prepareImpl() +ReturnOpStatus ExchangeSenderSinkOp::prepareImpl() { return writer->isWritable() ? OperatorStatus::NEED_INPUT : OperatorStatus::WAITING; } -OperatorStatus ExchangeSenderSinkOp::awaitImpl() +ReturnOpStatus ExchangeSenderSinkOp::awaitImpl() { return writer->isWritable() ? OperatorStatus::NEED_INPUT : OperatorStatus::WAITING; } diff --git a/dbms/src/Operators/ExchangeSenderSinkOp.h b/dbms/src/Operators/ExchangeSenderSinkOp.h index b4e4702d518..37fe293275d 100644 --- a/dbms/src/Operators/ExchangeSenderSinkOp.h +++ b/dbms/src/Operators/ExchangeSenderSinkOp.h @@ -37,11 +37,11 @@ class ExchangeSenderSinkOp : public SinkOp void operatePrefixImpl() override; void operateSuffixImpl() override; - OperatorStatus writeImpl(Block && block) override; + ReturnOpStatus writeImpl(Block && block) override; - OperatorStatus prepareImpl() override; + ReturnOpStatus prepareImpl() override; - OperatorStatus awaitImpl() override; + ReturnOpStatus awaitImpl() override; private: std::unique_ptr writer; diff --git a/dbms/src/Operators/Expand2TransformOp.cpp b/dbms/src/Operators/Expand2TransformOp.cpp index 7be61ebd032..1f583db2cf3 100644 --- a/dbms/src/Operators/Expand2TransformOp.cpp +++ b/dbms/src/Operators/Expand2TransformOp.cpp @@ -16,7 +16,7 @@ namespace DB { -OperatorStatus Expand2TransformOp::transformImpl(Block & block) +ReturnOpStatus Expand2TransformOp::transformImpl(Block & block) { if (likely(block)) expand_transform_action.transform(block); @@ -24,7 +24,7 @@ OperatorStatus Expand2TransformOp::transformImpl(Block & block) return OperatorStatus::HAS_OUTPUT; } -OperatorStatus Expand2TransformOp::tryOutputImpl(Block & block) +ReturnOpStatus Expand2TransformOp::tryOutputImpl(Block & block) { if (expand_transform_action.tryOutput(block)) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/Expand2TransformOp.h b/dbms/src/Operators/Expand2TransformOp.h index fe42c8adc34..c7564d009fb 100644 --- a/dbms/src/Operators/Expand2TransformOp.h +++ b/dbms/src/Operators/Expand2TransformOp.h @@ -35,9 +35,9 @@ class Expand2TransformOp : public TransformOp String getName() const override { return "Expand2TransformOp"; } protected: - OperatorStatus transformImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; - OperatorStatus tryOutputImpl(Block & block) override; + ReturnOpStatus tryOutputImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/ExpressionTransformOp.cpp b/dbms/src/Operators/ExpressionTransformOp.cpp index b973f755ff5..159ba6e01f3 100644 --- a/dbms/src/Operators/ExpressionTransformOp.cpp +++ b/dbms/src/Operators/ExpressionTransformOp.cpp @@ -17,7 +17,7 @@ namespace DB { -OperatorStatus ExpressionTransformOp::transformImpl(Block & block) +ReturnOpStatus ExpressionTransformOp::transformImpl(Block & block) { if (likely(block)) expression->execute(block); diff --git a/dbms/src/Operators/ExpressionTransformOp.h b/dbms/src/Operators/ExpressionTransformOp.h index fc791b15246..700486ed5bf 100644 --- a/dbms/src/Operators/ExpressionTransformOp.h +++ b/dbms/src/Operators/ExpressionTransformOp.h @@ -36,7 +36,7 @@ class ExpressionTransformOp : public TransformOp String getName() const override { return "ExpressionTransformOp"; } protected: - OperatorStatus transformImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/FilterTransformOp.cpp b/dbms/src/Operators/FilterTransformOp.cpp index bc6751faef3..cba65a01d9f 100644 --- a/dbms/src/Operators/FilterTransformOp.cpp +++ b/dbms/src/Operators/FilterTransformOp.cpp @@ -16,7 +16,7 @@ namespace DB { -OperatorStatus FilterTransformOp::transformImpl(Block & block) +ReturnOpStatus FilterTransformOp::transformImpl(Block & block) { if (unlikely(filter_transform_action.alwaysFalse())) { diff --git a/dbms/src/Operators/FilterTransformOp.h b/dbms/src/Operators/FilterTransformOp.h index 61c05b3c810..8ed10d8af55 100644 --- a/dbms/src/Operators/FilterTransformOp.h +++ b/dbms/src/Operators/FilterTransformOp.h @@ -36,7 +36,7 @@ class FilterTransformOp : public TransformOp String getName() const override { return "FilterTransformOp"; } protected: - OperatorStatus transformImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.cpp b/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.cpp index c155d15d8f0..f52d75c2ec5 100644 --- a/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.cpp +++ b/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.cpp @@ -20,7 +20,7 @@ String GeneratedColumnPlaceHolderTransformOp::getName() const return "GeneratedColumnPlaceholderTransformOp"; } -OperatorStatus GeneratedColumnPlaceHolderTransformOp::transformImpl(Block & block) +ReturnOpStatus GeneratedColumnPlaceHolderTransformOp::transformImpl(Block & block) { action.transform(block); return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.h b/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.h index 89865fc6996..f56113e0c19 100644 --- a/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.h +++ b/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.h @@ -34,7 +34,7 @@ class GeneratedColumnPlaceHolderTransformOp : public TransformOp String getName() const override; protected: - OperatorStatus transformImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/GetResultSinkOp.cpp b/dbms/src/Operators/GetResultSinkOp.cpp index 16e8fc37a9e..334c85e6456 100644 --- a/dbms/src/Operators/GetResultSinkOp.cpp +++ b/dbms/src/Operators/GetResultSinkOp.cpp @@ -16,7 +16,7 @@ namespace DB { -OperatorStatus GetResultSinkOp::writeImpl(Block && block) +ReturnOpStatus GetResultSinkOp::writeImpl(Block && block) { if (!block) return OperatorStatus::FINISHED; @@ -37,12 +37,12 @@ OperatorStatus GetResultSinkOp::writeImpl(Block && block) } } -OperatorStatus GetResultSinkOp::prepareImpl() +ReturnOpStatus GetResultSinkOp::prepareImpl() { return awaitImpl(); } -OperatorStatus GetResultSinkOp::awaitImpl() +ReturnOpStatus GetResultSinkOp::awaitImpl() { if (!t_block) return OperatorStatus::NEED_INPUT; diff --git a/dbms/src/Operators/GetResultSinkOp.h b/dbms/src/Operators/GetResultSinkOp.h index dda52eec23c..c2b66a7b6ff 100644 --- a/dbms/src/Operators/GetResultSinkOp.h +++ b/dbms/src/Operators/GetResultSinkOp.h @@ -36,11 +36,11 @@ class GetResultSinkOp : public SinkOp String getName() const override { return "GetResultSinkOp"; } protected: - OperatorStatus writeImpl(Block && block) override; + ReturnOpStatus writeImpl(Block && block) override; - OperatorStatus prepareImpl() override; + ReturnOpStatus prepareImpl() override; - OperatorStatus awaitImpl() override; + ReturnOpStatus awaitImpl() override; private: ResultQueuePtr result_queue; diff --git a/dbms/src/Operators/HashJoinBuildSink.cpp b/dbms/src/Operators/HashJoinBuildSink.cpp index 5a4330c6d24..e6fdda95618 100644 --- a/dbms/src/Operators/HashJoinBuildSink.cpp +++ b/dbms/src/Operators/HashJoinBuildSink.cpp @@ -17,7 +17,7 @@ namespace DB { -OperatorStatus HashJoinBuildSink::writeImpl(Block && block) +ReturnOpStatus HashJoinBuildSink::writeImpl(Block && block) { if unlikely (!block) { @@ -35,13 +35,13 @@ OperatorStatus HashJoinBuildSink::writeImpl(Block && block) return join_ptr->hasBuildSideMarkedSpillData(op_index) ? OperatorStatus::IO_OUT : OperatorStatus::NEED_INPUT; } -OperatorStatus HashJoinBuildSink::prepareImpl() +ReturnOpStatus HashJoinBuildSink::prepareImpl() { join_ptr->checkAndMarkPartitionSpilledIfNeeded(op_index); return join_ptr->hasBuildSideMarkedSpillData(op_index) ? OperatorStatus::IO_OUT : OperatorStatus::NEED_INPUT; } -OperatorStatus HashJoinBuildSink::executeIOImpl() +ReturnOpStatus HashJoinBuildSink::executeIOImpl() { join_ptr->flushBuildSideMarkedSpillData(op_index); if (is_finish_status) diff --git a/dbms/src/Operators/HashJoinBuildSink.h b/dbms/src/Operators/HashJoinBuildSink.h index fd7788209a8..6f729ea8300 100644 --- a/dbms/src/Operators/HashJoinBuildSink.h +++ b/dbms/src/Operators/HashJoinBuildSink.h @@ -37,11 +37,11 @@ class HashJoinBuildSink : public SinkOp String getName() const override { return "HashJoinBuildSink"; } protected: - OperatorStatus writeImpl(Block && block) override; + ReturnOpStatus writeImpl(Block && block) override; - OperatorStatus prepareImpl() override; + ReturnOpStatus prepareImpl() override; - OperatorStatus executeIOImpl() override; + ReturnOpStatus executeIOImpl() override; private: JoinPtr join_ptr; diff --git a/dbms/src/Operators/HashJoinProbeTransformOp.cpp b/dbms/src/Operators/HashJoinProbeTransformOp.cpp index ee5de46641e..679d81ef66c 100644 --- a/dbms/src/Operators/HashJoinProbeTransformOp.cpp +++ b/dbms/src/Operators/HashJoinProbeTransformOp.cpp @@ -71,7 +71,7 @@ void HashJoinProbeTransformOp::operateSuffixImpl() scan_hash_map_rows); } -OperatorStatus HashJoinProbeTransformOp::onOutput(Block & block) +ReturnOpStatus HashJoinProbeTransformOp::onOutput(Block & block) { while (true) { @@ -83,7 +83,7 @@ OperatorStatus HashJoinProbeTransformOp::onOutput(Block & block) if (probe_process_info.all_rows_joined_finish) { if (auto ret = probe_transform->tryFillProcessInfoInRestoreProbeStage(probe_process_info); - ret != OperatorStatus::HAS_OUTPUT) + ret.status != OperatorStatus::HAS_OUTPUT) return ret; } case ProbeStatus::PROBE: @@ -147,23 +147,23 @@ OperatorStatus HashJoinProbeTransformOp::onOutput(Block & block) } } -OperatorStatus HashJoinProbeTransformOp::transformImpl(Block & block) +ReturnOpStatus HashJoinProbeTransformOp::transformImpl(Block & block) { assert(status == ProbeStatus::PROBE); assert(probe_process_info.all_rows_joined_finish); if (auto ret = probe_transform->tryFillProcessInfoInProbeStage(probe_process_info, block); - ret != OperatorStatus::HAS_OUTPUT) + ret.status != OperatorStatus::HAS_OUTPUT) return ret; return onOutput(block); } -OperatorStatus HashJoinProbeTransformOp::tryOutputImpl(Block & block) +ReturnOpStatus HashJoinProbeTransformOp::tryOutputImpl(Block & block) { if (status == ProbeStatus::PROBE && probe_process_info.all_rows_joined_finish) { if (auto ret = probe_transform->tryFillProcessInfoInProbeStage(probe_process_info); - ret != OperatorStatus::HAS_OUTPUT) + ret.status != OperatorStatus::HAS_OUTPUT) return ret; } @@ -206,7 +206,7 @@ void HashJoinProbeTransformOp::onGetRestoreJoin() } } -OperatorStatus HashJoinProbeTransformOp::awaitImpl() +ReturnOpStatus HashJoinProbeTransformOp::awaitImpl() { while (true) { @@ -240,7 +240,7 @@ OperatorStatus HashJoinProbeTransformOp::awaitImpl() } } -OperatorStatus HashJoinProbeTransformOp::executeIOImpl() +ReturnOpStatus HashJoinProbeTransformOp::executeIOImpl() { switch (status) { diff --git a/dbms/src/Operators/HashJoinProbeTransformOp.h b/dbms/src/Operators/HashJoinProbeTransformOp.h index 9be1164c5d9..fc586d1a2a1 100644 --- a/dbms/src/Operators/HashJoinProbeTransformOp.h +++ b/dbms/src/Operators/HashJoinProbeTransformOp.h @@ -34,20 +34,20 @@ class HashJoinProbeTransformOp : public TransformOp String getName() const override { return "HashJoinProbeTransformOp"; } protected: - OperatorStatus transformImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; - OperatorStatus tryOutputImpl(Block & block) override; + ReturnOpStatus tryOutputImpl(Block & block) override; - OperatorStatus awaitImpl() override; + ReturnOpStatus awaitImpl() override; - OperatorStatus executeIOImpl() override; + ReturnOpStatus executeIOImpl() override; void transformHeaderImpl(Block & header_) override; void operateSuffixImpl() override; private: - OperatorStatus onOutput(Block & block); + ReturnOpStatus onOutput(Block & block); inline void onWaitProbeFinishDone(); diff --git a/dbms/src/Operators/HashProbeTransformExec.cpp b/dbms/src/Operators/HashProbeTransformExec.cpp index a3260f38769..944479d569a 100644 --- a/dbms/src/Operators/HashProbeTransformExec.cpp +++ b/dbms/src/Operators/HashProbeTransformExec.cpp @@ -154,7 +154,7 @@ Block HashProbeTransformExec::popProbeRestoredBlock() return OperatorStatus::CANCELLED; \ continue; -OperatorStatus HashProbeTransformExec::tryFillProcessInfoInRestoreProbeStage(ProbeProcessInfo & probe_process_info) +ReturnOpStatus HashProbeTransformExec::tryFillProcessInfoInRestoreProbeStage(ProbeProcessInfo & probe_process_info) { while (true) { @@ -205,7 +205,7 @@ OperatorStatus HashProbeTransformExec::tryFillProcessInfoInRestoreProbeStage(Pro } } -OperatorStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage(ProbeProcessInfo & probe_process_info) +ReturnOpStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage(ProbeProcessInfo & probe_process_info) { while (true) { @@ -231,7 +231,7 @@ OperatorStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage(ProbeProce #undef CONTINUE -OperatorStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage( +ReturnOpStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage( ProbeProcessInfo & probe_process_info, Block & input) { diff --git a/dbms/src/Operators/HashProbeTransformExec.h b/dbms/src/Operators/HashProbeTransformExec.h index 55af07b88c4..61a250d9339 100644 --- a/dbms/src/Operators/HashProbeTransformExec.h +++ b/dbms/src/Operators/HashProbeTransformExec.h @@ -82,10 +82,10 @@ class HashProbeTransformExec : public std::enable_shared_from_thisreadSuffix(); } - OperatorStatus readImpl(Block & block) override + ReturnOpStatus readImpl(Block & block) override { if (unlikely(is_done)) return OperatorStatus::HAS_OUTPUT; @@ -53,7 +53,7 @@ class IOBlockInputStreamSourceOp : public SourceOp return OperatorStatus::HAS_OUTPUT; } - OperatorStatus executeIOImpl() override + ReturnOpStatus executeIOImpl() override { if unlikely (is_done || ret) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/LimitTransformOp.cpp b/dbms/src/Operators/LimitTransformOp.cpp index ad05e395bd2..508645316fe 100644 --- a/dbms/src/Operators/LimitTransformOp.cpp +++ b/dbms/src/Operators/LimitTransformOp.cpp @@ -18,7 +18,7 @@ namespace DB { template -OperatorStatus LimitTransformOp::transformImpl(Block & block) +ReturnOpStatus LimitTransformOp::transformImpl(Block & block) { if (!action->transform(block)) block = {}; diff --git a/dbms/src/Operators/LimitTransformOp.h b/dbms/src/Operators/LimitTransformOp.h index 8cbd20a4084..51fd6c7ac48 100644 --- a/dbms/src/Operators/LimitTransformOp.h +++ b/dbms/src/Operators/LimitTransformOp.h @@ -31,7 +31,7 @@ class LimitTransformOp : public TransformOp String getName() const override { return "LimitTransformOp"; } protected: - OperatorStatus transformImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/LocalAggregateTransform.cpp b/dbms/src/Operators/LocalAggregateTransform.cpp index a105e62406c..b0a4eff7c34 100644 --- a/dbms/src/Operators/LocalAggregateTransform.cpp +++ b/dbms/src/Operators/LocalAggregateTransform.cpp @@ -49,7 +49,7 @@ LocalAggregateTransform::LocalAggregateTransform( }); } -OperatorStatus LocalAggregateTransform::transformImpl(Block & block) +ReturnOpStatus LocalAggregateTransform::transformImpl(Block & block) { switch (status) { @@ -68,7 +68,7 @@ OperatorStatus LocalAggregateTransform::transformImpl(Block & block) } } -OperatorStatus LocalAggregateTransform::fromBuildToConvergent(Block & block) +ReturnOpStatus LocalAggregateTransform::fromBuildToConvergent(Block & block) { // status from build to convergent. assert(status == LocalAggStatus::build); @@ -79,7 +79,7 @@ OperatorStatus LocalAggregateTransform::fromBuildToConvergent(Block & block) return OperatorStatus::HAS_OUTPUT; } -OperatorStatus LocalAggregateTransform::fromBuildToFinalSpillOrRestore() +ReturnOpStatus LocalAggregateTransform::fromBuildToFinalSpillOrRestore() { assert(status == LocalAggStatus::build); if (agg_context.needSpill(task_index, /*try_mark_need_spill=*/true)) @@ -106,7 +106,7 @@ OperatorStatus LocalAggregateTransform::tryFromBuildToSpill() return OperatorStatus::NEED_INPUT; } -OperatorStatus LocalAggregateTransform::tryOutputImpl(Block & block) +ReturnOpStatus LocalAggregateTransform::tryOutputImpl(Block & block) { switch (status) { @@ -128,7 +128,7 @@ OperatorStatus LocalAggregateTransform::tryOutputImpl(Block & block) } } -OperatorStatus LocalAggregateTransform::executeIOImpl() +ReturnOpStatus LocalAggregateTransform::executeIOImpl() { switch (status) { diff --git a/dbms/src/Operators/LocalAggregateTransform.h b/dbms/src/Operators/LocalAggregateTransform.h index 235ab28c682..4f72d58ff7c 100644 --- a/dbms/src/Operators/LocalAggregateTransform.h +++ b/dbms/src/Operators/LocalAggregateTransform.h @@ -33,20 +33,20 @@ class LocalAggregateTransform : public TransformOp String getName() const override { return "LocalAggregateTransform"; } protected: - OperatorStatus transformImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; - OperatorStatus tryOutputImpl(Block & block) override; + ReturnOpStatus tryOutputImpl(Block & block) override; - OperatorStatus executeIOImpl() override; + ReturnOpStatus executeIOImpl() override; void transformHeaderImpl(Block & header_) override; private: OperatorStatus tryFromBuildToSpill(); - OperatorStatus fromBuildToConvergent(Block & block); + ReturnOpStatus fromBuildToConvergent(Block & block); - OperatorStatus fromBuildToFinalSpillOrRestore(); + ReturnOpStatus fromBuildToFinalSpillOrRestore(); private: Aggregator::Params params; diff --git a/dbms/src/Operators/MergeSortTransformOp.cpp b/dbms/src/Operators/MergeSortTransformOp.cpp index 6e25bee9658..4b91cffe227 100644 --- a/dbms/src/Operators/MergeSortTransformOp.cpp +++ b/dbms/src/Operators/MergeSortTransformOp.cpp @@ -51,7 +51,7 @@ Block MergeSortTransformOp::getMergeOutput() return block; } -OperatorStatus MergeSortTransformOp::fromPartialToMerge(Block & block) +ReturnOpStatus MergeSortTransformOp::fromPartialToMerge(Block & block) { assert(status == MergeSortStatus::PARTIAL); // convert to merge phase. @@ -72,7 +72,7 @@ OperatorStatus MergeSortTransformOp::fromPartialToMerge(Block & block) return OperatorStatus::HAS_OUTPUT; } -OperatorStatus MergeSortTransformOp::fromPartialToRestore() +ReturnOpStatus MergeSortTransformOp::fromPartialToRestore() { assert(status == MergeSortStatus::PARTIAL); // convert to restore phase. @@ -99,7 +99,7 @@ OperatorStatus MergeSortTransformOp::fromPartialToRestore() return OperatorStatus::IO_IN; } -OperatorStatus MergeSortTransformOp::fromPartialToSpill() +ReturnOpStatus MergeSortTransformOp::fromPartialToSpill() { assert(status == MergeSortStatus::PARTIAL); // convert to restore phase. @@ -121,7 +121,7 @@ OperatorStatus MergeSortTransformOp::fromPartialToSpill() return OperatorStatus::IO_OUT; } -OperatorStatus MergeSortTransformOp::fromSpillToPartial() +ReturnOpStatus MergeSortTransformOp::fromSpillToPartial() { assert(status == MergeSortStatus::SPILL); assert(cached_handler); @@ -133,7 +133,7 @@ OperatorStatus MergeSortTransformOp::fromSpillToPartial() return OperatorStatus::NEED_INPUT; } -OperatorStatus MergeSortTransformOp::transformImpl(Block & block) +ReturnOpStatus MergeSortTransformOp::transformImpl(Block & block) { switch (status) { @@ -162,7 +162,7 @@ OperatorStatus MergeSortTransformOp::transformImpl(Block & block) } } -OperatorStatus MergeSortTransformOp::tryOutputImpl(Block & block) +ReturnOpStatus MergeSortTransformOp::tryOutputImpl(Block & block) { switch (status) { @@ -195,7 +195,7 @@ OperatorStatus MergeSortTransformOp::tryOutputImpl(Block & block) } } -OperatorStatus MergeSortTransformOp::executeIOImpl() +ReturnOpStatus MergeSortTransformOp::executeIOImpl() { switch (status) { diff --git a/dbms/src/Operators/MergeSortTransformOp.h b/dbms/src/Operators/MergeSortTransformOp.h index d463b16c208..a377a351338 100644 --- a/dbms/src/Operators/MergeSortTransformOp.h +++ b/dbms/src/Operators/MergeSortTransformOp.h @@ -54,10 +54,10 @@ class MergeSortTransformOp : public TransformOp void operatePrefixImpl() override; void operateSuffixImpl() override; - OperatorStatus transformImpl(Block & block) override; - OperatorStatus tryOutputImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; + ReturnOpStatus tryOutputImpl(Block & block) override; - OperatorStatus executeIOImpl() override; + ReturnOpStatus executeIOImpl() override; void transformHeaderImpl(Block & header_) override; @@ -65,14 +65,14 @@ class MergeSortTransformOp : public TransformOp Block getMergeOutput(); // PARTIAL◄─────►SPILL - OperatorStatus fromPartialToSpill(); - OperatorStatus fromSpillToPartial(); + ReturnOpStatus fromPartialToSpill(); + ReturnOpStatus fromSpillToPartial(); // PARTIAL─────►RESTORE - OperatorStatus fromPartialToRestore(); + ReturnOpStatus fromPartialToRestore(); // PARTIAL─────►MERGE - OperatorStatus fromPartialToMerge(Block & block); + ReturnOpStatus fromPartialToMerge(Block & block); private: bool hasSpilledData() const { return sort_spill_context->hasSpilledData(); } diff --git a/dbms/src/Operators/NullSourceOp.h b/dbms/src/Operators/NullSourceOp.h index 43a03802522..0cf6775de0e 100644 --- a/dbms/src/Operators/NullSourceOp.h +++ b/dbms/src/Operators/NullSourceOp.h @@ -33,7 +33,7 @@ class NullSourceOp : public SourceOp IOProfileInfoPtr getIOProfileInfo() const override { return IOProfileInfo::createForLocal(profile_info_ptr); } protected: - OperatorStatus readImpl(Block & block) override + ReturnOpStatus readImpl(Block & block) override { block = {}; return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/Operator.cpp b/dbms/src/Operators/Operator.cpp index 109a339be41..14e2e07718b 100644 --- a/dbms/src/Operators/Operator.cpp +++ b/dbms/src/Operators/Operator.cpp @@ -45,14 +45,16 @@ void Operator::operateSuffix() if (unlikely(exec_context.isCancelled())) \ return OperatorStatus::CANCELLED; -OperatorStatus Operator::await() +ReturnOpStatus Operator::await() { // `exec_context.is_cancelled` has been checked by `EventTask`. // If `exec_context.is_cancelled` is checked here, the overhead of `exec_context.is_cancelled` will be amplified by the high frequency of `await` calls. - auto op_status = awaitImpl(); + auto return_status = awaitImpl(); #ifndef NDEBUG - assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus( + return_status.status, + {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); @@ -66,110 +68,112 @@ OperatorStatus Operator::await() // ┌────────────────waiting time───────────┐ // [non-waiting, waiting, waiting, waiting, .., waiting, non-waiting] - if (op_status != OperatorStatus::WAITING) + if (return_status.status != OperatorStatus::WAITING) { exec_context.triggerAutoSpill(); profile_info.update(); } - return op_status; + return return_status; } -OperatorStatus Operator::executeIO() +ReturnOpStatus Operator::executeIO() { CHECK_IS_CANCELLED profile_info.anchor(); - auto op_status = executeIOImpl(); + auto return_status = executeIOImpl(); #ifndef NDEBUG - assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus( + return_status.status, + {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return op_status; + return return_status; } -OperatorStatus SourceOp::read(Block & block) +ReturnOpStatus SourceOp::read(Block & block) { CHECK_IS_CANCELLED profile_info.anchor(); assert(!block); - auto op_status = readImpl(block); + auto return_status = readImpl(block); #ifndef NDEBUG - if (op_status == OperatorStatus::HAS_OUTPUT && block) + if (return_status.status == OperatorStatus::HAS_OUTPUT && block) { Block header = getHeader(); assertBlocksHaveEqualStructure(block, header, getName()); } - assertOperatorStatus(op_status, {OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus(return_status.status, {OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); - if (op_status == OperatorStatus::HAS_OUTPUT) + if (return_status.status == OperatorStatus::HAS_OUTPUT) profile_info.update(block); else profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return op_status; + return return_status; } -OperatorStatus TransformOp::transform(Block & block) +ReturnOpStatus TransformOp::transform(Block & block) { CHECK_IS_CANCELLED profile_info.anchor(); - auto op_status = transformImpl(block); + auto return_status = transformImpl(block); #ifndef NDEBUG - if (op_status == OperatorStatus::HAS_OUTPUT && block) + if (return_status.status == OperatorStatus::HAS_OUTPUT && block) { Block header = getHeader(); assertBlocksHaveEqualStructure(block, header, getName()); } - assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus(return_status.status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); - if (op_status == OperatorStatus::HAS_OUTPUT) + if (return_status.status == OperatorStatus::HAS_OUTPUT) profile_info.update(block); else profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return op_status; + return return_status; } -OperatorStatus TransformOp::tryOutput(Block & block) +ReturnOpStatus TransformOp::tryOutput(Block & block) { CHECK_IS_CANCELLED profile_info.anchor(); assert(!block); - auto op_status = tryOutputImpl(block); + auto return_status = tryOutputImpl(block); #ifndef NDEBUG - if (op_status == OperatorStatus::HAS_OUTPUT && block) + if (return_status.status == OperatorStatus::HAS_OUTPUT && block) { Block header = getHeader(); assertBlocksHaveEqualStructure(block, header, getName()); } - assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus(return_status.status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); - if (op_status == OperatorStatus::HAS_OUTPUT) + if (return_status.status == OperatorStatus::HAS_OUTPUT) profile_info.update(block); else profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return op_status; + return return_status; } -OperatorStatus SinkOp::prepare() +ReturnOpStatus SinkOp::prepare() { CHECK_IS_CANCELLED profile_info.anchor(); - auto op_status = prepareImpl(); + auto return_status = prepareImpl(); #ifndef NDEBUG - assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT}); + assertOperatorStatus(return_status.status, {OperatorStatus::NEED_INPUT}); #endif profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return op_status; + return return_status; } -OperatorStatus SinkOp::write(Block && block) +ReturnOpStatus SinkOp::write(Block && block) { CHECK_IS_CANCELLED profile_info.anchor(block); @@ -180,14 +184,14 @@ OperatorStatus SinkOp::write(Block && block) assertBlocksHaveEqualStructure(block, header, getName()); } #endif - auto op_status = writeImpl(std::move(block)); + auto return_status = writeImpl(std::move(block)); #ifndef NDEBUG - assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT}); + assertOperatorStatus(return_status.status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT}); #endif exec_context.triggerAutoSpill(); profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return op_status; + return return_status; } #undef CHECK_IS_CANCELLED diff --git a/dbms/src/Operators/Operator.h b/dbms/src/Operators/Operator.h index 57ee92263db..6b6c478e2cb 100644 --- a/dbms/src/Operators/Operator.h +++ b/dbms/src/Operators/Operator.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -37,6 +38,7 @@ enum class OperatorStatus CANCELLED, /// waiting status WAITING, + WAIT_FOR_NOTIFY, /// io status IO_IN, IO_OUT, @@ -49,6 +51,26 @@ enum class OperatorStatus class PipelineExecutorContext; +struct ReturnOpStatus +{ + ReturnOpStatus(OperatorStatus status_) // NOLINT(google-explicit-constructor) + : status(status_) + , future(nullptr) + { + assert(status != OperatorStatus::WAIT_FOR_NOTIFY); + } + + ReturnOpStatus(NotifyFuturePtr && furture_) // NOLINT(google-explicit-constructor) + : status(OperatorStatus::WAIT_FOR_NOTIFY) + , future(std::move(furture_)) + { + assert(future != nullptr); + } + + OperatorStatus status; + NotifyFuturePtr future{nullptr}; +}; + class Operator { public: @@ -60,10 +82,10 @@ class Operator virtual ~Operator() = default; // running status may return are NEED_INPUT and HAS_OUTPUT here. - OperatorStatus executeIO(); + ReturnOpStatus executeIO(); // running status may return are NEED_INPUT and HAS_OUTPUT here. - OperatorStatus await(); + ReturnOpStatus await(); // These two methods are used to set state, log and etc, and should not perform calculation logic. void operatePrefix(); @@ -93,9 +115,9 @@ class Operator virtual void operatePrefixImpl() {} virtual void operateSuffixImpl() {} - virtual OperatorStatus executeIOImpl() { throw Exception("Unsupport"); } + virtual ReturnOpStatus executeIOImpl() { throw Exception("Unsupport"); } - virtual OperatorStatus awaitImpl() { throw Exception("Unsupport"); } + virtual ReturnOpStatus awaitImpl() { throw Exception("Unsupport"); } protected: PipelineExecutorContext & exec_context; @@ -117,8 +139,8 @@ class SourceOp : public Operator // read will inplace the block when return status is HAS_OUTPUT; // Even after source has finished, source op still needs to return an empty block and HAS_OUTPUT, // because there are many operators that need an empty block as input, such as JoinProbe and WindowFunction. - OperatorStatus read(Block & block); - virtual OperatorStatus readImpl(Block & block) = 0; + ReturnOpStatus read(Block & block); + virtual ReturnOpStatus readImpl(Block & block) = 0; }; using SourceOpPtr = std::unique_ptr; using SourceOps = std::vector; @@ -131,14 +153,14 @@ class TransformOp : public Operator {} // running status may return are NEED_INPUT and HAS_OUTPUT here. // tryOutput will inplace the block when return status is HAS_OUPUT; do nothing to the block when NEED_INPUT or others. - OperatorStatus tryOutput(Block &); - virtual OperatorStatus tryOutputImpl(Block &) { return OperatorStatus::NEED_INPUT; } + ReturnOpStatus tryOutput(Block &); + virtual ReturnOpStatus tryOutputImpl(Block &) { return OperatorStatus::NEED_INPUT; } // running status may return are NEED_INPUT and HAS_OUTPUT here. // transform will inplace the block and if the return status is HAS_OUTPUT, this block can be used as input to subsequent operators. // Even if an empty block is input, transform will still return HAS_OUTPUT, // because there are many operators that need an empty block as input, such as JoinProbe and WindowFunction. - OperatorStatus transform(Block & block); - virtual OperatorStatus transformImpl(Block & block) = 0; + ReturnOpStatus transform(Block & block); + virtual ReturnOpStatus transformImpl(Block & block) = 0; virtual void transformHeaderImpl(Block & header_) = 0; void transformHeader(Block & header_) @@ -158,11 +180,11 @@ class SinkOp : public Operator SinkOp(PipelineExecutorContext & exec_context_, const String & req_id) : Operator(exec_context_, req_id) {} - OperatorStatus prepare(); - virtual OperatorStatus prepareImpl() { return OperatorStatus::NEED_INPUT; } + ReturnOpStatus prepare(); + virtual ReturnOpStatus prepareImpl() { return OperatorStatus::NEED_INPUT; } - OperatorStatus write(Block && block); - virtual OperatorStatus writeImpl(Block && block) = 0; + ReturnOpStatus write(Block && block); + virtual ReturnOpStatus writeImpl(Block && block) = 0; }; using SinkOpPtr = std::unique_ptr; } // namespace DB diff --git a/dbms/src/Operators/OperatorHelper.cpp b/dbms/src/Operators/OperatorHelper.cpp index 814e0ac6547..d266bd5124f 100644 --- a/dbms/src/Operators/OperatorHelper.cpp +++ b/dbms/src/Operators/OperatorHelper.cpp @@ -26,6 +26,7 @@ void assertOperatorStatus(OperatorStatus status, std::initializer_list res; @@ -87,9 +87,9 @@ class SharedQueueSourceOp : public SourceOp String getName() const override { return "SharedQueueSourceOp"; } - OperatorStatus readImpl(Block & block) override; + ReturnOpStatus readImpl(Block & block) override; - OperatorStatus awaitImpl() override; + ReturnOpStatus awaitImpl() override; private: std::optional res; diff --git a/dbms/src/Operators/UnorderedSourceOp.cpp b/dbms/src/Operators/UnorderedSourceOp.cpp index da1702ef468..eeb49c8ff39 100644 --- a/dbms/src/Operators/UnorderedSourceOp.cpp +++ b/dbms/src/Operators/UnorderedSourceOp.cpp @@ -20,18 +20,18 @@ namespace DB { -OperatorStatus UnorderedSourceOp::readImpl(Block & block) +ReturnOpStatus UnorderedSourceOp::readImpl(Block & block) { if unlikely (done) return OperatorStatus::HAS_OUTPUT; auto await_status = awaitImpl(); - if (await_status == OperatorStatus::HAS_OUTPUT) + if (await_status.status == OperatorStatus::HAS_OUTPUT) std::swap(block, t_block); return await_status; } -OperatorStatus UnorderedSourceOp::awaitImpl() +ReturnOpStatus UnorderedSourceOp::awaitImpl() { if unlikely (done) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/UnorderedSourceOp.h b/dbms/src/Operators/UnorderedSourceOp.h index 963ff9adefe..daa60fda577 100644 --- a/dbms/src/Operators/UnorderedSourceOp.h +++ b/dbms/src/Operators/UnorderedSourceOp.h @@ -76,8 +76,8 @@ class UnorderedSourceOp : public SourceOp protected: void operatePrefixImpl() override; - OperatorStatus readImpl(Block & block) override; - OperatorStatus awaitImpl() override; + ReturnOpStatus readImpl(Block & block) override; + ReturnOpStatus awaitImpl() override; private: DM::SegmentReadTaskPoolPtr task_pool; diff --git a/dbms/src/Operators/WindowTransformOp.cpp b/dbms/src/Operators/WindowTransformOp.cpp index e65df50b0f0..b091802ef7e 100644 --- a/dbms/src/Operators/WindowTransformOp.cpp +++ b/dbms/src/Operators/WindowTransformOp.cpp @@ -37,7 +37,7 @@ void WindowTransformOp::operateSuffixImpl() action->cleanUp(); } -OperatorStatus WindowTransformOp::transformImpl(Block & block) +ReturnOpStatus WindowTransformOp::transformImpl(Block & block) { assert(action); assert(!action->input_is_finished); @@ -57,7 +57,7 @@ OperatorStatus WindowTransformOp::transformImpl(Block & block) } } -OperatorStatus WindowTransformOp::tryOutputImpl(Block & block) +ReturnOpStatus WindowTransformOp::tryOutputImpl(Block & block) { assert(action); block = action->tryGetOutputBlock(); diff --git a/dbms/src/Operators/WindowTransformOp.h b/dbms/src/Operators/WindowTransformOp.h index 34f452bf547..906f24a980d 100644 --- a/dbms/src/Operators/WindowTransformOp.h +++ b/dbms/src/Operators/WindowTransformOp.h @@ -32,8 +32,8 @@ class WindowTransformOp : public TransformOp protected: void operateSuffixImpl() override; - OperatorStatus transformImpl(Block & block) override; - OperatorStatus tryOutputImpl(Block & block) override; + ReturnOpStatus transformImpl(Block & block) override; + ReturnOpStatus tryOutputImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/tests/gtest_concat_source.cpp b/dbms/src/Operators/tests/gtest_concat_source.cpp index c6090ee5bd4..a4f1a3dccb9 100644 --- a/dbms/src/Operators/tests/gtest_concat_source.cpp +++ b/dbms/src/Operators/tests/gtest_concat_source.cpp @@ -37,7 +37,7 @@ class MockSourceOp : public SourceOp String getName() const override { return "MockSourceOp"; } protected: - OperatorStatus readImpl(Block & block) override + ReturnOpStatus readImpl(Block & block) override { std::swap(block, output); return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h index ac2ad29f024..f0b039e8f68 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h @@ -13,6 +13,7 @@ // limitations under the License. #pragma once +#include #include #include @@ -20,8 +21,6 @@ #include #include -#include - namespace DB::DM { template diff --git a/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.cpp b/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.cpp index 17e467295c0..0901125a35a 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.cpp @@ -48,7 +48,7 @@ void RNSegmentSourceOp::operatePrefixImpl() workers->startInBackground(); } -OperatorStatus RNSegmentSourceOp::startGettingNextReadyTask() +ReturnOpStatus RNSegmentSourceOp::startGettingNextReadyTask() { // Start timing the time of get next ready task. wait_stop_watch.start(); @@ -56,7 +56,7 @@ OperatorStatus RNSegmentSourceOp::startGettingNextReadyTask() return awaitImpl(); } -OperatorStatus RNSegmentSourceOp::readImpl(Block & block) +ReturnOpStatus RNSegmentSourceOp::readImpl(Block & block) { if unlikely (done) { @@ -75,7 +75,7 @@ OperatorStatus RNSegmentSourceOp::readImpl(Block & block) return current_seg_task ? OperatorStatus::IO_IN : startGettingNextReadyTask(); } -OperatorStatus RNSegmentSourceOp::awaitImpl() +ReturnOpStatus RNSegmentSourceOp::awaitImpl() { if unlikely (done || t_block.has_value()) { @@ -117,7 +117,7 @@ OperatorStatus RNSegmentSourceOp::awaitImpl() } } -OperatorStatus RNSegmentSourceOp::executeIOImpl() +ReturnOpStatus RNSegmentSourceOp::executeIOImpl() { if unlikely (done || t_block.has_value()) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.h b/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.h index d3a4174655d..4e5d5cb86b4 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.h +++ b/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.h @@ -55,14 +55,14 @@ class RNSegmentSourceOp : public SourceOp void operatePrefixImpl() override; - OperatorStatus readImpl(Block & block) override; + ReturnOpStatus readImpl(Block & block) override; - OperatorStatus awaitImpl() override; + ReturnOpStatus awaitImpl() override; - OperatorStatus executeIOImpl() override; + ReturnOpStatus executeIOImpl() override; private: - OperatorStatus startGettingNextReadyTask(); + ReturnOpStatus startGettingNextReadyTask(); private: const RNWorkersPtr workers; diff --git a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h index 1f200cab82e..b45322e3224 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h @@ -167,10 +167,7 @@ class SegmentReadTaskPool : private boost::noncopyable std::once_flag & addToSchedulerFlag() { return add_to_scheduler; } - void registerPipeTask(TaskPtr && task) - { - q.registerPipeTask(std::move(task)); - } + void registerPipeTask(TaskPtr && task) { q.registerPipeTask(std::move(task)); } public: const uint64_t pool_id; From 11b364fb97a2d8dd9c45cc6d1422d1831b4c478f Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 16:57:56 +0800 Subject: [PATCH 03/19] for profile --- dbms/src/Common/TiFlashMetrics.h | 4 +++- .../Pipeline/Schedule/Reactor/WaitReactor.cpp | 1 + .../Schedule/Tasks/PipeConditionVariable.h | 3 +++ .../Pipeline/Schedule/Tasks/PipelineTask.h | 4 +++- dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 2 +- .../Pipeline/Schedule/Tasks/TaskProfileInfo.h | 11 ++++++++-- dbms/src/Operators/Operator.h | 6 ++++-- dbms/src/Operators/UnorderedSourceOp.cpp | 10 ++++----- dbms/src/Operators/UnorderedSourceOp.h | 21 ++++++++++++++++++- 9 files changed, 48 insertions(+), 14 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 36ae631b072..13f7725256a 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -722,6 +722,7 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva "pipeline scheduler", \ Gauge, \ F(type_waiting_tasks_count, {"type", "waiting_tasks_count"}), \ + F(type_wait_for_notify_tasks_count, {"type", "wait_for_notify_tasks_count"}), \ F(type_cpu_pending_tasks_count, {"type", "cpu_pending_tasks_count"}), \ F(type_cpu_executing_tasks_count, {"type", "cpu_executing_tasks_count"}), \ F(type_io_pending_tasks_count, {"type", "io_pending_tasks_count"}), \ @@ -735,7 +736,8 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_io_execute, {{"type", "io_execute"}}, ExpBuckets{0.005, 2, 20}), \ F(type_cpu_queue, {{"type", "cpu_queue"}}, ExpBuckets{0.005, 2, 20}), \ F(type_io_queue, {{"type", "io_queue"}}, ExpBuckets{0.005, 2, 20}), \ - F(type_await, {{"type", "await"}}, ExpBuckets{0.005, 2, 20})) \ + F(type_await, {{"type", "await"}}, ExpBuckets{0.005, 2, 20}), \ + F(type_wait_for_notify, {{"type", "wait_for_notify"}}, ExpBuckets{0.005, 2, 20})) \ M(tiflash_pipeline_task_execute_max_time_seconds_per_round, \ "Bucketed histogram of pipeline task execute max time per round in seconds", \ Histogram, /* these command usually cost several hundred milliseconds to several seconds, increase the start bucket to 5ms */ \ diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp index dde678d37ef..1d6e5230ee8 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp @@ -30,6 +30,7 @@ WaitReactor::WaitReactor(TaskScheduler & scheduler_) : scheduler(scheduler_) { GET_METRIC(tiflash_pipeline_scheduler, type_waiting_tasks_count).Set(0); + GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count).Set(0); thread = std::thread(&WaitReactor::loop, this); } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index 143a627dbc9..77dc55ef97f 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -25,6 +25,7 @@ class PipeConditionVariable public: void registerTask(TaskPtr && task) { + GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count).Increment(); assert(task->getStatus() == ExecTaskStatus::WAIT_FOR_NOTIFY); tasks.push_back(std::move(task)); } @@ -36,6 +37,8 @@ class PipeConditionVariable auto task = std::move(tasks.back()); tasks.pop_back(); task->notify(); + task->profile_info.elapsedWaitForNotifyTime(); + GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count).Decrement(); TaskScheduler::instance->submitToCPUTaskThreadPool(std::move(task)); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h index fcc2ff804ff..7fb20cebe61 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h @@ -43,7 +43,9 @@ class PipelineTask void doFinalizeImpl() override { - runFinalize(profile_info.getCPUPendingTimeNs() + profile_info.getIOPendingTimeNs() + getScheduleDuration()); + runFinalize( + profile_info.getCPUPendingTimeNs() + profile_info.getIOPendingTimeNs() + + profile_info.getWaitForNotifyTimeNs() + getScheduleDuration()); } }; } // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 04bace9add7..7a4ca5d4528 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -37,7 +37,7 @@ struct ReturnStatus assert(status != ExecTaskStatus::WAIT_FOR_NOTIFY); } - ReturnStatus(NotifyFuturePtr && furture_) // NOLINT(google-explicit-constructor) + ReturnStatus(NotifyFuturePtr furture_) // NOLINT(google-explicit-constructor) : status(ExecTaskStatus::WAIT_FOR_NOTIFY) , future(std::move(furture_)) { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/TaskProfileInfo.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/TaskProfileInfo.h index b431e42a0e5..27cf25f9b1a 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/TaskProfileInfo.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/TaskProfileInfo.h @@ -31,16 +31,18 @@ class ProfileInfo ALWAYS_INLINE UInt64 getIOExecuteTimeNs() const { return io_execute_time_ns; } ALWAYS_INLINE UInt64 getIOPendingTimeNs() const { return io_pending_time_ns; } ALWAYS_INLINE UInt64 getAwaitTimeNs() const { return await_time_ns; } + ALWAYS_INLINE UInt64 getWaitForNotifyTimeNs() const { return wait_for_notify_time_ns; } ALWAYS_INLINE String toJson() const { return fmt::format( - R"({{"cpu_execute_time_ns":{},"cpu_pending_time_ns":{},"io_execute_time_ns":{},"io_pending_time_ns":{},"await_time_ns":{}}})", + R"({{"cpu_execute_time_ns":{},"cpu_pending_time_ns":{},"io_execute_time_ns":{},"io_pending_time_ns":{},"await_time_ns":{},"wait_for_notify_time_ns":{}}})", cpu_execute_time_ns, cpu_pending_time_ns, io_execute_time_ns, io_pending_time_ns, - await_time_ns); + await_time_ns, + wait_for_notify_time_ns); } protected: @@ -49,6 +51,7 @@ class ProfileInfo UnitType io_execute_time_ns = 0; UnitType io_pending_time_ns = 0; UnitType await_time_ns = 0; + UnitType wait_for_notify_time_ns = 0; }; class TaskProfileInfo : public ProfileInfo @@ -78,6 +81,8 @@ class TaskProfileInfo : public ProfileInfo ALWAYS_INLINE void elapsedAwaitTime() { await_time_ns += elapsedFromPrev(); } + ALWAYS_INLINE void elapsedWaitForNotifyTime() { wait_for_notify_time_ns += elapsedFromPrev(); } + ALWAYS_INLINE void reportMetrics() const { #ifdef __APPLE__ @@ -111,6 +116,7 @@ class TaskProfileInfo : public ProfileInfo REPORT_DURATION_METRICS(type_io_execute, io_execute_time_ns); REPORT_DURATION_METRICS(type_io_queue, io_pending_time_ns); REPORT_DURATION_METRICS(type_await, await_time_ns); + REPORT_DURATION_METRICS(type_wait_for_notify, wait_for_notify_time_ns); REPORT_ROUND_METRICS(type_cpu, cpu_execute_max_time_ns_per_round); REPORT_ROUND_METRICS(type_io, io_execute_max_time_ns_per_round); @@ -136,6 +142,7 @@ class QueryProfileInfo : public ProfileInfo io_execute_time_ns += task_profile_info.getIOExecuteTimeNs(); io_pending_time_ns += task_profile_info.getIOPendingTimeNs(); await_time_ns += task_profile_info.getAwaitTimeNs(); + wait_for_notify_time_ns += task_profile_info.getWaitForNotifyTimeNs(); } }; } // namespace DB diff --git a/dbms/src/Operators/Operator.h b/dbms/src/Operators/Operator.h index 6b6c478e2cb..a3f4de2e7ef 100644 --- a/dbms/src/Operators/Operator.h +++ b/dbms/src/Operators/Operator.h @@ -16,7 +16,6 @@ #include #include -#include #include #include @@ -51,6 +50,9 @@ enum class OperatorStatus class PipelineExecutorContext; +struct NotifyFuture; +using NotifyFuturePtr = std::shared_ptr; + struct ReturnOpStatus { ReturnOpStatus(OperatorStatus status_) // NOLINT(google-explicit-constructor) @@ -60,7 +62,7 @@ struct ReturnOpStatus assert(status != OperatorStatus::WAIT_FOR_NOTIFY); } - ReturnOpStatus(NotifyFuturePtr && furture_) // NOLINT(google-explicit-constructor) + ReturnOpStatus(NotifyFuturePtr furture_) // NOLINT(google-explicit-constructor) : status(OperatorStatus::WAIT_FOR_NOTIFY) , future(std::move(furture_)) { diff --git a/dbms/src/Operators/UnorderedSourceOp.cpp b/dbms/src/Operators/UnorderedSourceOp.cpp index eeb49c8ff39..348865b82a1 100644 --- a/dbms/src/Operators/UnorderedSourceOp.cpp +++ b/dbms/src/Operators/UnorderedSourceOp.cpp @@ -25,23 +25,21 @@ ReturnOpStatus UnorderedSourceOp::readImpl(Block & block) if unlikely (done) return OperatorStatus::HAS_OUTPUT; - auto await_status = awaitImpl(); + auto await_status = doFetchBlock(); if (await_status.status == OperatorStatus::HAS_OUTPUT) std::swap(block, t_block); return await_status; } -ReturnOpStatus UnorderedSourceOp::awaitImpl() +ReturnOpStatus UnorderedSourceOp::doFetchBlock() { - if unlikely (done) - return OperatorStatus::HAS_OUTPUT; - if unlikely (t_block) + if (t_block) return OperatorStatus::HAS_OUTPUT; while (true) { if (!task_pool->tryPopBlock(t_block)) - return OperatorStatus::WAITING; + return {notify_future}; if (t_block) { if unlikely (t_block.rows() == 0) diff --git a/dbms/src/Operators/UnorderedSourceOp.h b/dbms/src/Operators/UnorderedSourceOp.h index daa60fda577..490fa4a9de7 100644 --- a/dbms/src/Operators/UnorderedSourceOp.h +++ b/dbms/src/Operators/UnorderedSourceOp.h @@ -17,12 +17,27 @@ #include #include #include +#include #include #include #include namespace DB { +class ScanNotifyFuture : public NotifyFuture +{ +public: + explicit ScanNotifyFuture(const DM::SegmentReadTaskPoolPtr & task_pool_) + : task_pool(task_pool_) + { + assert(task_pool); + } + + void registerTask(TaskPtr && task) override { task_pool->registerPipeTask(std::move(task)); } + +private: + DM::SegmentReadTaskPoolPtr task_pool; +}; /// Read blocks asyncly from Storage Layer by using read thread, /// The result can not guarantee the keep_order property @@ -40,6 +55,7 @@ class UnorderedSourceOp : public SourceOp : SourceOp(exec_context_, req_id) , task_pool(task_pool_) , ref_no(0) + , notify_future(std::make_shared(task_pool_)) , waiting_rf_list(runtime_filter_list_) , max_wait_time_ms(max_wait_time_ms_) { @@ -77,11 +93,14 @@ class UnorderedSourceOp : public SourceOp void operatePrefixImpl() override; ReturnOpStatus readImpl(Block & block) override; - ReturnOpStatus awaitImpl() override; + +private: + ReturnOpStatus doFetchBlock(); private: DM::SegmentReadTaskPoolPtr task_pool; int64_t ref_no; + NotifyFuturePtr notify_future; // runtime filter RuntimeFilteList waiting_rf_list; From ff3fea4edfd73732e9076e9d081f994184daf3e1 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 17:30:09 +0800 Subject: [PATCH 04/19] fix build --- .../Pipeline/Schedule/Tasks/NotifyFuture.h | 4 +- dbms/src/Operators/UnorderedSourceOp.cpp | 40 +++++++++++++++++++ dbms/src/Operators/UnorderedSourceOp.h | 28 +------------ 3 files changed, 43 insertions(+), 29 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h index 408150a1566..aba86395f12 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h @@ -20,9 +20,9 @@ namespace DB { struct NotifyFuture { + NotifyFuture() = default; virtual ~NotifyFuture() = default; - virtual void registerTask(TaskPtr && task); + virtual void registerTask(TaskPtr && task) = 0; }; using NotifyFuturePtr = std::shared_ptr; - } // namespace DB diff --git a/dbms/src/Operators/UnorderedSourceOp.cpp b/dbms/src/Operators/UnorderedSourceOp.cpp index 348865b82a1..ca67cef44d7 100644 --- a/dbms/src/Operators/UnorderedSourceOp.cpp +++ b/dbms/src/Operators/UnorderedSourceOp.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include @@ -20,6 +21,45 @@ namespace DB { +namespace +{ +class ScanNotifyFuture : public NotifyFuture +{ +public: + explicit ScanNotifyFuture(const DM::SegmentReadTaskPoolPtr & task_pool_) + : task_pool(task_pool_) + { + assert(task_pool); + } + + ~ScanNotifyFuture() override = default; + + void registerTask(TaskPtr && task) override { task_pool->registerPipeTask(std::move(task)); } + +private: + DM::SegmentReadTaskPoolPtr task_pool; +}; +}; // namespace + +UnorderedSourceOp::UnorderedSourceOp( + PipelineExecutorContext & exec_context_, + const DM::SegmentReadTaskPoolPtr & task_pool_, + const DM::ColumnDefines & columns_to_read_, + int extra_table_id_index_, + const String & req_id, + const RuntimeFilteList & runtime_filter_list_, + int max_wait_time_ms_) + : SourceOp(exec_context_, req_id) + , task_pool(task_pool_) + , ref_no(0) + , notify_future(std::make_shared(task_pool_)) + , waiting_rf_list(runtime_filter_list_) + , max_wait_time_ms(max_wait_time_ms_) +{ + setHeader(AddExtraTableIDColumnTransformAction::buildHeader(columns_to_read_, extra_table_id_index_)); + ref_no = task_pool->increaseUnorderedInputStreamRefCount(); +} + ReturnOpStatus UnorderedSourceOp::readImpl(Block & block) { if unlikely (done) diff --git a/dbms/src/Operators/UnorderedSourceOp.h b/dbms/src/Operators/UnorderedSourceOp.h index 490fa4a9de7..1fadb39b84d 100644 --- a/dbms/src/Operators/UnorderedSourceOp.h +++ b/dbms/src/Operators/UnorderedSourceOp.h @@ -17,28 +17,12 @@ #include #include #include -#include #include #include #include namespace DB { -class ScanNotifyFuture : public NotifyFuture -{ -public: - explicit ScanNotifyFuture(const DM::SegmentReadTaskPoolPtr & task_pool_) - : task_pool(task_pool_) - { - assert(task_pool); - } - - void registerTask(TaskPtr && task) override { task_pool->registerPipeTask(std::move(task)); } - -private: - DM::SegmentReadTaskPoolPtr task_pool; -}; - /// Read blocks asyncly from Storage Layer by using read thread, /// The result can not guarantee the keep_order property class UnorderedSourceOp : public SourceOp @@ -51,17 +35,7 @@ class UnorderedSourceOp : public SourceOp int extra_table_id_index_, const String & req_id, const RuntimeFilteList & runtime_filter_list_ = std::vector{}, - int max_wait_time_ms_ = 0) - : SourceOp(exec_context_, req_id) - , task_pool(task_pool_) - , ref_no(0) - , notify_future(std::make_shared(task_pool_)) - , waiting_rf_list(runtime_filter_list_) - , max_wait_time_ms(max_wait_time_ms_) - { - setHeader(AddExtraTableIDColumnTransformAction::buildHeader(columns_to_read_, extra_table_id_index_)); - ref_no = task_pool->increaseUnorderedInputStreamRefCount(); - } + int max_wait_time_ms_ = 0); ~UnorderedSourceOp() override { From 9511939d6378a0df2dabcd4c0907099ac5015e6c Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 17:42:29 +0800 Subject: [PATCH 05/19] fix notify --- .../Pipeline/Schedule/Tasks/PipeConditionVariable.h | 9 ++++++++- dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h | 7 +++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index 77dc55ef97f..0bf625b7e51 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -30,7 +30,7 @@ class PipeConditionVariable tasks.push_back(std::move(task)); } - void notifyOne() + bool notifyOne() { if (!tasks.empty()) { @@ -40,7 +40,14 @@ class PipeConditionVariable task->profile_info.elapsedWaitForNotifyTime(); GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count).Decrement(); TaskScheduler::instance->submitToCPUTaskThreadPool(std::move(task)); + return true; } + return false; + } + + void notifyAll() + { + while (notifyOne()) {} } private: diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h index f0b039e8f68..6ccddde41e2 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h @@ -68,7 +68,9 @@ class WorkQueue { std::lock_guard lock(mu); pipe_cv.registerTask(std::move(task)); - if (!queue.empty() || done) + if (done) + pipe_cv.notifyAll(); + if (!queue.empty()) pipe_cv.notifyOne(); } @@ -92,7 +94,7 @@ class WorkQueue } if (done) { - pipe_cv.notifyOne(); + pipe_cv.notifyAll(); return false; } queue.push(std::forward(item)); @@ -194,6 +196,7 @@ class WorkQueue std::lock_guard lock(mu); assert(!done); done = true; + pipe_cv.notifyAll(); } reader_cv.notify_all(); writer_cv.notify_all(); From c7084d50ef2fb833be10626d436376d88f058896 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 18:25:12 +0800 Subject: [PATCH 06/19] minor refactor --- .../Schedule/Tasks/PipeConditionVariable.h | 61 ++++++++++++++----- .../Schedule/ThreadPool/TaskThreadPool.cpp | 1 - .../DeltaMerge/ReadThread/WorkQueue.h | 16 +++-- 3 files changed, 59 insertions(+), 19 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index 0bf625b7e51..ffaca0ed808 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -17,40 +17,73 @@ #include #include +#include + namespace DB { // Must have lock to use this class class PipeConditionVariable { public: - void registerTask(TaskPtr && task) + inline void registerTask(TaskPtr && task) { - GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count).Increment(); + assert(task); assert(task->getStatus() == ExecTaskStatus::WAIT_FOR_NOTIFY); tasks.push_back(std::move(task)); + +#ifdef __APPLE__ + auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#else + thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#endif + metrics.Increment(); } - bool notifyOne() + inline void notifyOne() { if (!tasks.empty()) { - auto task = std::move(tasks.back()); - tasks.pop_back(); - task->notify(); - task->profile_info.elapsedWaitForNotifyTime(); - GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count).Decrement(); - TaskScheduler::instance->submitToCPUTaskThreadPool(std::move(task)); - return true; + auto task = std::move(tasks.front()); + tasks.pop_front(); + notifyTaskDirectly(std::move(task)); + +#ifdef __APPLE__ + auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#else + thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#endif + metrics.Decrement(); } - return false; } - void notifyAll() + inline void notifyAll() + { + size_t tasks_cnt = tasks.size(); + while (!tasks.empty()) + { + auto task = std::move(tasks.front()); + tasks.pop_front(); + notifyTaskDirectly(std::move(task)); + } + +#ifdef __APPLE__ + auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#else + thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#endif + metrics.Decrement(tasks_cnt); + } + + static inline void notifyTaskDirectly(TaskPtr && task) { - while (notifyOne()) {} + assert(task); + task->notify(); + task->profile_info.elapsedWaitForNotifyTime(); + assert(TaskScheduler::instance); + TaskScheduler::instance->submitToCPUTaskThreadPool(std::move(task)); } private: - std::vector tasks; + std::deque tasks; }; } // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp index 42620c623cd..b999e3b79ab 100644 --- a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp @@ -129,7 +129,6 @@ void TaskThreadPool::handleTask(TaskPtr & task) assert(return_status_after_exec.future); return_status_after_exec.future->registerTask(std::move(task)); break; - ; case FINISH_STATUS: task->finalize(); task->endTraceMemory(); diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h index 6ccddde41e2..76761c48975 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h @@ -67,11 +67,19 @@ class WorkQueue void registerPipeTask(TaskPtr && task) { std::lock_guard lock(mu); - pipe_cv.registerTask(std::move(task)); - if (done) + if (unlikely(done)) + { + PipeConditionVariable::notifyTaskDirectly(std::move(task)); pipe_cv.notifyAll(); - if (!queue.empty()) - pipe_cv.notifyOne(); + } + else if (!queue.empty()) + { + PipeConditionVariable::notifyTaskDirectly(std::move(task)); + } + else + { + pipe_cv.registerTask(std::move(task)); + } } /** From 2512249a81e27800b68f63df09955cd94589965d Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 19:04:33 +0800 Subject: [PATCH 07/19] __thread --- .../Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp | 4 ++-- .../Pipeline/Schedule/Tasks/PipeConditionVariable.h | 12 ++++++------ dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp | 13 +++++++------ dbms/src/Operators/UnorderedSourceOp.cpp | 2 +- 4 files changed, 16 insertions(+), 15 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp index 1d6e5230ee8..275c7dce231 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp @@ -167,8 +167,8 @@ void WaitReactor::react(WaitingTasks & local_waiting_tasks) ++task_it; } -#ifdef __APPLE__ - auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_waiting_tasks_count); +#if __APPLE__ && __clang__ + __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_waiting_tasks_count); #else thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_waiting_tasks_count); #endif diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index ffaca0ed808..51cbae5ddb9 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -31,8 +31,8 @@ class PipeConditionVariable assert(task->getStatus() == ExecTaskStatus::WAIT_FOR_NOTIFY); tasks.push_back(std::move(task)); -#ifdef __APPLE__ - auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#if __APPLE__ && __clang__ + __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #else thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #endif @@ -47,8 +47,8 @@ class PipeConditionVariable tasks.pop_front(); notifyTaskDirectly(std::move(task)); -#ifdef __APPLE__ - auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#if __APPLE__ && __clang__ + __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #else thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #endif @@ -66,8 +66,8 @@ class PipeConditionVariable notifyTaskDirectly(std::move(task)); } -#ifdef __APPLE__ - auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); +#if __APPLE__ && __clang__ + __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #else thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #endif diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index c9f4e130874..005e99fcca1 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -36,12 +36,13 @@ namespace // TODO supports more detailed status transfer metrics, such as from waiting to running. ALWAYS_INLINE void addToStatusMetrics(ExecTaskStatus to) { -#ifdef __APPLE__ -#define M(expect_status, metric_name) \ - case (expect_status): \ - { \ - GET_METRIC(tiflash_pipeline_task_change_to_status, metric_name).Increment(); \ - break; \ +#if __APPLE__ && __clang__ +#define M(expect_status, metric_name) \ + case (expect_status): \ + { \ + __thread auto & metrics_##metric_name = GET_METRIC(tiflash_pipeline_task_change_to_status, metric_name); \ + (metrics_##metric_name).Increment(); \ + break; \ } #else #define M(expect_status, metric_name) \ diff --git a/dbms/src/Operators/UnorderedSourceOp.cpp b/dbms/src/Operators/UnorderedSourceOp.cpp index ca67cef44d7..550320fe0ab 100644 --- a/dbms/src/Operators/UnorderedSourceOp.cpp +++ b/dbms/src/Operators/UnorderedSourceOp.cpp @@ -79,7 +79,7 @@ ReturnOpStatus UnorderedSourceOp::doFetchBlock() while (true) { if (!task_pool->tryPopBlock(t_block)) - return {notify_future}; + return notify_future; if (t_block) { if unlikely (t_block.rows() == 0) From 9914a5822ccd5c57fae69e6377003964b3bac910 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 19:28:04 +0800 Subject: [PATCH 08/19] tmp save --- dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp | 14 ++--- dbms/src/Flash/Pipeline/Exec/PipelineExec.h | 14 ++--- .../Exec/tests/gtest_simple_operator.cpp | 2 +- .../Schedule/Events/tests/gtest_event.cpp | 14 ++--- .../Pipeline/Schedule/Reactor/WaitReactor.cpp | 9 ++- .../Reactor/tests/gtest_wait_task_list.cpp | 2 +- .../TaskQueues/tests/bench_task_queue.cpp | 2 +- .../TaskQueues/tests/gtest_io_priority.cpp | 2 +- .../Schedule/TaskQueues/tests/gtest_mlfq.cpp | 2 +- .../tests/gtest_resource_control_queue.cpp | 2 +- .../Tasks/AggregateFinalSpillTask.cpp | 2 +- .../Schedule/Tasks/AggregateFinalSpillTask.h | 2 +- .../Pipeline/Schedule/Tasks/ExecTaskStatus.h | 41 ------------- .../Pipeline/Schedule/Tasks/IOEventTask.h | 4 +- .../Schedule/Tasks/LoadBucketTask.cpp | 2 +- .../Pipeline/Schedule/Tasks/LoadBucketTask.h | 2 +- .../Pipeline/Schedule/Tasks/NotifyFuture.cpp | 39 +++++++++++++ .../Pipeline/Schedule/Tasks/NotifyFuture.h | 10 ++++ .../Pipeline/Schedule/Tasks/PipelineTask.h | 6 +- .../Schedule/Tasks/PipelineTaskBase.h | 6 +- .../Pipeline/Schedule/Tasks/RFWaitTask.h | 4 +- .../Schedule/Tasks/SimplePipelineTask.h | 6 +- .../Schedule/Tasks/StreamRestoreTask.cpp | 8 +-- .../Schedule/Tasks/StreamRestoreTask.h | 6 +- .../Flash/Pipeline/Schedule/Tasks/Task.cpp | 6 +- dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 57 +++++++++---------- .../Schedule/ThreadPool/TaskThreadPool.cpp | 15 +++-- .../Schedule/ThreadPool/TaskThreadPoolImpl.h | 4 +- .../Schedule/tests/gtest_task_scheduler.cpp | 22 +++---- .../AddExtraTableIDColumnTransformOp.cpp | 2 +- .../AddExtraTableIDColumnTransformOp.h | 2 +- dbms/src/Operators/AggregateBuildSinkOp.cpp | 6 +- dbms/src/Operators/AggregateBuildSinkOp.h | 6 +- .../Operators/AggregateConvergentSourceOp.cpp | 2 +- .../Operators/AggregateConvergentSourceOp.h | 2 +- .../Operators/AggregateRestoreSourceOp.cpp | 4 +- dbms/src/Operators/AggregateRestoreSourceOp.h | 4 +- .../Operators/BlockInputStreamSourceOp.cpp | 2 +- dbms/src/Operators/BlockInputStreamSourceOp.h | 2 +- dbms/src/Operators/ConcatSourceOp.h | 14 ++--- .../Operators/CoprocessorReaderSourceOp.cpp | 4 +- .../src/Operators/CoprocessorReaderSourceOp.h | 4 +- .../src/Operators/DMSegmentThreadSourceOp.cpp | 4 +- dbms/src/Operators/DMSegmentThreadSourceOp.h | 4 +- .../Operators/ExchangeReceiverSourceOp.cpp | 4 +- dbms/src/Operators/ExchangeReceiverSourceOp.h | 4 +- dbms/src/Operators/ExchangeSenderSinkOp.cpp | 6 +- dbms/src/Operators/ExchangeSenderSinkOp.h | 6 +- dbms/src/Operators/Expand2TransformOp.cpp | 4 +- dbms/src/Operators/Expand2TransformOp.h | 4 +- dbms/src/Operators/ExpressionTransformOp.cpp | 2 +- dbms/src/Operators/ExpressionTransformOp.h | 2 +- dbms/src/Operators/FilterTransformOp.cpp | 2 +- dbms/src/Operators/FilterTransformOp.h | 2 +- .../GeneratedColumnPlaceHolderTransformOp.cpp | 2 +- .../GeneratedColumnPlaceHolderTransformOp.h | 2 +- dbms/src/Operators/GetResultSinkOp.cpp | 6 +- dbms/src/Operators/GetResultSinkOp.h | 6 +- dbms/src/Operators/HashJoinBuildSink.cpp | 6 +- dbms/src/Operators/HashJoinBuildSink.h | 6 +- .../Operators/HashJoinProbeTransformOp.cpp | 16 +++--- dbms/src/Operators/HashJoinProbeTransformOp.h | 10 ++-- dbms/src/Operators/HashProbeTransformExec.cpp | 6 +- dbms/src/Operators/HashProbeTransformExec.h | 6 +- .../Operators/IOBlockInputStreamSourceOp.h | 4 +- dbms/src/Operators/LimitTransformOp.cpp | 2 +- dbms/src/Operators/LimitTransformOp.h | 2 +- .../src/Operators/LocalAggregateTransform.cpp | 10 ++-- dbms/src/Operators/LocalAggregateTransform.h | 10 ++-- dbms/src/Operators/MergeSortTransformOp.cpp | 14 ++--- dbms/src/Operators/MergeSortTransformOp.h | 14 ++--- dbms/src/Operators/NullSourceOp.h | 2 +- dbms/src/Operators/Operator.cpp | 14 ++--- dbms/src/Operators/Operator.h | 51 +++++------------ dbms/src/Operators/PartialSortTransformOp.cpp | 2 +- dbms/src/Operators/PartialSortTransformOp.h | 2 +- dbms/src/Operators/SharedQueue.cpp | 10 ++-- dbms/src/Operators/SharedQueue.h | 10 ++-- dbms/src/Operators/UnorderedSourceOp.cpp | 16 ++++-- dbms/src/Operators/UnorderedSourceOp.h | 7 ++- dbms/src/Operators/WindowTransformOp.cpp | 4 +- dbms/src/Operators/WindowTransformOp.h | 4 +- .../Operators/tests/gtest_concat_source.cpp | 2 +- .../DeltaMerge/Remote/RNSegmentSourceOp.cpp | 8 +-- .../DeltaMerge/Remote/RNSegmentSourceOp.h | 8 +-- 85 files changed, 327 insertions(+), 338 deletions(-) delete mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/ExecTaskStatus.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp index 9b13b7a9227..2ea3fe516f2 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp @@ -89,7 +89,7 @@ void PipelineExec::executeSuffix() source_op->operateSuffix(); } -ReturnOpStatus PipelineExec::execute() +OperatorStatus PipelineExec::execute() { auto op_status = executeImpl(); #ifndef NDEBUG @@ -105,7 +105,7 @@ ReturnOpStatus PipelineExec::execute() * │ block * write◄────transform◄─── ... ◄───transform◄────────────┘ */ -ReturnOpStatus PipelineExec::executeImpl() +OperatorStatus PipelineExec::executeImpl() { Block block; size_t start_transform_op_index = 0; @@ -127,7 +127,7 @@ ReturnOpStatus PipelineExec::executeImpl() } // try fetch block from transform_ops and source_op. -ReturnOpStatus PipelineExec::fetchBlock(Block & block, size_t & start_transform_op_index) +OperatorStatus PipelineExec::fetchBlock(Block & block, size_t & start_transform_op_index) { auto op_status = sink_op->prepare(); HANDLE_OP_STATUS(sink_op, op_status, OperatorStatus::NEED_INPUT); @@ -144,7 +144,7 @@ ReturnOpStatus PipelineExec::fetchBlock(Block & block, size_t & start_transform_ HANDLE_LAST_OP_STATUS(source_op, op_status); } -ReturnOpStatus PipelineExec::executeIO() +OperatorStatus PipelineExec::executeIO() { auto op_status = executeIOImpl(); #ifndef NDEBUG @@ -156,7 +156,7 @@ ReturnOpStatus PipelineExec::executeIO() #endif return op_status; } -ReturnOpStatus PipelineExec::executeIOImpl() +OperatorStatus PipelineExec::executeIOImpl() { assert(io_op); auto op_status = io_op->executeIO(); @@ -167,7 +167,7 @@ ReturnOpStatus PipelineExec::executeIOImpl() return op_status; } -ReturnOpStatus PipelineExec::await() +OperatorStatus PipelineExec::await() { auto op_status = awaitImpl(); #ifndef NDEBUG @@ -179,7 +179,7 @@ ReturnOpStatus PipelineExec::await() #endif return op_status; } -ReturnOpStatus PipelineExec::awaitImpl() +OperatorStatus PipelineExec::awaitImpl() { assert(awaitable); auto op_status = awaitable->await(); diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.h b/dbms/src/Flash/Pipeline/Exec/PipelineExec.h index 645de8e6925..275c019cc97 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExec.h +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.h @@ -31,22 +31,22 @@ class PipelineExec : private boost::noncopyable void executePrefix(); void executeSuffix(); - ReturnOpStatus execute(); + OperatorStatus execute(); - ReturnOpStatus executeIO(); + OperatorStatus executeIO(); - ReturnOpStatus await(); + OperatorStatus await(); void finalizeProfileInfo(UInt64 extra_time); private: - inline ReturnOpStatus executeImpl(); + inline OperatorStatus executeImpl(); - inline ReturnOpStatus executeIOImpl(); + inline OperatorStatus executeIOImpl(); - inline ReturnOpStatus awaitImpl(); + inline OperatorStatus awaitImpl(); - inline ReturnOpStatus fetchBlock(Block & block, size_t & start_transform_op_index); + inline OperatorStatus fetchBlock(Block & block, size_t & start_transform_op_index); ALWAYS_INLINE void fillAwaitable(Operator * op) { diff --git a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp index d026ddab77f..398c7f72077 100644 --- a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp +++ b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp @@ -39,7 +39,7 @@ class SimpleGetResultSinkOp : public SinkOp String getName() const override { return "SimpleGetResultSinkOp"; } protected: - ReturnOpStatus writeImpl(Block && block) override + OperatorStatus writeImpl(Block && block) override { if (!block) return OperatorStatus::FINISHED; diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp index a3cf9f88b97..dda48c8b632 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp @@ -34,7 +34,7 @@ class BaseTask : public EventTask {} protected: - ReturnStatus executeImpl() override + ExecTaskStatus executeImpl() override { --counter; return ExecTaskStatus::FINISHED; @@ -75,7 +75,7 @@ class RunTask : public EventTask {} protected: - ReturnStatus executeImpl() override + ExecTaskStatus executeImpl() override { while ((--loop_count) > 0) return ExecTaskStatus::RUNNING; @@ -116,7 +116,7 @@ class DeadLoopTask : public EventTask {} protected: - ReturnStatus executeImpl() override + ExecTaskStatus executeImpl() override { std::this_thread::sleep_for(std::chrono::milliseconds(1)); return ExecTaskStatus::RUNNING; @@ -185,7 +185,7 @@ class ThrowExceptionTask : public EventTask {} protected: - ReturnStatus executeImpl() override { throw Exception("throw exception in doExecuteImpl"); } + ExecTaskStatus executeImpl() override { throw Exception("throw exception in doExecuteImpl"); } }; class ThrowExceptionEvent : public Event @@ -291,7 +291,7 @@ class TestPorfileTask : public EventTask protected: // executeImpl min_time ==> executeIOImpl min_time ==> awaitImpl min_time. - ReturnStatus executeImpl() override + ExecTaskStatus executeImpl() override { if (cpu_execute_time < min_time) { @@ -302,7 +302,7 @@ class TestPorfileTask : public EventTask return ExecTaskStatus::IO_IN; } - ReturnStatus executeIOImpl() override + ExecTaskStatus executeIOImpl() override { if (io_execute_time < min_time) { @@ -313,7 +313,7 @@ class TestPorfileTask : public EventTask return ExecTaskStatus::WAITING; } - ReturnStatus awaitImpl() override + ExecTaskStatus awaitImpl() override { if unlikely (!wait_stopwatch) wait_stopwatch.emplace(CLOCK_MONOTONIC_COARSE); diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp index 275c7dce231..8a3a2910a07 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/WaitReactor.cpp @@ -38,8 +38,8 @@ bool WaitReactor::awaitAndCollectReadyTask(WaitingTask && task) { assert(task.first); auto * task_ptr = task.second; - auto return_status = task_ptr->await(); - switch (return_status.status) + auto status = task_ptr->await(); + switch (status) { case ExecTaskStatus::WAITING: return false; @@ -53,8 +53,7 @@ bool WaitReactor::awaitAndCollectReadyTask(WaitingTask && task) io_tasks.push_back(std::move(task.first)); return true; case ExecTaskStatus::WAIT_FOR_NOTIFY: - assert(return_status.future); - return_status.future->registerTask(std::move(task.first)); + registerTaskToFuture(std::move(task.first)); return true; case FINISH_STATUS: task_ptr->profile_info.elapsedAwaitTime(); @@ -64,7 +63,7 @@ bool WaitReactor::awaitAndCollectReadyTask(WaitingTask && task) task.first.reset(); return true; default: - UNEXPECTED_STATUS(logger, return_status.status); + UNEXPECTED_STATUS(logger, status); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Reactor/tests/gtest_wait_task_list.cpp b/dbms/src/Flash/Pipeline/Schedule/Reactor/tests/gtest_wait_task_list.cpp index 62ccd9d1044..ad51a729214 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Reactor/tests/gtest_wait_task_list.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Reactor/tests/gtest_wait_task_list.cpp @@ -30,7 +30,7 @@ class PlainTask : public Task : Task(exec_context_) {} - ReturnStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } + ExecTaskStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } }; } // namespace diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp index bf6b62dc066..636cd06f588 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp @@ -33,7 +33,7 @@ namespace tests : Task(exec_context) \ {} \ \ - ReturnStatus executeImpl() override \ + ExecTaskStatus executeImpl() override \ { \ if (task_exec_cur_count <= task_exec_total_count) \ { \ diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp index 2bac2eec02a..398f5a4dff3 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_io_priority.cpp @@ -30,7 +30,7 @@ class MockIOTask : public Task : Task(exec_context_, "", is_io_in ? ExecTaskStatus::IO_IN : ExecTaskStatus::IO_OUT) {} - ReturnStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } + ExecTaskStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } }; } // namespace diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp index d5ce456a4a6..6d004f956a4 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_mlfq.cpp @@ -30,7 +30,7 @@ class PlainTask : public Task : Task(exec_context_) {} - ReturnStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } + ExecTaskStatus executeImpl() noexcept override { return ExecTaskStatus::FINISHED; } }; } // namespace diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp index 9d242eb4d40..5003117c91c 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_resource_control_queue.cpp @@ -39,7 +39,7 @@ class SimpleTask : public Task ~SimpleTask() override = default; - ReturnStatus executeImpl() noexcept override + ExecTaskStatus executeImpl() noexcept override { if (exec_time_counter < total_exec_times) { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.cpp index 2d7b918aedf..d3367abd39a 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.cpp @@ -35,7 +35,7 @@ void AggregateFinalSpillTask::doFinalizeImpl() agg_context.reset(); } -ReturnStatus AggregateFinalSpillTask::executeIOImpl() +ExecTaskStatus AggregateFinalSpillTask::executeIOImpl() { agg_context->spillData(index); return ExecTaskStatus::FINISHED; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.h index 846f5ca1c31..1fa9af3ddf3 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/AggregateFinalSpillTask.h @@ -32,7 +32,7 @@ class AggregateFinalSpillTask : public OutputIOEventTask size_t index_); protected: - ReturnStatus executeIOImpl() override; + ExecTaskStatus executeIOImpl() override; void doFinalizeImpl() override; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/ExecTaskStatus.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/ExecTaskStatus.h deleted file mode 100644 index 14bf4864275..00000000000 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/ExecTaskStatus.h +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -namespace DB -{ -/** - * CANCELLED/ERROR/FINISHED - * ▲ - * │ - * ┌───────────────────────────────────────────────┐ - * │ ┌──►RUNNING◄──┐ │ - * INIT───►│ │ │ │ - * │ ▼ ▼ │ - * │ WAIT_FOR_NOTIFY/WATITING◄────────►IO_IN/OUT │ - * └───────────────────────────────────────────────┘ - */ -enum class ExecTaskStatus -{ - WAIT_FOR_NOTIFY, - WAITING, - RUNNING, - IO_IN, - IO_OUT, - FINISHED, - ERROR, - CANCELLED, -}; -} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/IOEventTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/IOEventTask.h index 6e206e9bcfa..7fa435bef22 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/IOEventTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/IOEventTask.h @@ -27,7 +27,7 @@ class IOEventTask : public EventTask {} private: - ReturnStatus executeImpl() final + ExecTaskStatus executeImpl() final { if constexpr (is_input) return ExecTaskStatus::IO_IN; @@ -35,7 +35,7 @@ class IOEventTask : public EventTask return ExecTaskStatus::IO_OUT; } - ReturnStatus awaitImpl() final + ExecTaskStatus awaitImpl() final { if constexpr (is_input) return ExecTaskStatus::IO_IN; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.cpp index 74e748c9a3c..7a17ed5f4e8 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.cpp @@ -17,7 +17,7 @@ namespace DB { -ReturnStatus LoadBucketTask::executeIOImpl() +ExecTaskStatus LoadBucketTask::executeIOImpl() { input.load(); return ExecTaskStatus::FINISHED; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.h index 2d317eca8ce..f4d452f35ec 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/LoadBucketTask.h @@ -33,7 +33,7 @@ class LoadBucketTask : public InputIOEventTask {} private: - ReturnStatus executeIOImpl() override; + ExecTaskStatus executeIOImpl() override; private: SpilledBucketInput & input; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp new file mode 100644 index 00000000000..d12a03e2ca7 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp @@ -0,0 +1,39 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +namespace DB +{ +#if __APPLE__ && __clang__ +__thread NotifyFuturePtr current_notify_future = nullptr; +#else +thread_local NotifyFuturePtr current_notify_future = nullptr; +#endif + +void setNotifyFuture(NotifyFuturePtr new_future) +{ + assert(current_notify_future == nullptr); + current_notify_future = std::move(new_future); +} + +void registerTaskToFuture(TaskPtr && task) +{ + assert(current_notify_future != nullptr); + current_notify_future->registerTask(std::move(task)); + current_notify_future.reset(); +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h index aba86395f12..3cf47966bff 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h @@ -25,4 +25,14 @@ struct NotifyFuture virtual void registerTask(TaskPtr && task) = 0; }; using NotifyFuturePtr = std::shared_ptr; + +#if __APPLE__ && __clang__ +extern __thread NotifyFuturePtr current_notify_future; +#else +extern thread_local NotifyFuturePtr current_notify_future; +#endif + +void setNotifyFuture(NotifyFuturePtr new_future); +void registerTaskToFuture(TaskPtr && task); + } // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h index 7fb20cebe61..318e56179d4 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h @@ -35,11 +35,11 @@ class PipelineTask {} protected: - ReturnStatus executeImpl() override { return runExecute(); } + ExecTaskStatus executeImpl() override { return runExecute(); } - ReturnStatus executeIOImpl() override { return runExecuteIO(); } + ExecTaskStatus executeIOImpl() override { return runExecuteIO(); } - ReturnStatus awaitImpl() override { return runAwait(); } + ExecTaskStatus awaitImpl() override { return runAwait(); } void doFinalizeImpl() override { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h index 95fc498d918..9af1f47957f 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h @@ -70,7 +70,7 @@ class PipelineTaskBase } protected: - ReturnStatus runExecute() + ExecTaskStatus runExecute() { assert(pipeline_exec); auto op_status = (pipeline_exec)->execute(); @@ -86,7 +86,7 @@ class PipelineTaskBase } } - ReturnStatus runExecuteIO() + ExecTaskStatus runExecuteIO() { assert(pipeline_exec); auto op_status = (pipeline_exec)->executeIO(); @@ -105,7 +105,7 @@ class PipelineTaskBase } } - ReturnStatus runAwait() + ExecTaskStatus runAwait() { assert(pipeline_exec); auto op_status = (pipeline_exec)->await(); diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/RFWaitTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/RFWaitTask.h index b2f23653f98..245d45f8e30 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/RFWaitTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/RFWaitTask.h @@ -75,9 +75,9 @@ class RFWaitTask : public Task } private: - ReturnStatus executeImpl() override { return ExecTaskStatus::WAITING; } + ExecTaskStatus executeImpl() override { return ExecTaskStatus::WAITING; } - ReturnStatus awaitImpl() override + ExecTaskStatus awaitImpl() override { filterAndMoveReadyRfs(waiting_rf_list, ready_rf_list); if (waiting_rf_list.empty() || stopwatch.elapsed() >= max_wait_time_ns) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/SimplePipelineTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/SimplePipelineTask.h index 6552fb712bf..c713bfe95f5 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/SimplePipelineTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/SimplePipelineTask.h @@ -34,11 +34,11 @@ class SimplePipelineTask {} protected: - ReturnStatus executeImpl() override { return runExecute(); } + ExecTaskStatus executeImpl() override { return runExecute(); } - ReturnStatus executeIOImpl() override { return runExecuteIO(); } + ExecTaskStatus executeIOImpl() override { return runExecuteIO(); } - ReturnStatus awaitImpl() override { return runAwait(); } + ExecTaskStatus awaitImpl() override { return runAwait(); } void finalizeImpl() override { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.cpp index 5710371eed7..d075b22cd16 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.cpp @@ -20,7 +20,7 @@ namespace DB { namespace { -ALWAYS_INLINE ReturnStatus tryPushBlock(const ResultQueuePtr & result_queue, Block & block) +ALWAYS_INLINE ExecTaskStatus tryPushBlock(const ResultQueuePtr & result_queue, Block & block) { assert(block); auto ret = result_queue->tryPush(std::move(block)); @@ -53,12 +53,12 @@ StreamRestoreTask::StreamRestoreTask( assert(result_queue); } -ReturnStatus StreamRestoreTask::executeImpl() +ExecTaskStatus StreamRestoreTask::executeImpl() { return is_done ? ExecTaskStatus::FINISHED : ExecTaskStatus::IO_IN; } -ReturnStatus StreamRestoreTask::awaitImpl() +ExecTaskStatus StreamRestoreTask::awaitImpl() { if (unlikely(is_done)) return ExecTaskStatus::FINISHED; @@ -68,7 +68,7 @@ ReturnStatus StreamRestoreTask::awaitImpl() return tryPushBlock(result_queue, block); } -ReturnStatus StreamRestoreTask::executeIOImpl() +ExecTaskStatus StreamRestoreTask::executeIOImpl() { if (!block) { diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.h index f3054f330b6..2d6aa7f08d7 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/StreamRestoreTask.h @@ -33,11 +33,11 @@ class StreamRestoreTask : public Task const ResultQueuePtr & result_queue_); protected: - ReturnStatus executeImpl() override; + ExecTaskStatus executeImpl() override; - ReturnStatus awaitImpl() override; + ExecTaskStatus awaitImpl() override; - ReturnStatus executeIOImpl() override; + ExecTaskStatus executeIOImpl() override; void finalizeImpl() override; diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index 005e99fcca1..c989b06ba20 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -128,21 +128,21 @@ Task::~Task() return task_status; \ } -ReturnStatus Task::execute() +ExecTaskStatus Task::execute() { assert(mem_tracker_ptr == current_memory_tracker); assert(task_status == ExecTaskStatus::RUNNING); EXECUTE(executeImpl); } -ReturnStatus Task::executeIO() +ExecTaskStatus Task::executeIO() { assert(mem_tracker_ptr == current_memory_tracker); assert(task_status == ExecTaskStatus::IO_IN || task_status == ExecTaskStatus::IO_OUT); EXECUTE(executeIOImpl); } -ReturnStatus Task::await() +ExecTaskStatus Task::await() { // Because await only performs polling checks and does not involve computing/memory tracker memory allocation, // await will not invoke MemoryTracker, so current_memory_tracker must be nullptr here. diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 7a4ca5d4528..364da124f05 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -17,37 +17,36 @@ #include #include #include -#include #include #include namespace DB { -class PipelineExecutorContext; - -struct NotifyFuture; -using NotifyFuturePtr = std::shared_ptr; - -struct ReturnStatus +/** + * CANCELLED/ERROR/FINISHED + * ▲ + * │ + * ┌───────────────────────────────────────────────┐ + * │ ┌──►RUNNING◄──┐ │ + * INIT───►│ │ │ │ + * │ ▼ ▼ │ + * │ WAIT_FOR_NOTIFY/WATITING◄────────►IO_IN/OUT │ + * └───────────────────────────────────────────────┘ + */ +enum class ExecTaskStatus { - ReturnStatus(ExecTaskStatus status_) // NOLINT(google-explicit-constructor) - : status(status_) - , future(nullptr) - { - assert(status != ExecTaskStatus::WAIT_FOR_NOTIFY); - } - - ReturnStatus(NotifyFuturePtr furture_) // NOLINT(google-explicit-constructor) - : status(ExecTaskStatus::WAIT_FOR_NOTIFY) - , future(std::move(furture_)) - { - assert(future != nullptr); - } - - ExecTaskStatus status; - NotifyFuturePtr future{nullptr}; + WAIT_FOR_NOTIFY, + WAITING, + RUNNING, + IO_IN, + IO_OUT, + FINISHED, + ERROR, + CANCELLED, }; +class PipelineExecutorContext; + class Task { public: @@ -63,11 +62,11 @@ class Task ExecTaskStatus getStatus() const { return task_status; } - ReturnStatus execute(); + ExecTaskStatus execute(); - ReturnStatus executeIO(); + ExecTaskStatus executeIO(); - ReturnStatus await(); + ExecTaskStatus await(); void notify(); @@ -99,10 +98,10 @@ class Task LoggerPtr log; protected: - virtual ReturnStatus executeImpl() = 0; - virtual ReturnStatus executeIOImpl() { return ExecTaskStatus::RUNNING; } + virtual ExecTaskStatus executeImpl() = 0; + virtual ExecTaskStatus executeIOImpl() { return ExecTaskStatus::RUNNING; } // Avoid allocating memory in `await` if possible. - virtual ReturnStatus awaitImpl() { return ExecTaskStatus::RUNNING; } + virtual ExecTaskStatus awaitImpl() { return ExecTaskStatus::RUNNING; } // Used to release held resources, just like `Event::finishImpl`. virtual void finalizeImpl() {} diff --git a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp index b999e3b79ab..600af3f80d4 100644 --- a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp @@ -96,21 +96,21 @@ void TaskThreadPool::handleTask(TaskPtr & task) metrics.incExecutingTask(); metrics.elapsedPendingTime(task); - ExecTaskStatus status_before_exec = task->getStatus(); - ReturnStatus return_status_after_exec = status_before_exec; + auto status_before_exec = task->getStatus(); + auto status_after_exec = status_before_exec; UInt64 total_time_spent = 0; while (true) { - return_status_after_exec = Impl::exec(task); + status_after_exec = Impl::exec(task); total_time_spent += task->profile_info.elapsedFromPrev(); // The executing task should yield if it takes more than `YIELD_MAX_TIME_SPENT_NS`. - if (!Impl::isTargetStatus(return_status_after_exec.status) || total_time_spent >= YIELD_MAX_TIME_SPENT_NS) + if (!Impl::isTargetStatus(status_after_exec) || total_time_spent >= YIELD_MAX_TIME_SPENT_NS) break; } task_queue->updateStatistics(task, status_before_exec, total_time_spent); metrics.addExecuteTime(task, total_time_spent); metrics.decExecutingTask(); - switch (return_status_after_exec.status) + switch (status_after_exec) { case ExecTaskStatus::RUNNING: task->endTraceMemory(); @@ -126,8 +126,7 @@ void TaskThreadPool::handleTask(TaskPtr & task) scheduler.submitToWaitReactor(std::move(task)); break; case ExecTaskStatus::WAIT_FOR_NOTIFY: - assert(return_status_after_exec.future); - return_status_after_exec.future->registerTask(std::move(task)); + registerTaskToFuture(std::move(task)); break; case FINISH_STATUS: task->finalize(); @@ -135,7 +134,7 @@ void TaskThreadPool::handleTask(TaskPtr & task) task.reset(); break; default: - UNEXPECTED_STATUS(task->log, return_status_after_exec.status); + UNEXPECTED_STATUS(task->log, status_after_exec); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h index 13923a6e8c9..5c7e1984148 100644 --- a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h +++ b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPoolImpl.h @@ -29,7 +29,7 @@ struct CPUImpl static bool isTargetStatus(ExecTaskStatus status) { return status == ExecTaskStatus::RUNNING; } - static ReturnStatus exec(TaskPtr & task) { return task->execute(); } + static ExecTaskStatus exec(TaskPtr & task) { return task->execute(); } static TaskQueuePtr newTaskQueue(TaskQueueType type); }; @@ -45,7 +45,7 @@ struct IOImpl return status == ExecTaskStatus::IO_IN || status == ExecTaskStatus::IO_OUT; } - static ReturnStatus exec(TaskPtr & task) { return task->executeIO(); } + static ExecTaskStatus exec(TaskPtr & task) { return task->executeIO(); } static TaskQueuePtr newTaskQueue(TaskQueueType type); }; diff --git a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp index e915a55effc..c69a84b3a30 100644 --- a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp @@ -32,7 +32,7 @@ class SimpleTask : public Task {} protected: - ReturnStatus executeImpl() noexcept override + ExecTaskStatus executeImpl() noexcept override { while ((--loop_count) > 0) return ExecTaskStatus::RUNNING; @@ -51,7 +51,7 @@ class SimpleWaitingTask : public Task {} protected: - ReturnStatus executeImpl() noexcept override + ExecTaskStatus executeImpl() noexcept override { if (loop_count > 0) { @@ -66,7 +66,7 @@ class SimpleWaitingTask : public Task return ExecTaskStatus::FINISHED; } - ReturnStatus awaitImpl() noexcept override + ExecTaskStatus awaitImpl() noexcept override { if (loop_count > 0) { @@ -93,7 +93,7 @@ class SimpleBlockedTask : public Task {} protected: - ReturnStatus executeImpl() override + ExecTaskStatus executeImpl() override { if (loop_count > 0) { @@ -108,7 +108,7 @@ class SimpleBlockedTask : public Task return ExecTaskStatus::FINISHED; } - ReturnStatus executeIOImpl() override + ExecTaskStatus executeIOImpl() override { if (loop_count > 0) { @@ -140,19 +140,19 @@ class MemoryTraceTask : public Task static constexpr Int64 MEMORY_TRACER_SUBMIT_THRESHOLD = 1024 * 1024; // 1 MiB protected: - ReturnStatus executeImpl() noexcept override + ExecTaskStatus executeImpl() noexcept override { CurrentMemoryTracker::alloc(MEMORY_TRACER_SUBMIT_THRESHOLD - 10); return ExecTaskStatus::IO_IN; } - ReturnStatus executeIOImpl() override + ExecTaskStatus executeIOImpl() override { CurrentMemoryTracker::alloc(MEMORY_TRACER_SUBMIT_THRESHOLD + 10); return ExecTaskStatus::WAITING; } - ReturnStatus awaitImpl() override + ExecTaskStatus awaitImpl() override { // await wouldn't call MemoryTracker. return ExecTaskStatus::FINISHED; @@ -167,11 +167,11 @@ class DeadLoopTask : public Task {} protected: - ReturnStatus executeImpl() override { return ExecTaskStatus::WAITING; } + ExecTaskStatus executeImpl() override { return ExecTaskStatus::WAITING; } - ReturnStatus awaitImpl() override { return ExecTaskStatus::IO_IN; } + ExecTaskStatus awaitImpl() override { return ExecTaskStatus::IO_IN; } - ReturnStatus executeIOImpl() override { return ExecTaskStatus::RUNNING; } + ExecTaskStatus executeIOImpl() override { return ExecTaskStatus::RUNNING; } }; } // namespace diff --git a/dbms/src/Operators/AddExtraTableIDColumnTransformOp.cpp b/dbms/src/Operators/AddExtraTableIDColumnTransformOp.cpp index 1374f9c60ff..67ab63da3bc 100644 --- a/dbms/src/Operators/AddExtraTableIDColumnTransformOp.cpp +++ b/dbms/src/Operators/AddExtraTableIDColumnTransformOp.cpp @@ -17,7 +17,7 @@ namespace DB { -ReturnOpStatus AddExtraTableIDColumnTransformOp::transformImpl(Block & block) +OperatorStatus AddExtraTableIDColumnTransformOp::transformImpl(Block & block) { if (!action.transform(block, physical_table_id)) block = {}; diff --git a/dbms/src/Operators/AddExtraTableIDColumnTransformOp.h b/dbms/src/Operators/AddExtraTableIDColumnTransformOp.h index e790186b729..57905629bff 100644 --- a/dbms/src/Operators/AddExtraTableIDColumnTransformOp.h +++ b/dbms/src/Operators/AddExtraTableIDColumnTransformOp.h @@ -42,7 +42,7 @@ class AddExtraTableIDColumnTransformOp : public TransformOp IOProfileInfoPtr getIOProfileInfo() const override { return IOProfileInfo::createForLocal(profile_info_ptr); } protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/AggregateBuildSinkOp.cpp b/dbms/src/Operators/AggregateBuildSinkOp.cpp index af027867742..f3bc51d21b1 100644 --- a/dbms/src/Operators/AggregateBuildSinkOp.cpp +++ b/dbms/src/Operators/AggregateBuildSinkOp.cpp @@ -17,7 +17,7 @@ namespace DB { -ReturnOpStatus AggregateBuildSinkOp::prepareImpl() +OperatorStatus AggregateBuildSinkOp::prepareImpl() { while (agg_context->hasLocalDataToBuild(index)) { @@ -28,7 +28,7 @@ ReturnOpStatus AggregateBuildSinkOp::prepareImpl() return agg_context->isTaskMarkedForSpill(index) ? OperatorStatus::IO_OUT : OperatorStatus::NEED_INPUT; } -ReturnOpStatus AggregateBuildSinkOp::writeImpl(Block && block) +OperatorStatus AggregateBuildSinkOp::writeImpl(Block && block) { if (unlikely(!block)) { @@ -44,7 +44,7 @@ ReturnOpStatus AggregateBuildSinkOp::writeImpl(Block && block) return agg_context->needSpill(index) ? OperatorStatus::IO_OUT : OperatorStatus::NEED_INPUT; } -ReturnOpStatus AggregateBuildSinkOp::executeIOImpl() +OperatorStatus AggregateBuildSinkOp::executeIOImpl() { agg_context->spillData(index); return is_final_spill ? OperatorStatus::FINISHED : OperatorStatus::NEED_INPUT; diff --git a/dbms/src/Operators/AggregateBuildSinkOp.h b/dbms/src/Operators/AggregateBuildSinkOp.h index 0d08106e362..5bed80c5f68 100644 --- a/dbms/src/Operators/AggregateBuildSinkOp.h +++ b/dbms/src/Operators/AggregateBuildSinkOp.h @@ -39,11 +39,11 @@ class AggregateBuildSinkOp : public SinkOp protected: void operateSuffixImpl() override; - ReturnOpStatus prepareImpl() override; + OperatorStatus prepareImpl() override; - ReturnOpStatus writeImpl(Block && block) override; + OperatorStatus writeImpl(Block && block) override; - ReturnOpStatus executeIOImpl() override; + OperatorStatus executeIOImpl() override; private: size_t index{}; diff --git a/dbms/src/Operators/AggregateConvergentSourceOp.cpp b/dbms/src/Operators/AggregateConvergentSourceOp.cpp index 3e88efe9dcc..053e28dbb4d 100644 --- a/dbms/src/Operators/AggregateConvergentSourceOp.cpp +++ b/dbms/src/Operators/AggregateConvergentSourceOp.cpp @@ -29,7 +29,7 @@ AggregateConvergentSourceOp::AggregateConvergentSourceOp( setHeader(agg_context->getHeader()); } -ReturnOpStatus AggregateConvergentSourceOp::readImpl(Block & block) +OperatorStatus AggregateConvergentSourceOp::readImpl(Block & block) { block = agg_context->readForConvergent(index); total_rows += block.rows(); diff --git a/dbms/src/Operators/AggregateConvergentSourceOp.h b/dbms/src/Operators/AggregateConvergentSourceOp.h index fbf888a7983..9d7e6032a36 100644 --- a/dbms/src/Operators/AggregateConvergentSourceOp.h +++ b/dbms/src/Operators/AggregateConvergentSourceOp.h @@ -35,7 +35,7 @@ class AggregateConvergentSourceOp : public SourceOp protected: void operateSuffixImpl() override; - ReturnOpStatus readImpl(Block & block) override; + OperatorStatus readImpl(Block & block) override; private: AggregateContextPtr agg_context; diff --git a/dbms/src/Operators/AggregateRestoreSourceOp.cpp b/dbms/src/Operators/AggregateRestoreSourceOp.cpp index e02ccabd062..a1e29c0ffb4 100644 --- a/dbms/src/Operators/AggregateRestoreSourceOp.cpp +++ b/dbms/src/Operators/AggregateRestoreSourceOp.cpp @@ -30,12 +30,12 @@ AggregateRestoreSourceOp::AggregateRestoreSourceOp( setHeader(agg_context->getHeader()); } -ReturnOpStatus AggregateRestoreSourceOp::readImpl(Block & block) +OperatorStatus AggregateRestoreSourceOp::readImpl(Block & block) { return restorer->tryPop(block) ? OperatorStatus::HAS_OUTPUT : OperatorStatus::WAITING; } -ReturnOpStatus AggregateRestoreSourceOp::awaitImpl() +OperatorStatus AggregateRestoreSourceOp::awaitImpl() { return restorer->tryLoadBucketData() == SharedLoadResult::RETRY ? OperatorStatus::WAITING : OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/AggregateRestoreSourceOp.h b/dbms/src/Operators/AggregateRestoreSourceOp.h index e05f6b4ad94..3fc474f0961 100644 --- a/dbms/src/Operators/AggregateRestoreSourceOp.h +++ b/dbms/src/Operators/AggregateRestoreSourceOp.h @@ -34,9 +34,9 @@ class AggregateRestoreSourceOp : public SourceOp String getName() const override { return "AggregateRestoreSourceOp"; } protected: - ReturnOpStatus readImpl(Block & block) override; + OperatorStatus readImpl(Block & block) override; - ReturnOpStatus awaitImpl() override; + OperatorStatus awaitImpl() override; private: AggregateContextPtr agg_context; diff --git a/dbms/src/Operators/BlockInputStreamSourceOp.cpp b/dbms/src/Operators/BlockInputStreamSourceOp.cpp index ece8d9133fe..a47a12e4bb5 100644 --- a/dbms/src/Operators/BlockInputStreamSourceOp.cpp +++ b/dbms/src/Operators/BlockInputStreamSourceOp.cpp @@ -38,7 +38,7 @@ void BlockInputStreamSourceOp::operateSuffixImpl() impl->readSuffix(); } -ReturnOpStatus BlockInputStreamSourceOp::readImpl(Block & block) +OperatorStatus BlockInputStreamSourceOp::readImpl(Block & block) { block = impl->read(); return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/BlockInputStreamSourceOp.h b/dbms/src/Operators/BlockInputStreamSourceOp.h index 437ee8b7d12..22136c558c0 100644 --- a/dbms/src/Operators/BlockInputStreamSourceOp.h +++ b/dbms/src/Operators/BlockInputStreamSourceOp.h @@ -38,7 +38,7 @@ class BlockInputStreamSourceOp : public SourceOp void operatePrefixImpl() override; void operateSuffixImpl() override; - ReturnOpStatus readImpl(Block & block) override; + OperatorStatus readImpl(Block & block) override; private: BlockInputStreamPtr impl; diff --git a/dbms/src/Operators/ConcatSourceOp.h b/dbms/src/Operators/ConcatSourceOp.h index e96ece88750..29f47be70e8 100644 --- a/dbms/src/Operators/ConcatSourceOp.h +++ b/dbms/src/Operators/ConcatSourceOp.h @@ -34,7 +34,7 @@ class SetBlockSinkOp : public SinkOp String getName() const override { return "SetBlockSinkOp"; } protected: - ReturnOpStatus writeImpl(Block && block) override + OperatorStatus writeImpl(Block && block) override { if unlikely (!block) return OperatorStatus::FINISHED; @@ -89,7 +89,7 @@ class ConcatSourceOp : public SourceOp exec_pool.clear(); } - ReturnOpStatus readImpl(Block & block) override + OperatorStatus readImpl(Block & block) override { if unlikely (done) return OperatorStatus::HAS_OUTPUT; @@ -104,7 +104,7 @@ class ConcatSourceOp : public SourceOp { assert(cur_exec); auto status = cur_exec->execute(); - switch (status.status) + switch (status) { case OperatorStatus::NEED_INPUT: assert(res); @@ -125,25 +125,25 @@ class ConcatSourceOp : public SourceOp } } - ReturnOpStatus executeIOImpl() override + OperatorStatus executeIOImpl() override { if unlikely (done || res) return OperatorStatus::HAS_OUTPUT; assert(cur_exec); auto status = cur_exec->executeIO(); - assert(status.status != OperatorStatus::FINISHED); + assert(status != OperatorStatus::FINISHED); return status; } - ReturnOpStatus awaitImpl() override + OperatorStatus awaitImpl() override { if unlikely (done || res) return OperatorStatus::HAS_OUTPUT; assert(cur_exec); auto status = cur_exec->await(); - assert(status.status != OperatorStatus::FINISHED); + assert(status != OperatorStatus::FINISHED); return status; } diff --git a/dbms/src/Operators/CoprocessorReaderSourceOp.cpp b/dbms/src/Operators/CoprocessorReaderSourceOp.cpp index 0ce661271c9..e223227c4a3 100644 --- a/dbms/src/Operators/CoprocessorReaderSourceOp.cpp +++ b/dbms/src/Operators/CoprocessorReaderSourceOp.cpp @@ -54,7 +54,7 @@ Block CoprocessorReaderSourceOp::popFromBlockQueue() return block; } -ReturnOpStatus CoprocessorReaderSourceOp::readImpl(Block & block) +OperatorStatus CoprocessorReaderSourceOp::readImpl(Block & block) { if (!block_queue.empty()) { @@ -112,7 +112,7 @@ ReturnOpStatus CoprocessorReaderSourceOp::readImpl(Block & block) return await_status; } } -ReturnOpStatus CoprocessorReaderSourceOp::awaitImpl() +OperatorStatus CoprocessorReaderSourceOp::awaitImpl() { if unlikely (!block_queue.empty()) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/CoprocessorReaderSourceOp.h b/dbms/src/Operators/CoprocessorReaderSourceOp.h index 520b9f84fca..8acb6f255ed 100644 --- a/dbms/src/Operators/CoprocessorReaderSourceOp.h +++ b/dbms/src/Operators/CoprocessorReaderSourceOp.h @@ -36,8 +36,8 @@ class CoprocessorReaderSourceOp : public SourceOp void operatePrefixImpl() override; void operateSuffixImpl() override; - ReturnOpStatus readImpl(Block & block) override; - ReturnOpStatus awaitImpl() override; + OperatorStatus readImpl(Block & block) override; + OperatorStatus awaitImpl() override; IOProfileInfoPtr getIOProfileInfo() const override { return io_profile_info; } diff --git a/dbms/src/Operators/DMSegmentThreadSourceOp.cpp b/dbms/src/Operators/DMSegmentThreadSourceOp.cpp index 3248044f026..05d7a9d07aa 100644 --- a/dbms/src/Operators/DMSegmentThreadSourceOp.cpp +++ b/dbms/src/Operators/DMSegmentThreadSourceOp.cpp @@ -57,7 +57,7 @@ void DMSegmentThreadSourceOp::operateSuffixImpl() LOG_DEBUG(log, "Finish read {} rows from storage", total_rows); } -ReturnOpStatus DMSegmentThreadSourceOp::readImpl(Block & block) +OperatorStatus DMSegmentThreadSourceOp::readImpl(Block & block) { if (done) { @@ -74,7 +74,7 @@ ReturnOpStatus DMSegmentThreadSourceOp::readImpl(Block & block) return OperatorStatus::IO_IN; } -ReturnOpStatus DMSegmentThreadSourceOp::executeIOImpl() +OperatorStatus DMSegmentThreadSourceOp::executeIOImpl() { if (unlikely(done)) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/DMSegmentThreadSourceOp.h b/dbms/src/Operators/DMSegmentThreadSourceOp.h index 6548c3be62c..3e89c01fcce 100644 --- a/dbms/src/Operators/DMSegmentThreadSourceOp.h +++ b/dbms/src/Operators/DMSegmentThreadSourceOp.h @@ -49,9 +49,9 @@ class DMSegmentThreadSourceOp : public SourceOp protected: void operateSuffixImpl() override; - ReturnOpStatus readImpl(Block & block) override; + OperatorStatus readImpl(Block & block) override; - ReturnOpStatus executeIOImpl() override; + OperatorStatus executeIOImpl() override; private: DM::DMContextPtr dm_context; diff --git a/dbms/src/Operators/ExchangeReceiverSourceOp.cpp b/dbms/src/Operators/ExchangeReceiverSourceOp.cpp index 9e2fd624708..3df7886c980 100644 --- a/dbms/src/Operators/ExchangeReceiverSourceOp.cpp +++ b/dbms/src/Operators/ExchangeReceiverSourceOp.cpp @@ -29,7 +29,7 @@ Block ExchangeReceiverSourceOp::popFromBlockQueue() return block; } -ReturnOpStatus ExchangeReceiverSourceOp::readImpl(Block & block) +OperatorStatus ExchangeReceiverSourceOp::readImpl(Block & block) { if (!block_queue.empty()) { @@ -93,7 +93,7 @@ ReturnOpStatus ExchangeReceiverSourceOp::readImpl(Block & block) } } -ReturnOpStatus ExchangeReceiverSourceOp::awaitImpl() +OperatorStatus ExchangeReceiverSourceOp::awaitImpl() { if unlikely (!block_queue.empty()) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/ExchangeReceiverSourceOp.h b/dbms/src/Operators/ExchangeReceiverSourceOp.h index 002ecf26cc5..f7f1e86c99f 100644 --- a/dbms/src/Operators/ExchangeReceiverSourceOp.h +++ b/dbms/src/Operators/ExchangeReceiverSourceOp.h @@ -46,9 +46,9 @@ class ExchangeReceiverSourceOp : public SourceOp protected: void operateSuffixImpl() override; - ReturnOpStatus readImpl(Block & block) override; + OperatorStatus readImpl(Block & block) override; - ReturnOpStatus awaitImpl() override; + OperatorStatus awaitImpl() override; private: Block popFromBlockQueue(); diff --git a/dbms/src/Operators/ExchangeSenderSinkOp.cpp b/dbms/src/Operators/ExchangeSenderSinkOp.cpp index c6d085a9ee8..664e22e3cc5 100644 --- a/dbms/src/Operators/ExchangeSenderSinkOp.cpp +++ b/dbms/src/Operators/ExchangeSenderSinkOp.cpp @@ -26,7 +26,7 @@ void ExchangeSenderSinkOp::operateSuffixImpl() LOG_DEBUG(log, "finish write with {} rows", total_rows); } -ReturnOpStatus ExchangeSenderSinkOp::writeImpl(Block && block) +OperatorStatus ExchangeSenderSinkOp::writeImpl(Block && block) { if (!block) { @@ -39,12 +39,12 @@ ReturnOpStatus ExchangeSenderSinkOp::writeImpl(Block && block) return OperatorStatus::NEED_INPUT; } -ReturnOpStatus ExchangeSenderSinkOp::prepareImpl() +OperatorStatus ExchangeSenderSinkOp::prepareImpl() { return writer->isWritable() ? OperatorStatus::NEED_INPUT : OperatorStatus::WAITING; } -ReturnOpStatus ExchangeSenderSinkOp::awaitImpl() +OperatorStatus ExchangeSenderSinkOp::awaitImpl() { return writer->isWritable() ? OperatorStatus::NEED_INPUT : OperatorStatus::WAITING; } diff --git a/dbms/src/Operators/ExchangeSenderSinkOp.h b/dbms/src/Operators/ExchangeSenderSinkOp.h index 37fe293275d..b4e4702d518 100644 --- a/dbms/src/Operators/ExchangeSenderSinkOp.h +++ b/dbms/src/Operators/ExchangeSenderSinkOp.h @@ -37,11 +37,11 @@ class ExchangeSenderSinkOp : public SinkOp void operatePrefixImpl() override; void operateSuffixImpl() override; - ReturnOpStatus writeImpl(Block && block) override; + OperatorStatus writeImpl(Block && block) override; - ReturnOpStatus prepareImpl() override; + OperatorStatus prepareImpl() override; - ReturnOpStatus awaitImpl() override; + OperatorStatus awaitImpl() override; private: std::unique_ptr writer; diff --git a/dbms/src/Operators/Expand2TransformOp.cpp b/dbms/src/Operators/Expand2TransformOp.cpp index 1f583db2cf3..7be61ebd032 100644 --- a/dbms/src/Operators/Expand2TransformOp.cpp +++ b/dbms/src/Operators/Expand2TransformOp.cpp @@ -16,7 +16,7 @@ namespace DB { -ReturnOpStatus Expand2TransformOp::transformImpl(Block & block) +OperatorStatus Expand2TransformOp::transformImpl(Block & block) { if (likely(block)) expand_transform_action.transform(block); @@ -24,7 +24,7 @@ ReturnOpStatus Expand2TransformOp::transformImpl(Block & block) return OperatorStatus::HAS_OUTPUT; } -ReturnOpStatus Expand2TransformOp::tryOutputImpl(Block & block) +OperatorStatus Expand2TransformOp::tryOutputImpl(Block & block) { if (expand_transform_action.tryOutput(block)) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/Expand2TransformOp.h b/dbms/src/Operators/Expand2TransformOp.h index c7564d009fb..fe42c8adc34 100644 --- a/dbms/src/Operators/Expand2TransformOp.h +++ b/dbms/src/Operators/Expand2TransformOp.h @@ -35,9 +35,9 @@ class Expand2TransformOp : public TransformOp String getName() const override { return "Expand2TransformOp"; } protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; - ReturnOpStatus tryOutputImpl(Block & block) override; + OperatorStatus tryOutputImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/ExpressionTransformOp.cpp b/dbms/src/Operators/ExpressionTransformOp.cpp index 159ba6e01f3..b973f755ff5 100644 --- a/dbms/src/Operators/ExpressionTransformOp.cpp +++ b/dbms/src/Operators/ExpressionTransformOp.cpp @@ -17,7 +17,7 @@ namespace DB { -ReturnOpStatus ExpressionTransformOp::transformImpl(Block & block) +OperatorStatus ExpressionTransformOp::transformImpl(Block & block) { if (likely(block)) expression->execute(block); diff --git a/dbms/src/Operators/ExpressionTransformOp.h b/dbms/src/Operators/ExpressionTransformOp.h index 700486ed5bf..fc791b15246 100644 --- a/dbms/src/Operators/ExpressionTransformOp.h +++ b/dbms/src/Operators/ExpressionTransformOp.h @@ -36,7 +36,7 @@ class ExpressionTransformOp : public TransformOp String getName() const override { return "ExpressionTransformOp"; } protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/FilterTransformOp.cpp b/dbms/src/Operators/FilterTransformOp.cpp index cba65a01d9f..bc6751faef3 100644 --- a/dbms/src/Operators/FilterTransformOp.cpp +++ b/dbms/src/Operators/FilterTransformOp.cpp @@ -16,7 +16,7 @@ namespace DB { -ReturnOpStatus FilterTransformOp::transformImpl(Block & block) +OperatorStatus FilterTransformOp::transformImpl(Block & block) { if (unlikely(filter_transform_action.alwaysFalse())) { diff --git a/dbms/src/Operators/FilterTransformOp.h b/dbms/src/Operators/FilterTransformOp.h index 8ed10d8af55..61c05b3c810 100644 --- a/dbms/src/Operators/FilterTransformOp.h +++ b/dbms/src/Operators/FilterTransformOp.h @@ -36,7 +36,7 @@ class FilterTransformOp : public TransformOp String getName() const override { return "FilterTransformOp"; } protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.cpp b/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.cpp index f52d75c2ec5..c155d15d8f0 100644 --- a/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.cpp +++ b/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.cpp @@ -20,7 +20,7 @@ String GeneratedColumnPlaceHolderTransformOp::getName() const return "GeneratedColumnPlaceholderTransformOp"; } -ReturnOpStatus GeneratedColumnPlaceHolderTransformOp::transformImpl(Block & block) +OperatorStatus GeneratedColumnPlaceHolderTransformOp::transformImpl(Block & block) { action.transform(block); return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.h b/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.h index f56113e0c19..89865fc6996 100644 --- a/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.h +++ b/dbms/src/Operators/GeneratedColumnPlaceHolderTransformOp.h @@ -34,7 +34,7 @@ class GeneratedColumnPlaceHolderTransformOp : public TransformOp String getName() const override; protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/GetResultSinkOp.cpp b/dbms/src/Operators/GetResultSinkOp.cpp index 334c85e6456..16e8fc37a9e 100644 --- a/dbms/src/Operators/GetResultSinkOp.cpp +++ b/dbms/src/Operators/GetResultSinkOp.cpp @@ -16,7 +16,7 @@ namespace DB { -ReturnOpStatus GetResultSinkOp::writeImpl(Block && block) +OperatorStatus GetResultSinkOp::writeImpl(Block && block) { if (!block) return OperatorStatus::FINISHED; @@ -37,12 +37,12 @@ ReturnOpStatus GetResultSinkOp::writeImpl(Block && block) } } -ReturnOpStatus GetResultSinkOp::prepareImpl() +OperatorStatus GetResultSinkOp::prepareImpl() { return awaitImpl(); } -ReturnOpStatus GetResultSinkOp::awaitImpl() +OperatorStatus GetResultSinkOp::awaitImpl() { if (!t_block) return OperatorStatus::NEED_INPUT; diff --git a/dbms/src/Operators/GetResultSinkOp.h b/dbms/src/Operators/GetResultSinkOp.h index c2b66a7b6ff..dda52eec23c 100644 --- a/dbms/src/Operators/GetResultSinkOp.h +++ b/dbms/src/Operators/GetResultSinkOp.h @@ -36,11 +36,11 @@ class GetResultSinkOp : public SinkOp String getName() const override { return "GetResultSinkOp"; } protected: - ReturnOpStatus writeImpl(Block && block) override; + OperatorStatus writeImpl(Block && block) override; - ReturnOpStatus prepareImpl() override; + OperatorStatus prepareImpl() override; - ReturnOpStatus awaitImpl() override; + OperatorStatus awaitImpl() override; private: ResultQueuePtr result_queue; diff --git a/dbms/src/Operators/HashJoinBuildSink.cpp b/dbms/src/Operators/HashJoinBuildSink.cpp index e6fdda95618..5a4330c6d24 100644 --- a/dbms/src/Operators/HashJoinBuildSink.cpp +++ b/dbms/src/Operators/HashJoinBuildSink.cpp @@ -17,7 +17,7 @@ namespace DB { -ReturnOpStatus HashJoinBuildSink::writeImpl(Block && block) +OperatorStatus HashJoinBuildSink::writeImpl(Block && block) { if unlikely (!block) { @@ -35,13 +35,13 @@ ReturnOpStatus HashJoinBuildSink::writeImpl(Block && block) return join_ptr->hasBuildSideMarkedSpillData(op_index) ? OperatorStatus::IO_OUT : OperatorStatus::NEED_INPUT; } -ReturnOpStatus HashJoinBuildSink::prepareImpl() +OperatorStatus HashJoinBuildSink::prepareImpl() { join_ptr->checkAndMarkPartitionSpilledIfNeeded(op_index); return join_ptr->hasBuildSideMarkedSpillData(op_index) ? OperatorStatus::IO_OUT : OperatorStatus::NEED_INPUT; } -ReturnOpStatus HashJoinBuildSink::executeIOImpl() +OperatorStatus HashJoinBuildSink::executeIOImpl() { join_ptr->flushBuildSideMarkedSpillData(op_index); if (is_finish_status) diff --git a/dbms/src/Operators/HashJoinBuildSink.h b/dbms/src/Operators/HashJoinBuildSink.h index 6f729ea8300..fd7788209a8 100644 --- a/dbms/src/Operators/HashJoinBuildSink.h +++ b/dbms/src/Operators/HashJoinBuildSink.h @@ -37,11 +37,11 @@ class HashJoinBuildSink : public SinkOp String getName() const override { return "HashJoinBuildSink"; } protected: - ReturnOpStatus writeImpl(Block && block) override; + OperatorStatus writeImpl(Block && block) override; - ReturnOpStatus prepareImpl() override; + OperatorStatus prepareImpl() override; - ReturnOpStatus executeIOImpl() override; + OperatorStatus executeIOImpl() override; private: JoinPtr join_ptr; diff --git a/dbms/src/Operators/HashJoinProbeTransformOp.cpp b/dbms/src/Operators/HashJoinProbeTransformOp.cpp index 679d81ef66c..ee5de46641e 100644 --- a/dbms/src/Operators/HashJoinProbeTransformOp.cpp +++ b/dbms/src/Operators/HashJoinProbeTransformOp.cpp @@ -71,7 +71,7 @@ void HashJoinProbeTransformOp::operateSuffixImpl() scan_hash_map_rows); } -ReturnOpStatus HashJoinProbeTransformOp::onOutput(Block & block) +OperatorStatus HashJoinProbeTransformOp::onOutput(Block & block) { while (true) { @@ -83,7 +83,7 @@ ReturnOpStatus HashJoinProbeTransformOp::onOutput(Block & block) if (probe_process_info.all_rows_joined_finish) { if (auto ret = probe_transform->tryFillProcessInfoInRestoreProbeStage(probe_process_info); - ret.status != OperatorStatus::HAS_OUTPUT) + ret != OperatorStatus::HAS_OUTPUT) return ret; } case ProbeStatus::PROBE: @@ -147,23 +147,23 @@ ReturnOpStatus HashJoinProbeTransformOp::onOutput(Block & block) } } -ReturnOpStatus HashJoinProbeTransformOp::transformImpl(Block & block) +OperatorStatus HashJoinProbeTransformOp::transformImpl(Block & block) { assert(status == ProbeStatus::PROBE); assert(probe_process_info.all_rows_joined_finish); if (auto ret = probe_transform->tryFillProcessInfoInProbeStage(probe_process_info, block); - ret.status != OperatorStatus::HAS_OUTPUT) + ret != OperatorStatus::HAS_OUTPUT) return ret; return onOutput(block); } -ReturnOpStatus HashJoinProbeTransformOp::tryOutputImpl(Block & block) +OperatorStatus HashJoinProbeTransformOp::tryOutputImpl(Block & block) { if (status == ProbeStatus::PROBE && probe_process_info.all_rows_joined_finish) { if (auto ret = probe_transform->tryFillProcessInfoInProbeStage(probe_process_info); - ret.status != OperatorStatus::HAS_OUTPUT) + ret != OperatorStatus::HAS_OUTPUT) return ret; } @@ -206,7 +206,7 @@ void HashJoinProbeTransformOp::onGetRestoreJoin() } } -ReturnOpStatus HashJoinProbeTransformOp::awaitImpl() +OperatorStatus HashJoinProbeTransformOp::awaitImpl() { while (true) { @@ -240,7 +240,7 @@ ReturnOpStatus HashJoinProbeTransformOp::awaitImpl() } } -ReturnOpStatus HashJoinProbeTransformOp::executeIOImpl() +OperatorStatus HashJoinProbeTransformOp::executeIOImpl() { switch (status) { diff --git a/dbms/src/Operators/HashJoinProbeTransformOp.h b/dbms/src/Operators/HashJoinProbeTransformOp.h index fc586d1a2a1..9be1164c5d9 100644 --- a/dbms/src/Operators/HashJoinProbeTransformOp.h +++ b/dbms/src/Operators/HashJoinProbeTransformOp.h @@ -34,20 +34,20 @@ class HashJoinProbeTransformOp : public TransformOp String getName() const override { return "HashJoinProbeTransformOp"; } protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; - ReturnOpStatus tryOutputImpl(Block & block) override; + OperatorStatus tryOutputImpl(Block & block) override; - ReturnOpStatus awaitImpl() override; + OperatorStatus awaitImpl() override; - ReturnOpStatus executeIOImpl() override; + OperatorStatus executeIOImpl() override; void transformHeaderImpl(Block & header_) override; void operateSuffixImpl() override; private: - ReturnOpStatus onOutput(Block & block); + OperatorStatus onOutput(Block & block); inline void onWaitProbeFinishDone(); diff --git a/dbms/src/Operators/HashProbeTransformExec.cpp b/dbms/src/Operators/HashProbeTransformExec.cpp index 944479d569a..a3260f38769 100644 --- a/dbms/src/Operators/HashProbeTransformExec.cpp +++ b/dbms/src/Operators/HashProbeTransformExec.cpp @@ -154,7 +154,7 @@ Block HashProbeTransformExec::popProbeRestoredBlock() return OperatorStatus::CANCELLED; \ continue; -ReturnOpStatus HashProbeTransformExec::tryFillProcessInfoInRestoreProbeStage(ProbeProcessInfo & probe_process_info) +OperatorStatus HashProbeTransformExec::tryFillProcessInfoInRestoreProbeStage(ProbeProcessInfo & probe_process_info) { while (true) { @@ -205,7 +205,7 @@ ReturnOpStatus HashProbeTransformExec::tryFillProcessInfoInRestoreProbeStage(Pro } } -ReturnOpStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage(ProbeProcessInfo & probe_process_info) +OperatorStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage(ProbeProcessInfo & probe_process_info) { while (true) { @@ -231,7 +231,7 @@ ReturnOpStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage(ProbeProce #undef CONTINUE -ReturnOpStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage( +OperatorStatus HashProbeTransformExec::tryFillProcessInfoInProbeStage( ProbeProcessInfo & probe_process_info, Block & input) { diff --git a/dbms/src/Operators/HashProbeTransformExec.h b/dbms/src/Operators/HashProbeTransformExec.h index 61a250d9339..55af07b88c4 100644 --- a/dbms/src/Operators/HashProbeTransformExec.h +++ b/dbms/src/Operators/HashProbeTransformExec.h @@ -82,10 +82,10 @@ class HashProbeTransformExec : public std::enable_shared_from_thisreadSuffix(); } - ReturnOpStatus readImpl(Block & block) override + OperatorStatus readImpl(Block & block) override { if (unlikely(is_done)) return OperatorStatus::HAS_OUTPUT; @@ -53,7 +53,7 @@ class IOBlockInputStreamSourceOp : public SourceOp return OperatorStatus::HAS_OUTPUT; } - ReturnOpStatus executeIOImpl() override + OperatorStatus executeIOImpl() override { if unlikely (is_done || ret) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/LimitTransformOp.cpp b/dbms/src/Operators/LimitTransformOp.cpp index 508645316fe..ad05e395bd2 100644 --- a/dbms/src/Operators/LimitTransformOp.cpp +++ b/dbms/src/Operators/LimitTransformOp.cpp @@ -18,7 +18,7 @@ namespace DB { template -ReturnOpStatus LimitTransformOp::transformImpl(Block & block) +OperatorStatus LimitTransformOp::transformImpl(Block & block) { if (!action->transform(block)) block = {}; diff --git a/dbms/src/Operators/LimitTransformOp.h b/dbms/src/Operators/LimitTransformOp.h index 51fd6c7ac48..8cbd20a4084 100644 --- a/dbms/src/Operators/LimitTransformOp.h +++ b/dbms/src/Operators/LimitTransformOp.h @@ -31,7 +31,7 @@ class LimitTransformOp : public TransformOp String getName() const override { return "LimitTransformOp"; } protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/LocalAggregateTransform.cpp b/dbms/src/Operators/LocalAggregateTransform.cpp index b0a4eff7c34..a105e62406c 100644 --- a/dbms/src/Operators/LocalAggregateTransform.cpp +++ b/dbms/src/Operators/LocalAggregateTransform.cpp @@ -49,7 +49,7 @@ LocalAggregateTransform::LocalAggregateTransform( }); } -ReturnOpStatus LocalAggregateTransform::transformImpl(Block & block) +OperatorStatus LocalAggregateTransform::transformImpl(Block & block) { switch (status) { @@ -68,7 +68,7 @@ ReturnOpStatus LocalAggregateTransform::transformImpl(Block & block) } } -ReturnOpStatus LocalAggregateTransform::fromBuildToConvergent(Block & block) +OperatorStatus LocalAggregateTransform::fromBuildToConvergent(Block & block) { // status from build to convergent. assert(status == LocalAggStatus::build); @@ -79,7 +79,7 @@ ReturnOpStatus LocalAggregateTransform::fromBuildToConvergent(Block & block) return OperatorStatus::HAS_OUTPUT; } -ReturnOpStatus LocalAggregateTransform::fromBuildToFinalSpillOrRestore() +OperatorStatus LocalAggregateTransform::fromBuildToFinalSpillOrRestore() { assert(status == LocalAggStatus::build); if (agg_context.needSpill(task_index, /*try_mark_need_spill=*/true)) @@ -106,7 +106,7 @@ OperatorStatus LocalAggregateTransform::tryFromBuildToSpill() return OperatorStatus::NEED_INPUT; } -ReturnOpStatus LocalAggregateTransform::tryOutputImpl(Block & block) +OperatorStatus LocalAggregateTransform::tryOutputImpl(Block & block) { switch (status) { @@ -128,7 +128,7 @@ ReturnOpStatus LocalAggregateTransform::tryOutputImpl(Block & block) } } -ReturnOpStatus LocalAggregateTransform::executeIOImpl() +OperatorStatus LocalAggregateTransform::executeIOImpl() { switch (status) { diff --git a/dbms/src/Operators/LocalAggregateTransform.h b/dbms/src/Operators/LocalAggregateTransform.h index 4f72d58ff7c..235ab28c682 100644 --- a/dbms/src/Operators/LocalAggregateTransform.h +++ b/dbms/src/Operators/LocalAggregateTransform.h @@ -33,20 +33,20 @@ class LocalAggregateTransform : public TransformOp String getName() const override { return "LocalAggregateTransform"; } protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; - ReturnOpStatus tryOutputImpl(Block & block) override; + OperatorStatus tryOutputImpl(Block & block) override; - ReturnOpStatus executeIOImpl() override; + OperatorStatus executeIOImpl() override; void transformHeaderImpl(Block & header_) override; private: OperatorStatus tryFromBuildToSpill(); - ReturnOpStatus fromBuildToConvergent(Block & block); + OperatorStatus fromBuildToConvergent(Block & block); - ReturnOpStatus fromBuildToFinalSpillOrRestore(); + OperatorStatus fromBuildToFinalSpillOrRestore(); private: Aggregator::Params params; diff --git a/dbms/src/Operators/MergeSortTransformOp.cpp b/dbms/src/Operators/MergeSortTransformOp.cpp index 4b91cffe227..6e25bee9658 100644 --- a/dbms/src/Operators/MergeSortTransformOp.cpp +++ b/dbms/src/Operators/MergeSortTransformOp.cpp @@ -51,7 +51,7 @@ Block MergeSortTransformOp::getMergeOutput() return block; } -ReturnOpStatus MergeSortTransformOp::fromPartialToMerge(Block & block) +OperatorStatus MergeSortTransformOp::fromPartialToMerge(Block & block) { assert(status == MergeSortStatus::PARTIAL); // convert to merge phase. @@ -72,7 +72,7 @@ ReturnOpStatus MergeSortTransformOp::fromPartialToMerge(Block & block) return OperatorStatus::HAS_OUTPUT; } -ReturnOpStatus MergeSortTransformOp::fromPartialToRestore() +OperatorStatus MergeSortTransformOp::fromPartialToRestore() { assert(status == MergeSortStatus::PARTIAL); // convert to restore phase. @@ -99,7 +99,7 @@ ReturnOpStatus MergeSortTransformOp::fromPartialToRestore() return OperatorStatus::IO_IN; } -ReturnOpStatus MergeSortTransformOp::fromPartialToSpill() +OperatorStatus MergeSortTransformOp::fromPartialToSpill() { assert(status == MergeSortStatus::PARTIAL); // convert to restore phase. @@ -121,7 +121,7 @@ ReturnOpStatus MergeSortTransformOp::fromPartialToSpill() return OperatorStatus::IO_OUT; } -ReturnOpStatus MergeSortTransformOp::fromSpillToPartial() +OperatorStatus MergeSortTransformOp::fromSpillToPartial() { assert(status == MergeSortStatus::SPILL); assert(cached_handler); @@ -133,7 +133,7 @@ ReturnOpStatus MergeSortTransformOp::fromSpillToPartial() return OperatorStatus::NEED_INPUT; } -ReturnOpStatus MergeSortTransformOp::transformImpl(Block & block) +OperatorStatus MergeSortTransformOp::transformImpl(Block & block) { switch (status) { @@ -162,7 +162,7 @@ ReturnOpStatus MergeSortTransformOp::transformImpl(Block & block) } } -ReturnOpStatus MergeSortTransformOp::tryOutputImpl(Block & block) +OperatorStatus MergeSortTransformOp::tryOutputImpl(Block & block) { switch (status) { @@ -195,7 +195,7 @@ ReturnOpStatus MergeSortTransformOp::tryOutputImpl(Block & block) } } -ReturnOpStatus MergeSortTransformOp::executeIOImpl() +OperatorStatus MergeSortTransformOp::executeIOImpl() { switch (status) { diff --git a/dbms/src/Operators/MergeSortTransformOp.h b/dbms/src/Operators/MergeSortTransformOp.h index a377a351338..d463b16c208 100644 --- a/dbms/src/Operators/MergeSortTransformOp.h +++ b/dbms/src/Operators/MergeSortTransformOp.h @@ -54,10 +54,10 @@ class MergeSortTransformOp : public TransformOp void operatePrefixImpl() override; void operateSuffixImpl() override; - ReturnOpStatus transformImpl(Block & block) override; - ReturnOpStatus tryOutputImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; + OperatorStatus tryOutputImpl(Block & block) override; - ReturnOpStatus executeIOImpl() override; + OperatorStatus executeIOImpl() override; void transformHeaderImpl(Block & header_) override; @@ -65,14 +65,14 @@ class MergeSortTransformOp : public TransformOp Block getMergeOutput(); // PARTIAL◄─────►SPILL - ReturnOpStatus fromPartialToSpill(); - ReturnOpStatus fromSpillToPartial(); + OperatorStatus fromPartialToSpill(); + OperatorStatus fromSpillToPartial(); // PARTIAL─────►RESTORE - ReturnOpStatus fromPartialToRestore(); + OperatorStatus fromPartialToRestore(); // PARTIAL─────►MERGE - ReturnOpStatus fromPartialToMerge(Block & block); + OperatorStatus fromPartialToMerge(Block & block); private: bool hasSpilledData() const { return sort_spill_context->hasSpilledData(); } diff --git a/dbms/src/Operators/NullSourceOp.h b/dbms/src/Operators/NullSourceOp.h index 0cf6775de0e..43a03802522 100644 --- a/dbms/src/Operators/NullSourceOp.h +++ b/dbms/src/Operators/NullSourceOp.h @@ -33,7 +33,7 @@ class NullSourceOp : public SourceOp IOProfileInfoPtr getIOProfileInfo() const override { return IOProfileInfo::createForLocal(profile_info_ptr); } protected: - ReturnOpStatus readImpl(Block & block) override + OperatorStatus readImpl(Block & block) override { block = {}; return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/Operator.cpp b/dbms/src/Operators/Operator.cpp index 14e2e07718b..03da1645ede 100644 --- a/dbms/src/Operators/Operator.cpp +++ b/dbms/src/Operators/Operator.cpp @@ -45,7 +45,7 @@ void Operator::operateSuffix() if (unlikely(exec_context.isCancelled())) \ return OperatorStatus::CANCELLED; -ReturnOpStatus Operator::await() +OperatorStatus Operator::await() { // `exec_context.is_cancelled` has been checked by `EventTask`. // If `exec_context.is_cancelled` is checked here, the overhead of `exec_context.is_cancelled` will be amplified by the high frequency of `await` calls. @@ -76,7 +76,7 @@ ReturnOpStatus Operator::await() return return_status; } -ReturnOpStatus Operator::executeIO() +OperatorStatus Operator::executeIO() { CHECK_IS_CANCELLED profile_info.anchor(); @@ -92,7 +92,7 @@ ReturnOpStatus Operator::executeIO() return return_status; } -ReturnOpStatus SourceOp::read(Block & block) +OperatorStatus SourceOp::read(Block & block) { CHECK_IS_CANCELLED profile_info.anchor(); @@ -115,7 +115,7 @@ ReturnOpStatus SourceOp::read(Block & block) return return_status; } -ReturnOpStatus TransformOp::transform(Block & block) +OperatorStatus TransformOp::transform(Block & block) { CHECK_IS_CANCELLED profile_info.anchor(); @@ -137,7 +137,7 @@ ReturnOpStatus TransformOp::transform(Block & block) return return_status; } -ReturnOpStatus TransformOp::tryOutput(Block & block) +OperatorStatus TransformOp::tryOutput(Block & block) { CHECK_IS_CANCELLED profile_info.anchor(); @@ -160,7 +160,7 @@ ReturnOpStatus TransformOp::tryOutput(Block & block) return return_status; } -ReturnOpStatus SinkOp::prepare() +OperatorStatus SinkOp::prepare() { CHECK_IS_CANCELLED profile_info.anchor(); @@ -173,7 +173,7 @@ ReturnOpStatus SinkOp::prepare() return return_status; } -ReturnOpStatus SinkOp::write(Block && block) +OperatorStatus SinkOp::write(Block && block) { CHECK_IS_CANCELLED profile_info.anchor(block); diff --git a/dbms/src/Operators/Operator.h b/dbms/src/Operators/Operator.h index a3f4de2e7ef..d6deec19a02 100644 --- a/dbms/src/Operators/Operator.h +++ b/dbms/src/Operators/Operator.h @@ -50,29 +50,6 @@ enum class OperatorStatus class PipelineExecutorContext; -struct NotifyFuture; -using NotifyFuturePtr = std::shared_ptr; - -struct ReturnOpStatus -{ - ReturnOpStatus(OperatorStatus status_) // NOLINT(google-explicit-constructor) - : status(status_) - , future(nullptr) - { - assert(status != OperatorStatus::WAIT_FOR_NOTIFY); - } - - ReturnOpStatus(NotifyFuturePtr furture_) // NOLINT(google-explicit-constructor) - : status(OperatorStatus::WAIT_FOR_NOTIFY) - , future(std::move(furture_)) - { - assert(future != nullptr); - } - - OperatorStatus status; - NotifyFuturePtr future{nullptr}; -}; - class Operator { public: @@ -84,10 +61,10 @@ class Operator virtual ~Operator() = default; // running status may return are NEED_INPUT and HAS_OUTPUT here. - ReturnOpStatus executeIO(); + OperatorStatus executeIO(); // running status may return are NEED_INPUT and HAS_OUTPUT here. - ReturnOpStatus await(); + OperatorStatus await(); // These two methods are used to set state, log and etc, and should not perform calculation logic. void operatePrefix(); @@ -117,9 +94,9 @@ class Operator virtual void operatePrefixImpl() {} virtual void operateSuffixImpl() {} - virtual ReturnOpStatus executeIOImpl() { throw Exception("Unsupport"); } + virtual OperatorStatus executeIOImpl() { throw Exception("Unsupport"); } - virtual ReturnOpStatus awaitImpl() { throw Exception("Unsupport"); } + virtual OperatorStatus awaitImpl() { throw Exception("Unsupport"); } protected: PipelineExecutorContext & exec_context; @@ -141,8 +118,8 @@ class SourceOp : public Operator // read will inplace the block when return status is HAS_OUTPUT; // Even after source has finished, source op still needs to return an empty block and HAS_OUTPUT, // because there are many operators that need an empty block as input, such as JoinProbe and WindowFunction. - ReturnOpStatus read(Block & block); - virtual ReturnOpStatus readImpl(Block & block) = 0; + OperatorStatus read(Block & block); + virtual OperatorStatus readImpl(Block & block) = 0; }; using SourceOpPtr = std::unique_ptr; using SourceOps = std::vector; @@ -155,14 +132,14 @@ class TransformOp : public Operator {} // running status may return are NEED_INPUT and HAS_OUTPUT here. // tryOutput will inplace the block when return status is HAS_OUPUT; do nothing to the block when NEED_INPUT or others. - ReturnOpStatus tryOutput(Block &); - virtual ReturnOpStatus tryOutputImpl(Block &) { return OperatorStatus::NEED_INPUT; } + OperatorStatus tryOutput(Block &); + virtual OperatorStatus tryOutputImpl(Block &) { return OperatorStatus::NEED_INPUT; } // running status may return are NEED_INPUT and HAS_OUTPUT here. // transform will inplace the block and if the return status is HAS_OUTPUT, this block can be used as input to subsequent operators. // Even if an empty block is input, transform will still return HAS_OUTPUT, // because there are many operators that need an empty block as input, such as JoinProbe and WindowFunction. - ReturnOpStatus transform(Block & block); - virtual ReturnOpStatus transformImpl(Block & block) = 0; + OperatorStatus transform(Block & block); + virtual OperatorStatus transformImpl(Block & block) = 0; virtual void transformHeaderImpl(Block & header_) = 0; void transformHeader(Block & header_) @@ -182,11 +159,11 @@ class SinkOp : public Operator SinkOp(PipelineExecutorContext & exec_context_, const String & req_id) : Operator(exec_context_, req_id) {} - ReturnOpStatus prepare(); - virtual ReturnOpStatus prepareImpl() { return OperatorStatus::NEED_INPUT; } + OperatorStatus prepare(); + virtual OperatorStatus prepareImpl() { return OperatorStatus::NEED_INPUT; } - ReturnOpStatus write(Block && block); - virtual ReturnOpStatus writeImpl(Block && block) = 0; + OperatorStatus write(Block && block); + virtual OperatorStatus writeImpl(Block && block) = 0; }; using SinkOpPtr = std::unique_ptr; } // namespace DB diff --git a/dbms/src/Operators/PartialSortTransformOp.cpp b/dbms/src/Operators/PartialSortTransformOp.cpp index 81fb451bc8c..df3a036142b 100644 --- a/dbms/src/Operators/PartialSortTransformOp.cpp +++ b/dbms/src/Operators/PartialSortTransformOp.cpp @@ -17,7 +17,7 @@ namespace DB { -ReturnOpStatus PartialSortTransformOp::transformImpl(Block & block) +OperatorStatus PartialSortTransformOp::transformImpl(Block & block) { sortBlock(block, order_desc, limit); return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/PartialSortTransformOp.h b/dbms/src/Operators/PartialSortTransformOp.h index 53e6e95014d..0e28d53254e 100644 --- a/dbms/src/Operators/PartialSortTransformOp.h +++ b/dbms/src/Operators/PartialSortTransformOp.h @@ -37,7 +37,7 @@ class PartialSortTransformOp : public TransformOp String getName() const override { return "PartialSortTransformOp"; } protected: - ReturnOpStatus transformImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; void transformHeaderImpl(Block & /*header_*/) override {} diff --git a/dbms/src/Operators/SharedQueue.cpp b/dbms/src/Operators/SharedQueue.cpp index d012a1d82fd..609323fa09e 100644 --- a/dbms/src/Operators/SharedQueue.cpp +++ b/dbms/src/Operators/SharedQueue.cpp @@ -50,7 +50,7 @@ void SharedQueue::producerFinish() queue.finish(); } -ReturnOpStatus SharedQueueSinkOp::writeImpl(Block && block) +OperatorStatus SharedQueueSinkOp::writeImpl(Block && block) { if unlikely (!block) return OperatorStatus::FINISHED; @@ -60,12 +60,12 @@ ReturnOpStatus SharedQueueSinkOp::writeImpl(Block && block) return awaitImpl(); } -ReturnOpStatus SharedQueueSinkOp::prepareImpl() +OperatorStatus SharedQueueSinkOp::prepareImpl() { return awaitImpl(); } -ReturnOpStatus SharedQueueSinkOp::awaitImpl() +OperatorStatus SharedQueueSinkOp::awaitImpl() { if (!res) return OperatorStatus::NEED_INPUT; @@ -87,7 +87,7 @@ ReturnOpStatus SharedQueueSinkOp::awaitImpl() } } -ReturnOpStatus SharedQueueSourceOp::readImpl(Block & block) +OperatorStatus SharedQueueSourceOp::readImpl(Block & block) { auto await_status = awaitImpl(); if (await_status.status == OperatorStatus::HAS_OUTPUT && res) @@ -98,7 +98,7 @@ ReturnOpStatus SharedQueueSourceOp::readImpl(Block & block) return await_status; } -ReturnOpStatus SharedQueueSourceOp::awaitImpl() +OperatorStatus SharedQueueSourceOp::awaitImpl() { if (res) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/SharedQueue.h b/dbms/src/Operators/SharedQueue.h index 8eb05693be2..59053167574 100644 --- a/dbms/src/Operators/SharedQueue.h +++ b/dbms/src/Operators/SharedQueue.h @@ -60,11 +60,11 @@ class SharedQueueSinkOp : public SinkOp String getName() const override { return "SharedQueueSinkOp"; } - ReturnOpStatus prepareImpl() override; + OperatorStatus prepareImpl() override; - ReturnOpStatus writeImpl(Block && block) override; + OperatorStatus writeImpl(Block && block) override; - ReturnOpStatus awaitImpl() override; + OperatorStatus awaitImpl() override; private: std::optional res; @@ -87,9 +87,9 @@ class SharedQueueSourceOp : public SourceOp String getName() const override { return "SharedQueueSourceOp"; } - ReturnOpStatus readImpl(Block & block) override; + OperatorStatus readImpl(Block & block) override; - ReturnOpStatus awaitImpl() override; + OperatorStatus awaitImpl() override; private: std::optional res; diff --git a/dbms/src/Operators/UnorderedSourceOp.cpp b/dbms/src/Operators/UnorderedSourceOp.cpp index 550320fe0ab..7973278f482 100644 --- a/dbms/src/Operators/UnorderedSourceOp.cpp +++ b/dbms/src/Operators/UnorderedSourceOp.cpp @@ -60,18 +60,18 @@ UnorderedSourceOp::UnorderedSourceOp( ref_no = task_pool->increaseUnorderedInputStreamRefCount(); } -ReturnOpStatus UnorderedSourceOp::readImpl(Block & block) +OperatorStatus UnorderedSourceOp::readImpl(Block & block) { if unlikely (done) return OperatorStatus::HAS_OUTPUT; - auto await_status = doFetchBlock(); - if (await_status.status == OperatorStatus::HAS_OUTPUT) + auto status = doFetchBlock(); + if (status == OperatorStatus::HAS_OUTPUT) std::swap(block, t_block); - return await_status; + return status; } -ReturnOpStatus UnorderedSourceOp::doFetchBlock() +OperatorStatus UnorderedSourceOp::doFetchBlock() { if (t_block) return OperatorStatus::HAS_OUTPUT; @@ -79,7 +79,11 @@ ReturnOpStatus UnorderedSourceOp::doFetchBlock() while (true) { if (!task_pool->tryPopBlock(t_block)) - return notify_future; + { + setNotifyFuture(notify_future); + return OperatorStatus::WAIT_FOR_NOTIFY; + } + if (t_block) { if unlikely (t_block.rows() == 0) diff --git a/dbms/src/Operators/UnorderedSourceOp.h b/dbms/src/Operators/UnorderedSourceOp.h index 1fadb39b84d..55f653031ef 100644 --- a/dbms/src/Operators/UnorderedSourceOp.h +++ b/dbms/src/Operators/UnorderedSourceOp.h @@ -23,6 +23,9 @@ namespace DB { +struct NotifyFuture; +using NotifyFuturePtr = std::shared_ptr; + /// Read blocks asyncly from Storage Layer by using read thread, /// The result can not guarantee the keep_order property class UnorderedSourceOp : public SourceOp @@ -66,10 +69,10 @@ class UnorderedSourceOp : public SourceOp protected: void operatePrefixImpl() override; - ReturnOpStatus readImpl(Block & block) override; + OperatorStatus readImpl(Block & block) override; private: - ReturnOpStatus doFetchBlock(); + OperatorStatus doFetchBlock(); private: DM::SegmentReadTaskPoolPtr task_pool; diff --git a/dbms/src/Operators/WindowTransformOp.cpp b/dbms/src/Operators/WindowTransformOp.cpp index b091802ef7e..e65df50b0f0 100644 --- a/dbms/src/Operators/WindowTransformOp.cpp +++ b/dbms/src/Operators/WindowTransformOp.cpp @@ -37,7 +37,7 @@ void WindowTransformOp::operateSuffixImpl() action->cleanUp(); } -ReturnOpStatus WindowTransformOp::transformImpl(Block & block) +OperatorStatus WindowTransformOp::transformImpl(Block & block) { assert(action); assert(!action->input_is_finished); @@ -57,7 +57,7 @@ ReturnOpStatus WindowTransformOp::transformImpl(Block & block) } } -ReturnOpStatus WindowTransformOp::tryOutputImpl(Block & block) +OperatorStatus WindowTransformOp::tryOutputImpl(Block & block) { assert(action); block = action->tryGetOutputBlock(); diff --git a/dbms/src/Operators/WindowTransformOp.h b/dbms/src/Operators/WindowTransformOp.h index 906f24a980d..34f452bf547 100644 --- a/dbms/src/Operators/WindowTransformOp.h +++ b/dbms/src/Operators/WindowTransformOp.h @@ -32,8 +32,8 @@ class WindowTransformOp : public TransformOp protected: void operateSuffixImpl() override; - ReturnOpStatus transformImpl(Block & block) override; - ReturnOpStatus tryOutputImpl(Block & block) override; + OperatorStatus transformImpl(Block & block) override; + OperatorStatus tryOutputImpl(Block & block) override; void transformHeaderImpl(Block & header_) override; diff --git a/dbms/src/Operators/tests/gtest_concat_source.cpp b/dbms/src/Operators/tests/gtest_concat_source.cpp index a4f1a3dccb9..c6090ee5bd4 100644 --- a/dbms/src/Operators/tests/gtest_concat_source.cpp +++ b/dbms/src/Operators/tests/gtest_concat_source.cpp @@ -37,7 +37,7 @@ class MockSourceOp : public SourceOp String getName() const override { return "MockSourceOp"; } protected: - ReturnOpStatus readImpl(Block & block) override + OperatorStatus readImpl(Block & block) override { std::swap(block, output); return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.cpp b/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.cpp index 0901125a35a..17e467295c0 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.cpp @@ -48,7 +48,7 @@ void RNSegmentSourceOp::operatePrefixImpl() workers->startInBackground(); } -ReturnOpStatus RNSegmentSourceOp::startGettingNextReadyTask() +OperatorStatus RNSegmentSourceOp::startGettingNextReadyTask() { // Start timing the time of get next ready task. wait_stop_watch.start(); @@ -56,7 +56,7 @@ ReturnOpStatus RNSegmentSourceOp::startGettingNextReadyTask() return awaitImpl(); } -ReturnOpStatus RNSegmentSourceOp::readImpl(Block & block) +OperatorStatus RNSegmentSourceOp::readImpl(Block & block) { if unlikely (done) { @@ -75,7 +75,7 @@ ReturnOpStatus RNSegmentSourceOp::readImpl(Block & block) return current_seg_task ? OperatorStatus::IO_IN : startGettingNextReadyTask(); } -ReturnOpStatus RNSegmentSourceOp::awaitImpl() +OperatorStatus RNSegmentSourceOp::awaitImpl() { if unlikely (done || t_block.has_value()) { @@ -117,7 +117,7 @@ ReturnOpStatus RNSegmentSourceOp::awaitImpl() } } -ReturnOpStatus RNSegmentSourceOp::executeIOImpl() +OperatorStatus RNSegmentSourceOp::executeIOImpl() { if unlikely (done || t_block.has_value()) return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.h b/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.h index 4e5d5cb86b4..d3a4174655d 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.h +++ b/dbms/src/Storages/DeltaMerge/Remote/RNSegmentSourceOp.h @@ -55,14 +55,14 @@ class RNSegmentSourceOp : public SourceOp void operatePrefixImpl() override; - ReturnOpStatus readImpl(Block & block) override; + OperatorStatus readImpl(Block & block) override; - ReturnOpStatus awaitImpl() override; + OperatorStatus awaitImpl() override; - ReturnOpStatus executeIOImpl() override; + OperatorStatus executeIOImpl() override; private: - ReturnOpStatus startGettingNextReadyTask(); + OperatorStatus startGettingNextReadyTask(); private: const RNWorkersPtr workers; From 71275bfe5647de9955acae5fb18c8a11f3106779 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 19:30:42 +0800 Subject: [PATCH 09/19] tmp --- dbms/src/Operators/Operator.cpp | 56 ++++++++++++++++----------------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/dbms/src/Operators/Operator.cpp b/dbms/src/Operators/Operator.cpp index 03da1645ede..6cd2a0e3b79 100644 --- a/dbms/src/Operators/Operator.cpp +++ b/dbms/src/Operators/Operator.cpp @@ -50,10 +50,10 @@ OperatorStatus Operator::await() // `exec_context.is_cancelled` has been checked by `EventTask`. // If `exec_context.is_cancelled` is checked here, the overhead of `exec_context.is_cancelled` will be amplified by the high frequency of `await` calls. - auto return_status = awaitImpl(); + auto op_status = awaitImpl(); #ifndef NDEBUG assertOperatorStatus( - return_status.status, + op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); @@ -68,28 +68,28 @@ OperatorStatus Operator::await() // ┌────────────────waiting time───────────┐ // [non-waiting, waiting, waiting, waiting, .., waiting, non-waiting] - if (return_status.status != OperatorStatus::WAITING) + if (op_status != OperatorStatus::WAITING) { exec_context.triggerAutoSpill(); profile_info.update(); } - return return_status; + return op_status; } OperatorStatus Operator::executeIO() { CHECK_IS_CANCELLED profile_info.anchor(); - auto return_status = executeIOImpl(); + auto op_status = executeIOImpl(); #ifndef NDEBUG assertOperatorStatus( - return_status.status, + op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return return_status; + return op_status; } OperatorStatus SourceOp::read(Block & block) @@ -97,44 +97,44 @@ OperatorStatus SourceOp::read(Block & block) CHECK_IS_CANCELLED profile_info.anchor(); assert(!block); - auto return_status = readImpl(block); + auto op_status = readImpl(block); #ifndef NDEBUG - if (return_status.status == OperatorStatus::HAS_OUTPUT && block) + if (op_status == OperatorStatus::HAS_OUTPUT && block) { Block header = getHeader(); assertBlocksHaveEqualStructure(block, header, getName()); } - assertOperatorStatus(return_status.status, {OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus(op_status, {OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); - if (return_status.status == OperatorStatus::HAS_OUTPUT) + if (op_status == OperatorStatus::HAS_OUTPUT) profile_info.update(block); else profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return return_status; + return op_status; } OperatorStatus TransformOp::transform(Block & block) { CHECK_IS_CANCELLED profile_info.anchor(); - auto return_status = transformImpl(block); + auto op_status = transformImpl(block); #ifndef NDEBUG - if (return_status.status == OperatorStatus::HAS_OUTPUT && block) + if (op_status == OperatorStatus::HAS_OUTPUT && block) { Block header = getHeader(); assertBlocksHaveEqualStructure(block, header, getName()); } - assertOperatorStatus(return_status.status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); - if (return_status.status == OperatorStatus::HAS_OUTPUT) + if (op_status == OperatorStatus::HAS_OUTPUT) profile_info.update(block); else profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return return_status; + return op_status; } OperatorStatus TransformOp::tryOutput(Block & block) @@ -142,35 +142,35 @@ OperatorStatus TransformOp::tryOutput(Block & block) CHECK_IS_CANCELLED profile_info.anchor(); assert(!block); - auto return_status = tryOutputImpl(block); + auto op_status = tryOutputImpl(block); #ifndef NDEBUG - if (return_status.status == OperatorStatus::HAS_OUTPUT && block) + if (op_status == OperatorStatus::HAS_OUTPUT && block) { Block header = getHeader(); assertBlocksHaveEqualStructure(block, header, getName()); } - assertOperatorStatus(return_status.status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); - if (return_status.status == OperatorStatus::HAS_OUTPUT) + if (op_status == OperatorStatus::HAS_OUTPUT) profile_info.update(block); else profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return return_status; + return op_status; } OperatorStatus SinkOp::prepare() { CHECK_IS_CANCELLED profile_info.anchor(); - auto return_status = prepareImpl(); + auto op_status = prepareImpl(); #ifndef NDEBUG - assertOperatorStatus(return_status.status, {OperatorStatus::NEED_INPUT}); + assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT}); #endif profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return return_status; + return op_status; } OperatorStatus SinkOp::write(Block && block) @@ -184,14 +184,14 @@ OperatorStatus SinkOp::write(Block && block) assertBlocksHaveEqualStructure(block, header, getName()); } #endif - auto return_status = writeImpl(std::move(block)); + auto op_status = writeImpl(std::move(block)); #ifndef NDEBUG - assertOperatorStatus(return_status.status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT}); + assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT}); #endif exec_context.triggerAutoSpill(); profile_info.update(); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); - return return_status; + return op_status; } #undef CHECK_IS_CANCELLED From afff674b07f407fca67c724bf79312501d53c5c7 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 21:15:24 +0800 Subject: [PATCH 10/19] update --- dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp | 24 +++---- .../TaskQueues/tests/bench_task_queue.cpp | 2 +- .../Pipeline/Schedule/Tasks/NotifyFuture.cpp | 2 - .../Schedule/Tasks/PipeConditionVariable.h | 6 +- .../Schedule/Tasks/PipelineTaskBase.h | 66 +++++++++---------- .../Flash/Pipeline/Schedule/Tasks/Task.cpp | 2 +- .../Operators/CoprocessorReaderSourceOp.cpp | 2 +- .../Operators/ExchangeReceiverSourceOp.cpp | 2 +- dbms/src/Operators/Operator.cpp | 8 +-- dbms/src/Operators/SharedQueue.cpp | 2 +- dbms/src/Operators/UnorderedSourceOp.cpp | 2 +- 11 files changed, 54 insertions(+), 64 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp index 2ea3fe516f2..53b1fda15fb 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp @@ -25,7 +25,7 @@ extern const char random_pipeline_model_execute_suffix_failpoint[]; } // namespace FailPoints #define HANDLE_OP_STATUS(op, op_status, expect_status) \ - switch ((op_status).status) \ + switch (op_status) \ { \ /* For the expected status, it will not return here, */ \ /* but instead return control to the macro caller, */ \ @@ -48,7 +48,7 @@ extern const char random_pipeline_model_execute_suffix_failpoint[]; #define HANDLE_LAST_OP_STATUS(op, op_status) \ assert(op); \ - switch ((op_status).status) \ + switch (op_status) \ { \ /* For the io status, the operator needs to be filled in io_op for later use in executeIO. */ \ case OperatorStatus::IO_IN: \ @@ -94,7 +94,7 @@ OperatorStatus PipelineExec::execute() auto op_status = executeImpl(); #ifndef NDEBUG // `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute`. - assertOperatorStatus(op_status.status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT}); + assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT}); #endif return op_status; } @@ -111,7 +111,7 @@ OperatorStatus PipelineExec::executeImpl() size_t start_transform_op_index = 0; auto op_status = fetchBlock(block, start_transform_op_index); // If the status `fetchBlock` returns isn't `HAS_OUTPUT`, it means that `fetchBlock` did not return a block. - if (op_status.status != OperatorStatus::HAS_OUTPUT) + if (op_status != OperatorStatus::HAS_OUTPUT) return op_status; // start from the next transform op after fetched block transform op. @@ -150,9 +150,7 @@ OperatorStatus PipelineExec::executeIO() #ifndef NDEBUG // `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute`. // `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute`. - assertOperatorStatus( - op_status.status, - {OperatorStatus::FINISHED, OperatorStatus::HAS_OUTPUT, OperatorStatus::NEED_INPUT}); + assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::HAS_OUTPUT, OperatorStatus::NEED_INPUT}); #endif return op_status; } @@ -160,9 +158,9 @@ OperatorStatus PipelineExec::executeIOImpl() { assert(io_op); auto op_status = io_op->executeIO(); - if (op_status.status == OperatorStatus::WAITING) + if (op_status == OperatorStatus::WAITING) fillAwaitable(io_op); - if (op_status.status != OperatorStatus::IO_IN && op_status.status != OperatorStatus::IO_OUT) + if (op_status != OperatorStatus::IO_IN && op_status != OperatorStatus::IO_OUT) io_op = nullptr; return op_status; } @@ -173,9 +171,7 @@ OperatorStatus PipelineExec::await() #ifndef NDEBUG // `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute`. // `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute`. - assertOperatorStatus( - op_status.status, - {OperatorStatus::FINISHED, OperatorStatus::HAS_OUTPUT, OperatorStatus::NEED_INPUT}); + assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::HAS_OUTPUT, OperatorStatus::NEED_INPUT}); #endif return op_status; } @@ -183,9 +179,9 @@ OperatorStatus PipelineExec::awaitImpl() { assert(awaitable); auto op_status = awaitable->await(); - if (op_status.status == OperatorStatus::IO_IN || op_status.status == OperatorStatus::IO_OUT) + if (op_status == OperatorStatus::IO_IN || op_status == OperatorStatus::IO_OUT) fillIOOp(awaitable); - if (op_status.status != OperatorStatus::WAITING) + if (op_status != OperatorStatus::WAITING) awaitable = nullptr; return op_status; } diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp index 636cd06f588..03397bed9d0 100644 --- a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/bench_task_queue.cpp @@ -33,7 +33,7 @@ namespace tests : Task(exec_context) \ {} \ \ - ExecTaskStatus executeImpl() override \ + ExecTaskStatus executeImpl() override \ { \ if (task_exec_cur_count <= task_exec_total_count) \ { \ diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp index d12a03e2ca7..362f4922844 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#pragma once - #include namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index 51cbae5ddb9..d60146514ac 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -32,7 +32,7 @@ class PipeConditionVariable tasks.push_back(std::move(task)); #if __APPLE__ && __clang__ - __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); + __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #else thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #endif @@ -48,7 +48,7 @@ class PipeConditionVariable notifyTaskDirectly(std::move(task)); #if __APPLE__ && __clang__ - __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); + __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #else thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #endif @@ -67,7 +67,7 @@ class PipeConditionVariable } #if __APPLE__ && __clang__ - __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); + __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #else thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #endif diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h index 9af1f47957f..34274e33880 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTaskBase.h @@ -29,30 +29,30 @@ namespace DB /// - OperatorStatus::WAITING ==> ExecTaskStatus::WAITING /// - OperatorStatus::NEED_INPUT/HAS_OUTPUT ==> ExecTaskStatus::RUNNING -#define MAP_NOT_RUNNING_TASK_STATUS(op_status) \ - case OperatorStatus::FINISHED: \ - { \ - return ExecTaskStatus::FINISHED; \ - } \ - case OperatorStatus::CANCELLED: \ - { \ - return ExecTaskStatus::CANCELLED; \ - } \ - case OperatorStatus::IO_IN: \ - { \ - return ExecTaskStatus::IO_IN; \ - } \ - case OperatorStatus::IO_OUT: \ - { \ - return ExecTaskStatus::IO_OUT; \ - } \ - case OperatorStatus::WAITING: \ - { \ - return ExecTaskStatus::WAITING; \ - } \ - case OperatorStatus::WAIT_FOR_NOTIFY: \ - { \ - return std::move((op_status).future); \ +#define MAP_NOT_RUNNING_TASK_STATUS \ + case OperatorStatus::FINISHED: \ + { \ + return ExecTaskStatus::FINISHED; \ + } \ + case OperatorStatus::CANCELLED: \ + { \ + return ExecTaskStatus::CANCELLED; \ + } \ + case OperatorStatus::IO_IN: \ + { \ + return ExecTaskStatus::IO_IN; \ + } \ + case OperatorStatus::IO_OUT: \ + { \ + return ExecTaskStatus::IO_OUT; \ + } \ + case OperatorStatus::WAITING: \ + { \ + return ExecTaskStatus::WAITING; \ + } \ + case OperatorStatus::WAIT_FOR_NOTIFY: \ + { \ + return ExecTaskStatus::WAIT_FOR_NOTIFY; \ } #define UNEXPECTED_OP_STATUS(op_status, function_name) \ @@ -74,15 +74,15 @@ class PipelineTaskBase { assert(pipeline_exec); auto op_status = (pipeline_exec)->execute(); - switch (op_status.status) + switch (op_status) { - MAP_NOT_RUNNING_TASK_STATUS(op_status) + MAP_NOT_RUNNING_TASK_STATUS /* After `pipeline_exec->execute`, `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute` */ /* And other states are unexpected. */ case OperatorStatus::NEED_INPUT: return ExecTaskStatus::RUNNING; default: - UNEXPECTED_OP_STATUS(op_status.status, "PipelineExec::execute"); + UNEXPECTED_OP_STATUS(op_status, "PipelineExec::execute"); } } @@ -90,9 +90,9 @@ class PipelineTaskBase { assert(pipeline_exec); auto op_status = (pipeline_exec)->executeIO(); - switch (op_status.status) + switch (op_status) { - MAP_NOT_RUNNING_TASK_STATUS(op_status) + MAP_NOT_RUNNING_TASK_STATUS /* After `pipeline_exec->executeIO`, */ /* - `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute` */ /* - `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute` */ @@ -101,7 +101,7 @@ class PipelineTaskBase case OperatorStatus::HAS_OUTPUT: return ExecTaskStatus::RUNNING; default: - UNEXPECTED_OP_STATUS(op_status.status, "PipelineExec::execute"); + UNEXPECTED_OP_STATUS(op_status, "PipelineExec::execute"); } } @@ -109,9 +109,9 @@ class PipelineTaskBase { assert(pipeline_exec); auto op_status = (pipeline_exec)->await(); - switch (op_status.status) + switch (op_status) { - MAP_NOT_RUNNING_TASK_STATUS(op_status) + MAP_NOT_RUNNING_TASK_STATUS /* After `pipeline_exec->await`, */ /* - `NEED_INPUT` means that pipeline_exec need data to do the calculations and expect the next call to `execute` */ /* - `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute` */ @@ -120,7 +120,7 @@ class PipelineTaskBase case OperatorStatus::HAS_OUTPUT: return ExecTaskStatus::RUNNING; default: - UNEXPECTED_OP_STATUS(op_status.status, "PipelineExec::await"); + UNEXPECTED_OP_STATUS(op_status, "PipelineExec::await"); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index c989b06ba20..e575b5e32b4 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -117,7 +117,7 @@ Task::~Task() auto return_status = (function()); \ FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_task_run_failpoint); \ FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_query_run); \ - switchStatus(return_status.status); \ + switchStatus(return_status); \ return return_status; \ } \ catch (...) \ diff --git a/dbms/src/Operators/CoprocessorReaderSourceOp.cpp b/dbms/src/Operators/CoprocessorReaderSourceOp.cpp index e223227c4a3..45d6f3d32fb 100644 --- a/dbms/src/Operators/CoprocessorReaderSourceOp.cpp +++ b/dbms/src/Operators/CoprocessorReaderSourceOp.cpp @@ -66,7 +66,7 @@ OperatorStatus CoprocessorReaderSourceOp::readImpl(Block & block) { assert(block_queue.empty()); auto await_status = awaitImpl(); - if (await_status.status == OperatorStatus::HAS_OUTPUT) + if (await_status == OperatorStatus::HAS_OUTPUT) { assert(reader_res); assert(reader_res->second || reader_res->first.finished); diff --git a/dbms/src/Operators/ExchangeReceiverSourceOp.cpp b/dbms/src/Operators/ExchangeReceiverSourceOp.cpp index 3df7886c980..8e017fa497c 100644 --- a/dbms/src/Operators/ExchangeReceiverSourceOp.cpp +++ b/dbms/src/Operators/ExchangeReceiverSourceOp.cpp @@ -41,7 +41,7 @@ OperatorStatus ExchangeReceiverSourceOp::readImpl(Block & block) { assert(block_queue.empty()); auto await_status = awaitImpl(); - if (await_status.status == OperatorStatus::HAS_OUTPUT) + if (await_status == OperatorStatus::HAS_OUTPUT) { assert(receive_status != ReceiveStatus::empty); auto result diff --git a/dbms/src/Operators/Operator.cpp b/dbms/src/Operators/Operator.cpp index 6cd2a0e3b79..109a339be41 100644 --- a/dbms/src/Operators/Operator.cpp +++ b/dbms/src/Operators/Operator.cpp @@ -52,9 +52,7 @@ OperatorStatus Operator::await() auto op_status = awaitImpl(); #ifndef NDEBUG - assertOperatorStatus( - op_status, - {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_operator_run_failpoint); @@ -82,9 +80,7 @@ OperatorStatus Operator::executeIO() profile_info.anchor(); auto op_status = executeIOImpl(); #ifndef NDEBUG - assertOperatorStatus( - op_status, - {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); + assertOperatorStatus(op_status, {OperatorStatus::FINISHED, OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); #endif exec_context.triggerAutoSpill(); profile_info.update(); diff --git a/dbms/src/Operators/SharedQueue.cpp b/dbms/src/Operators/SharedQueue.cpp index 609323fa09e..9d304102fea 100644 --- a/dbms/src/Operators/SharedQueue.cpp +++ b/dbms/src/Operators/SharedQueue.cpp @@ -90,7 +90,7 @@ OperatorStatus SharedQueueSinkOp::awaitImpl() OperatorStatus SharedQueueSourceOp::readImpl(Block & block) { auto await_status = awaitImpl(); - if (await_status.status == OperatorStatus::HAS_OUTPUT && res) + if (await_status == OperatorStatus::HAS_OUTPUT && res) { block = std::move(*res); res.reset(); diff --git a/dbms/src/Operators/UnorderedSourceOp.cpp b/dbms/src/Operators/UnorderedSourceOp.cpp index 7973278f482..beb62365a77 100644 --- a/dbms/src/Operators/UnorderedSourceOp.cpp +++ b/dbms/src/Operators/UnorderedSourceOp.cpp @@ -83,7 +83,7 @@ OperatorStatus UnorderedSourceOp::doFetchBlock() setNotifyFuture(notify_future); return OperatorStatus::WAIT_FOR_NOTIFY; } - + if (t_block) { if unlikely (t_block.rows() == 0) From 2b2b029d736485f1e6316b1bf2334fd8a5878da8 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 21:19:14 +0800 Subject: [PATCH 11/19] fix --- .../Exec/tests/gtest_simple_operator.cpp | 4 ++-- .../src/Flash/Pipeline/Schedule/Tasks/Task.cpp | 6 +++--- dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 18 +++++++++--------- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp index 398c7f72077..45b8ab09a3e 100644 --- a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp +++ b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp @@ -121,7 +121,7 @@ class SimpleOperatorTestRunner : public DB::tests::ExecutorTest }}; PipelineExecutorContext exec_context; auto op_pipeline = build(request, result_handler, exec_context); - while (op_pipeline->execute().status != OperatorStatus::FINISHED) {} + while (op_pipeline->execute() != OperatorStatus::FINISHED) {} ASSERT_COLUMNS_EQ_UR(expect_columns, vstackBlocks(std::move(blocks)).getColumnsWithTypeAndName()); } }; @@ -140,7 +140,7 @@ try PipelineExecutorContext exec_context; auto op_pipeline = build(request, result_handler, exec_context); exec_context.cancel(); - ASSERT_EQ(op_pipeline->execute().status, OperatorStatus::CANCELLED); + ASSERT_EQ(op_pipeline->execute(), OperatorStatus::CANCELLED); } CATCH diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index e575b5e32b4..4ab8a3ebf44 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -114,11 +114,11 @@ Task::~Task() } \ try \ { \ - auto return_status = (function()); \ + auto status = (function()); \ FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_task_run_failpoint); \ FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_query_run); \ - switchStatus(return_status); \ - return return_status; \ + switchStatus(status); \ + return status; \ } \ catch (...) \ { \ diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h index 364da124f05..1681eb45e1b 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -23,15 +23,15 @@ namespace DB { /** - * CANCELLED/ERROR/FINISHED - * ▲ - * │ - * ┌───────────────────────────────────────────────┐ - * │ ┌──►RUNNING◄──┐ │ - * INIT───►│ │ │ │ - * │ ▼ ▼ │ - * │ WAIT_FOR_NOTIFY/WATITING◄────────►IO_IN/OUT │ - * └───────────────────────────────────────────────┘ + * CANCELLED/ERROR/FINISHED + * ▲ + * │ + * ┌───────────────────────────────┐ + * │ ┌──►RUNNING◄──┐ │ + * INIT───►│ │ │ │ + * │ ▼ ▼ │ + * │ WATITING◄────────►IO_IN/OUT │ + * └───────────────────────────────┘ */ enum class ExecTaskStatus { From c7016339eb5097ccd819da1953370cbf6fe8348e Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 21:23:17 +0800 Subject: [PATCH 12/19] u --- dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index 4ab8a3ebf44..c7bb9592edb 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -118,7 +118,7 @@ Task::~Task() FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_pipeline_model_task_run_failpoint); \ FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_query_run); \ switchStatus(status); \ - return status; \ + return task_status; \ } \ catch (...) \ { \ From ca9d6f32e2e39a79c2f7dff685643fc3656d3f4b Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 21:28:42 +0800 Subject: [PATCH 13/19] refactor --- dbms/src/Operators/UnorderedSourceOp.cpp | 19 ++++--------------- dbms/src/Operators/UnorderedSourceOp.h | 4 ---- 2 files changed, 4 insertions(+), 19 deletions(-) diff --git a/dbms/src/Operators/UnorderedSourceOp.cpp b/dbms/src/Operators/UnorderedSourceOp.cpp index beb62365a77..8a2bba8a114 100644 --- a/dbms/src/Operators/UnorderedSourceOp.cpp +++ b/dbms/src/Operators/UnorderedSourceOp.cpp @@ -65,30 +65,19 @@ OperatorStatus UnorderedSourceOp::readImpl(Block & block) if unlikely (done) return OperatorStatus::HAS_OUTPUT; - auto status = doFetchBlock(); - if (status == OperatorStatus::HAS_OUTPUT) - std::swap(block, t_block); - return status; -} - -OperatorStatus UnorderedSourceOp::doFetchBlock() -{ - if (t_block) - return OperatorStatus::HAS_OUTPUT; - while (true) { - if (!task_pool->tryPopBlock(t_block)) + if (!task_pool->tryPopBlock(block)) { setNotifyFuture(notify_future); return OperatorStatus::WAIT_FOR_NOTIFY; } - if (t_block) + if (block) { - if unlikely (t_block.rows() == 0) + if unlikely (block.rows() == 0) { - t_block.clear(); + block.clear(); continue; } return OperatorStatus::HAS_OUTPUT; diff --git a/dbms/src/Operators/UnorderedSourceOp.h b/dbms/src/Operators/UnorderedSourceOp.h index 55f653031ef..7fefae7d6f6 100644 --- a/dbms/src/Operators/UnorderedSourceOp.h +++ b/dbms/src/Operators/UnorderedSourceOp.h @@ -71,9 +71,6 @@ class UnorderedSourceOp : public SourceOp OperatorStatus readImpl(Block & block) override; -private: - OperatorStatus doFetchBlock(); - private: DM::SegmentReadTaskPoolPtr task_pool; int64_t ref_no; @@ -84,6 +81,5 @@ class UnorderedSourceOp : public SourceOp int max_wait_time_ms; bool done = false; - Block t_block; }; } // namespace DB From f93842eca73d742bff94298437f7060f058d7fd3 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 26 Mar 2024 21:58:01 +0800 Subject: [PATCH 14/19] fiux --- dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp index 600af3f80d4..b379ef633ae 100644 --- a/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/ThreadPool/TaskThreadPool.cpp @@ -126,6 +126,7 @@ void TaskThreadPool::handleTask(TaskPtr & task) scheduler.submitToWaitReactor(std::move(task)); break; case ExecTaskStatus::WAIT_FOR_NOTIFY: + task->endTraceMemory(); registerTaskToFuture(std::move(task)); break; case FINISH_STATUS: From 4b6c36329999c29a057b109f47b5e8e88fb17138 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 27 Mar 2024 07:26:45 +0800 Subject: [PATCH 15/19] final --- .../Pipeline/Schedule/Tasks/NotifyFuture.cpp | 5 +++++ .../Pipeline/Schedule/Tasks/NotifyFuture.h | 1 + dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp | 6 ++++++ .../Storages/DeltaMerge/ReadThread/WorkQueue.h | 17 ++++++++++------- 4 files changed, 22 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp index 362f4922844..fc939e1eafa 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.cpp @@ -28,6 +28,11 @@ void setNotifyFuture(NotifyFuturePtr new_future) current_notify_future = std::move(new_future); } +void clearNotifyFuture() +{ + current_notify_future.reset(); +} + void registerTaskToFuture(TaskPtr && task) { assert(current_notify_future != nullptr); diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h index 3cf47966bff..4b30ed15389 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/NotifyFuture.h @@ -33,6 +33,7 @@ extern thread_local NotifyFuturePtr current_notify_future; #endif void setNotifyFuture(NotifyFuturePtr new_future); +void clearNotifyFuture(); void registerTaskToFuture(TaskPtr && task); } // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp index c7bb9592edb..76a0b1e22db 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -123,6 +124,7 @@ Task::~Task() catch (...) \ { \ LOG_WARNING(log, "error occurred and cancel the query"); \ + clearNotifyFuture(); \ exec_context.onErrorOccurred(std::current_exception()); \ switchStatus(ExecTaskStatus::ERROR); \ return task_status; \ @@ -130,6 +132,7 @@ Task::~Task() ExecTaskStatus Task::execute() { + assert(current_notify_future == nullptr); assert(mem_tracker_ptr == current_memory_tracker); assert(task_status == ExecTaskStatus::RUNNING); EXECUTE(executeImpl); @@ -137,6 +140,7 @@ ExecTaskStatus Task::execute() ExecTaskStatus Task::executeIO() { + assert(current_notify_future == nullptr); assert(mem_tracker_ptr == current_memory_tracker); assert(task_status == ExecTaskStatus::IO_IN || task_status == ExecTaskStatus::IO_OUT); EXECUTE(executeIOImpl); @@ -144,6 +148,7 @@ ExecTaskStatus Task::executeIO() ExecTaskStatus Task::await() { + assert(current_notify_future == nullptr); // Because await only performs polling checks and does not involve computing/memory tracker memory allocation, // await will not invoke MemoryTracker, so current_memory_tracker must be nullptr here. assert(current_memory_tracker == nullptr); @@ -161,6 +166,7 @@ void Task::notify() void Task::finalize() { + assert(current_notify_future == nullptr); // To make sure that `finalize` only called once. RUNTIME_ASSERT(!is_finalized, log, "finalize can only be called once."); is_finalized = true; diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h index 76761c48975..0f2099614f4 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h @@ -67,18 +67,21 @@ class WorkQueue void registerPipeTask(TaskPtr && task) { std::lock_guard lock(mu); - if (unlikely(done)) - { - PipeConditionVariable::notifyTaskDirectly(std::move(task)); - pipe_cv.notifyAll(); - } - else if (!queue.empty()) + if (!queue.empty()) { PipeConditionVariable::notifyTaskDirectly(std::move(task)); } else { - pipe_cv.registerTask(std::move(task)); + if (unlikely(done)) + { + PipeConditionVariable::notifyTaskDirectly(std::move(task)); + pipe_cv.notifyAll(); + } + else + { + pipe_cv.registerTask(std::move(task)); + } } } From f0c6bfdbbb5edd2305484182af250f43c32ebdc3 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 27 Mar 2024 07:29:55 +0800 Subject: [PATCH 16/19] final final --- dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h index 0f2099614f4..d8d9718bad0 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h @@ -105,7 +105,6 @@ class WorkQueue } if (done) { - pipe_cv.notifyAll(); return false; } queue.push(std::forward(item)); From a05bd14fab72db9b9bd9a25a98fa6c49fcfe5795 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 27 Mar 2024 12:45:13 +0800 Subject: [PATCH 17/19] refine for lock --- .../Schedule/Tasks/PipeConditionVariable.h | 39 ++++++++++++------- .../DeltaMerge/ReadThread/WorkQueue.h | 31 +++++++++------ 2 files changed, 44 insertions(+), 26 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h index d60146514ac..79b6ccba1ca 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipeConditionVariable.h @@ -21,7 +21,6 @@ namespace DB { -// Must have lock to use this class class PipeConditionVariable { public: @@ -29,7 +28,10 @@ class PipeConditionVariable { assert(task); assert(task->getStatus() == ExecTaskStatus::WAIT_FOR_NOTIFY); - tasks.push_back(std::move(task)); + { + std::lock_guard lock(mu); + tasks.push_back(std::move(task)); + } #if __APPLE__ && __clang__ __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); @@ -41,29 +43,37 @@ class PipeConditionVariable inline void notifyOne() { - if (!tasks.empty()) + TaskPtr task; { - auto task = std::move(tasks.front()); + std::lock_guard lock(mu); + if (tasks.empty()) + return; + task = std::move(tasks.front()); tasks.pop_front(); - notifyTaskDirectly(std::move(task)); + } + assert(task); + notifyTaskDirectly(std::move(task)); #if __APPLE__ && __clang__ - __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); + __thread auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #else - thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); + thread_local auto & metrics = GET_METRIC(tiflash_pipeline_scheduler, type_wait_for_notify_tasks_count); #endif - metrics.Decrement(); - } + metrics.Decrement(); } inline void notifyAll() { - size_t tasks_cnt = tasks.size(); - while (!tasks.empty()) + std::deque cur_tasks; { - auto task = std::move(tasks.front()); - tasks.pop_front(); - notifyTaskDirectly(std::move(task)); + std::lock_guard lock(mu); + std::swap(cur_tasks, tasks); + } + size_t tasks_cnt = cur_tasks.size(); + while (!cur_tasks.empty()) + { + notifyTaskDirectly(std::move(cur_tasks.front())); + cur_tasks.pop_front(); } #if __APPLE__ && __clang__ @@ -84,6 +94,7 @@ class PipeConditionVariable } private: + std::mutex mu; std::deque tasks; }; } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h index d8d9718bad0..98adc4839fc 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h @@ -66,22 +66,29 @@ class WorkQueue void registerPipeTask(TaskPtr && task) { - std::lock_guard lock(mu); - if (!queue.empty()) + bool has_item{false}; + bool is_done{false}; + { + std::lock_guard lock(mu); + has_item = !queue.empty(); + is_done = done; + } + if (has_item || (unlikely(is_done))) { PipeConditionVariable::notifyTaskDirectly(std::move(task)); } else { - if (unlikely(done)) - { - PipeConditionVariable::notifyTaskDirectly(std::move(task)); - pipe_cv.notifyAll(); - } - else + // double check for the last case. { - pipe_cv.registerTask(std::move(task)); + std::lock_guard lock(mu); + if (queue.empty() && !done) + { + pipe_cv.registerTask(std::move(task)); + return; + } } + PipeConditionVariable::notifyTaskDirectly(std::move(task)); } } @@ -113,8 +120,8 @@ class WorkQueue { *size = queue.size(); } - pipe_cv.notifyOne(); } + pipe_cv.notifyOne(); reader_cv.notify_one(); return true; } @@ -161,7 +168,7 @@ class WorkQueue bool tryPop(T & item) { { - std::unique_lock lock(mu); + std::lock_guard lock(mu); ++pop_times; if (queue.empty()) { @@ -206,8 +213,8 @@ class WorkQueue std::lock_guard lock(mu); assert(!done); done = true; - pipe_cv.notifyAll(); } + pipe_cv.notifyAll(); reader_cv.notify_all(); writer_cv.notify_all(); finish_cv.notify_all(); From 0ba130f3b6769e3a6cd1bf028b7aa12651be4582 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 27 Mar 2024 15:24:01 +0800 Subject: [PATCH 18/19] address comment --- dbms/src/Operators/UnorderedSourceOp.cpp | 26 +------------------ dbms/src/Operators/UnorderedSourceOp.h | 4 --- .../Storages/DeltaMerge/SegmentReadTaskPool.h | 7 +++-- 3 files changed, 6 insertions(+), 31 deletions(-) diff --git a/dbms/src/Operators/UnorderedSourceOp.cpp b/dbms/src/Operators/UnorderedSourceOp.cpp index 8a2bba8a114..074db28a43c 100644 --- a/dbms/src/Operators/UnorderedSourceOp.cpp +++ b/dbms/src/Operators/UnorderedSourceOp.cpp @@ -13,34 +13,11 @@ // limitations under the License. #include -#include #include #include -#include - namespace DB { -namespace -{ -class ScanNotifyFuture : public NotifyFuture -{ -public: - explicit ScanNotifyFuture(const DM::SegmentReadTaskPoolPtr & task_pool_) - : task_pool(task_pool_) - { - assert(task_pool); - } - - ~ScanNotifyFuture() override = default; - - void registerTask(TaskPtr && task) override { task_pool->registerPipeTask(std::move(task)); } - -private: - DM::SegmentReadTaskPoolPtr task_pool; -}; -}; // namespace - UnorderedSourceOp::UnorderedSourceOp( PipelineExecutorContext & exec_context_, const DM::SegmentReadTaskPoolPtr & task_pool_, @@ -52,7 +29,6 @@ UnorderedSourceOp::UnorderedSourceOp( : SourceOp(exec_context_, req_id) , task_pool(task_pool_) , ref_no(0) - , notify_future(std::make_shared(task_pool_)) , waiting_rf_list(runtime_filter_list_) , max_wait_time_ms(max_wait_time_ms_) { @@ -69,7 +45,7 @@ OperatorStatus UnorderedSourceOp::readImpl(Block & block) { if (!task_pool->tryPopBlock(block)) { - setNotifyFuture(notify_future); + setNotifyFuture(task_pool); return OperatorStatus::WAIT_FOR_NOTIFY; } diff --git a/dbms/src/Operators/UnorderedSourceOp.h b/dbms/src/Operators/UnorderedSourceOp.h index 7fefae7d6f6..65f0bb68550 100644 --- a/dbms/src/Operators/UnorderedSourceOp.h +++ b/dbms/src/Operators/UnorderedSourceOp.h @@ -23,9 +23,6 @@ namespace DB { -struct NotifyFuture; -using NotifyFuturePtr = std::shared_ptr; - /// Read blocks asyncly from Storage Layer by using read thread, /// The result can not guarantee the keep_order property class UnorderedSourceOp : public SourceOp @@ -74,7 +71,6 @@ class UnorderedSourceOp : public SourceOp private: DM::SegmentReadTaskPoolPtr task_pool; int64_t ref_no; - NotifyFuturePtr notify_future; // runtime filter RuntimeFilteList waiting_rf_list; diff --git a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h index b45322e3224..dfcddc1eed4 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.h @@ -14,6 +14,7 @@ #pragma once #include +#include #include #include #include @@ -97,7 +98,9 @@ class SegmentReadTasksWrapper std::unordered_map unordered_tasks; }; -class SegmentReadTaskPool : private boost::noncopyable +class SegmentReadTaskPool + : public NotifyFuture + , private boost::noncopyable { public: SegmentReadTaskPool( @@ -167,7 +170,7 @@ class SegmentReadTaskPool : private boost::noncopyable std::once_flag & addToSchedulerFlag() { return add_to_scheduler; } - void registerPipeTask(TaskPtr && task) { q.registerPipeTask(std::move(task)); } + void registerTask(TaskPtr && task) override { q.registerPipeTask(std::move(task)); } public: const uint64_t pool_id; From 20d297dd23f4cc7277d84c6a4b86d29ebfdb5e6a Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 27 Mar 2024 18:24:31 +0800 Subject: [PATCH 19/19] Update dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h Co-authored-by: Liqi Geng --- .../DeltaMerge/ReadThread/WorkQueue.h | 23 ++++--------------- 1 file changed, 4 insertions(+), 19 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h index 98adc4839fc..039bb9f4ffb 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/WorkQueue.h @@ -66,30 +66,15 @@ class WorkQueue void registerPipeTask(TaskPtr && task) { - bool has_item{false}; - bool is_done{false}; { std::lock_guard lock(mu); - has_item = !queue.empty(); - is_done = done; - } - if (has_item || (unlikely(is_done))) - { - PipeConditionVariable::notifyTaskDirectly(std::move(task)); - } - else - { - // double check for the last case. + if (queue.empty() && !done) { - std::lock_guard lock(mu); - if (queue.empty() && !done) - { - pipe_cv.registerTask(std::move(task)); - return; - } + pipe_cv.registerTask(std::move(task)); + return; } - PipeConditionVariable::notifyTaskDirectly(std::move(task)); } + PipeConditionVariable::notifyTaskDirectly(std::move(task)); } /**