From 90f409c71e8813df94ef648df629d8d4e76cbbfe Mon Sep 17 00:00:00 2001 From: SeaRise Date: Sun, 29 Jan 2023 16:17:53 +0800 Subject: [PATCH] pipeline: Support pipeline model framework and simple operator (#6520) ref pingcap/tiflash#6518 --- dbms/CMakeLists.txt | 3 +- dbms/src/CMakeLists.txt | 3 +- .../src/DataStreams/LimitBlockInputStream.cpp | 9 +- dbms/src/DataStreams/LimitBlockInputStream.h | 4 +- dbms/src/DataStreams/LimitTransformAction.cpp | 68 +- dbms/src/DataStreams/LimitTransformAction.h | 48 +- .../gtest_squashing_hash_join_transform.cpp | 2 +- dbms/src/Flash/CMakeLists.txt | 16 +- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 4 +- .../src/Flash/Executor/DataStreamExecutor.cpp | 4 +- dbms/src/Flash/Executor/DataStreamExecutor.h | 4 +- dbms/src/Flash/Executor/ExecutionResult.h | 43 + dbms/src/Flash/Executor/PipelineExecutor.cpp | 83 ++ dbms/src/Flash/Executor/PipelineExecutor.h | 72 ++ .../Flash/Executor/PipelineExecutorStatus.cpp | 72 ++ .../Flash/Executor/PipelineExecutorStatus.h | 72 ++ dbms/src/Flash/Executor/QueryExecutor.h | 28 +- dbms/src/Flash/Executor/tests/CMakeLists.txt | 15 + .../tests/gtest_pipeline_executor_status.cpp | 84 ++ dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp | 121 ++ dbms/src/Flash/Pipeline/Exec/PipelineExec.h | 65 ++ .../Pipeline/Exec/PipelineExecBuilder.cpp | 84 ++ .../Flash/Pipeline/Exec/PipelineExecBuilder.h | 63 + .../Flash/Pipeline/Exec/tests/CMakeLists.txt | 15 + .../Exec/tests/gtest_simple_operator.cpp | 187 +++ dbms/src/Flash/Pipeline/Pipeline.cpp | 147 +++ dbms/src/Flash/Pipeline/Pipeline.h | 78 ++ dbms/src/Flash/Pipeline/PipelineBuilder.h | 102 ++ .../Flash/Pipeline/Schedule/Events/Event.cpp | 131 +++ .../Flash/Pipeline/Schedule/Events/Event.h | 91 ++ .../Pipeline/Schedule/Events/PipelineEvent.h | 49 + .../Schedule/Events/PlainPipelineEvent.cpp | 33 + .../Schedule/Events/PlainPipelineEvent.h | 40 + .../Schedule/Events/tests/CMakeLists.txt | 15 + .../Schedule/Events/tests/gtest_event.cpp | 519 +++++++++ .../Schedule/TaskQueues/FiFOTaskQueue.cpp | 80 ++ .../Schedule/TaskQueues/FiFOTaskQueue.h | 46 + .../Pipeline/Schedule/TaskQueues/TaskQueue.h | 45 + .../Schedule/TaskQueues/tests/CMakeLists.txt | 15 + .../Schedule/TaskQueues/tests/gtest_fifo.cpp | 83 ++ .../Flash/Pipeline/Schedule/TaskScheduler.cpp | 74 ++ .../Flash/Pipeline/Schedule/TaskScheduler.h | 68 ++ .../Pipeline/Schedule/TaskThreadPool.cpp | 108 ++ .../Flash/Pipeline/Schedule/TaskThreadPool.h | 55 + .../Pipeline/Schedule/Tasks/EventTask.cpp | 47 + .../Flash/Pipeline/Schedule/Tasks/EventTask.h | 79 ++ .../Pipeline/Schedule/Tasks/PipelineTask.cpp | 90 ++ .../Pipeline/Schedule/Tasks/PipelineTask.h | 41 + dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h | 74 ++ .../Pipeline/Schedule/Tasks/TaskHelper.h | 45 + .../Flash/Pipeline/Schedule/WaitReactor.cpp | 160 +++ .../src/Flash/Pipeline/Schedule/WaitReactor.h | 53 + .../Pipeline/Schedule/WaitingTaskList.cpp | 78 ++ .../Flash/Pipeline/Schedule/WaitingTaskList.h | 45 + .../Pipeline/Schedule/tests/CMakeLists.txt | 15 + .../Schedule/tests/gtest_task_scheduler.cpp | 282 +++++ dbms/src/Flash/Pipeline/tests/CMakeLists.txt | 15 + .../Flash/Pipeline/tests/gtest_pipeline.cpp | 88 ++ dbms/src/Flash/Planner/PhysicalPlan.cpp | 59 +- dbms/src/Flash/Planner/PhysicalPlan.h | 14 +- dbms/src/Flash/Planner/PhysicalPlanNode.cpp | 26 +- dbms/src/Flash/Planner/PhysicalPlanNode.h | 22 +- dbms/src/Flash/Planner/PlanType.h | 3 +- dbms/src/Flash/Planner/Planner.cpp | 4 +- .../{plans => Plans}/PhysicalAggregation.cpp | 23 +- .../{plans => Plans}/PhysicalAggregation.h | 8 +- .../Planner/{plans => Plans}/PhysicalBinary.h | 11 +- .../PhysicalExchangeReceiver.cpp | 6 +- .../PhysicalExchangeReceiver.h | 6 +- .../PhysicalExchangeSender.cpp | 8 +- .../{plans => Plans}/PhysicalExchangeSender.h | 6 +- .../{plans => Plans}/PhysicalFilter.cpp | 18 +- .../Planner/{plans => Plans}/PhysicalFilter.h | 8 +- .../Planner/Plans/PhysicalGetResultSink.cpp | 34 + .../Planner/Plans/PhysicalGetResultSink.h | 67 ++ .../Planner/{plans => Plans}/PhysicalJoin.cpp | 40 +- .../Planner/{plans => Plans}/PhysicalJoin.h | 8 +- .../Planner/{plans => Plans}/PhysicalLeaf.h | 7 +- .../{plans => Plans}/PhysicalLimit.cpp | 20 +- .../Planner/{plans => Plans}/PhysicalLimit.h | 8 +- .../PhysicalMockExchangeReceiver.cpp | 17 +- .../PhysicalMockExchangeReceiver.h | 8 +- .../PhysicalMockExchangeSender.cpp | 8 +- .../PhysicalMockExchangeSender.h | 8 +- .../PhysicalMockTableScan.cpp | 17 +- .../{plans => Plans}/PhysicalMockTableScan.h | 8 +- .../{plans => Plans}/PhysicalProjection.cpp | 20 +- .../{plans => Plans}/PhysicalProjection.h | 8 +- .../{plans => Plans}/PhysicalTableScan.cpp | 6 +- .../{plans => Plans}/PhysicalTableScan.h | 6 +- .../Planner/{plans => Plans}/PhysicalTopN.cpp | 8 +- .../Planner/{plans => Plans}/PhysicalTopN.h | 6 +- .../Planner/{plans => Plans}/PhysicalUnary.h | 10 +- .../{plans => Plans}/PhysicalWindow.cpp | 8 +- .../Planner/{plans => Plans}/PhysicalWindow.h | 6 +- .../{plans => Plans}/PhysicalWindowSort.cpp | 8 +- .../{plans => Plans}/PhysicalWindowSort.h | 6 +- .../Planner/tests/gtest_physical_plan.cpp | 4 +- dbms/src/Flash/executeQuery.cpp | 57 +- dbms/src/Flash/executeQuery.h | 10 +- .../Flash/tests/gtest_executors_with_dm.cpp | 7 +- .../src/Flash/tests/gtest_filter_executor.cpp | 29 +- dbms/src/Flash/tests/gtest_limit_executor.cpp | 27 +- .../tests/gtest_pipeline_interpreter.cpp | 1034 +++++++++++++++++ .../Flash/tests/gtest_planner_interpreter.cpp | 3 +- .../Flash/tests/gtest_projection_executor.cpp | 27 +- dbms/src/Interpreters/Settings.h | 6 +- .../Operators/BlockInputStreamSourceOp.cpp | 41 + dbms/src/Operators/BlockInputStreamSourceOp.h | 43 + dbms/src/Operators/CMakeLists.txt | 17 + dbms/src/Operators/ExpressionTransformOp.cpp | 31 + dbms/src/Operators/ExpressionTransformOp.h | 48 + dbms/src/Operators/FilterTransformOp.cpp | 39 + dbms/src/Operators/FilterTransformOp.h | 48 + dbms/src/Operators/GetResultSinkOp.cpp | 29 + dbms/src/Operators/GetResultSinkOp.h | 43 + dbms/src/Operators/LimitTransformOp.cpp | 31 + dbms/src/Operators/LimitTransformOp.h | 48 + dbms/src/Operators/Operator.cpp | 103 ++ dbms/src/Operators/Operator.h | 127 ++ dbms/src/Operators/OperatorHelper.cpp | 44 + dbms/src/Operators/OperatorHelper.h | 24 + dbms/src/Operators/tests/CMakeLists.txt | 15 + dbms/src/Server/Server.cpp | 22 +- dbms/src/TestUtils/ExecutorTestUtils.cpp | 34 +- dbms/src/TestUtils/ExecutorTestUtils.h | 34 +- dbms/src/TestUtils/MPPTaskTestUtils.cpp | 3 +- .../WindowFunctions/tests/gtest_lead_lag.cpp | 6 +- 128 files changed, 6356 insertions(+), 302 deletions(-) rename dbms/src/{Flash => DataStreams}/tests/gtest_squashing_hash_join_transform.cpp (98%) create mode 100644 dbms/src/Flash/Executor/ExecutionResult.h create mode 100644 dbms/src/Flash/Executor/PipelineExecutor.cpp create mode 100644 dbms/src/Flash/Executor/PipelineExecutor.h create mode 100644 dbms/src/Flash/Executor/PipelineExecutorStatus.cpp create mode 100644 dbms/src/Flash/Executor/PipelineExecutorStatus.h create mode 100644 dbms/src/Flash/Executor/tests/CMakeLists.txt create mode 100644 dbms/src/Flash/Executor/tests/gtest_pipeline_executor_status.cpp create mode 100644 dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp create mode 100644 dbms/src/Flash/Pipeline/Exec/PipelineExec.h create mode 100644 dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.cpp create mode 100644 dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.h create mode 100644 dbms/src/Flash/Pipeline/Exec/tests/CMakeLists.txt create mode 100644 dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp create mode 100644 dbms/src/Flash/Pipeline/Pipeline.cpp create mode 100644 dbms/src/Flash/Pipeline/Pipeline.h create mode 100644 dbms/src/Flash/Pipeline/PipelineBuilder.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/Event.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/Event.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/PipelineEvent.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/tests/CMakeLists.txt create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskQueues/FiFOTaskQueue.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskQueues/FiFOTaskQueue.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskQueues/TaskQueue.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/CMakeLists.txt create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_fifo.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskScheduler.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskScheduler.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskThreadPool.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/TaskThreadPool.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/Tasks/TaskHelper.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/WaitReactor.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/WaitReactor.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/WaitingTaskList.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/WaitingTaskList.h create mode 100644 dbms/src/Flash/Pipeline/Schedule/tests/CMakeLists.txt create mode 100644 dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp create mode 100644 dbms/src/Flash/Pipeline/tests/CMakeLists.txt create mode 100644 dbms/src/Flash/Pipeline/tests/gtest_pipeline.cpp rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalAggregation.cpp (89%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalAggregation.h (90%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalBinary.h (77%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalExchangeReceiver.cpp (94%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalExchangeReceiver.h (88%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalExchangeSender.cpp (92%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalExchangeSender.h (91%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalFilter.cpp (77%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalFilter.h (83%) create mode 100644 dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.cpp create mode 100644 dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.h rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalJoin.cpp (90%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalJoin.h (91%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalLeaf.h (85%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalLimit.cpp (68%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalLimit.h (80%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalMockExchangeReceiver.cpp (78%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalMockExchangeReceiver.h (84%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalMockExchangeSender.cpp (85%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalMockExchangeSender.h (80%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalMockTableScan.cpp (88%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalMockTableScan.h (86%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalProjection.cpp (88%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalProjection.h (88%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalTableScan.cpp (95%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalTableScan.h (89%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalTopN.cpp (91%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalTopN.h (90%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalUnary.h (76%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalWindow.cpp (93%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalWindow.h (90%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalWindowSort.cpp (89%) rename dbms/src/Flash/Planner/{plans => Plans}/PhysicalWindowSort.h (90%) create mode 100644 dbms/src/Flash/tests/gtest_pipeline_interpreter.cpp create mode 100644 dbms/src/Operators/BlockInputStreamSourceOp.cpp create mode 100644 dbms/src/Operators/BlockInputStreamSourceOp.h create mode 100644 dbms/src/Operators/CMakeLists.txt create mode 100644 dbms/src/Operators/ExpressionTransformOp.cpp create mode 100644 dbms/src/Operators/ExpressionTransformOp.h create mode 100644 dbms/src/Operators/FilterTransformOp.cpp create mode 100644 dbms/src/Operators/FilterTransformOp.h create mode 100644 dbms/src/Operators/GetResultSinkOp.cpp create mode 100644 dbms/src/Operators/GetResultSinkOp.h create mode 100644 dbms/src/Operators/LimitTransformOp.cpp create mode 100644 dbms/src/Operators/LimitTransformOp.h create mode 100644 dbms/src/Operators/Operator.cpp create mode 100644 dbms/src/Operators/Operator.h create mode 100644 dbms/src/Operators/OperatorHelper.cpp create mode 100644 dbms/src/Operators/OperatorHelper.h create mode 100644 dbms/src/Operators/tests/CMakeLists.txt diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index be417c0a26e..f993aead814 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -1,4 +1,4 @@ -# Copyright 2022 PingCAP, Ltd. +# Copyright 2023 PingCAP, Ltd. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -62,6 +62,7 @@ add_headers_and_sources(clickhouse_common_io src/IO) add_headers_and_sources(dbms src/Analyzers) add_headers_and_sources(dbms src/Core) add_headers_and_sources(dbms src/DataStreams) +add_headers_and_sources(dbms src/Operators) add_headers_and_sources(dbms src/DataTypes) add_headers_and_sources(dbms src/Databases) add_headers_and_sources(dbms src/Debug) diff --git a/dbms/src/CMakeLists.txt b/dbms/src/CMakeLists.txt index f7ebf51f38a..058c15d7cc5 100644 --- a/dbms/src/CMakeLists.txt +++ b/dbms/src/CMakeLists.txt @@ -1,4 +1,4 @@ -# Copyright 2022 PingCAP, Ltd. +# Copyright 2023 PingCAP, Ltd. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ add_subdirectory (Columns) add_subdirectory (Common) add_subdirectory (Core) add_subdirectory (DataStreams) +add_subdirectory (Operators) add_subdirectory (DataTypes) add_subdirectory (Dictionaries) add_subdirectory (Storages) diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index ae9bfecffd0..13ed3d25929 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -24,8 +24,7 @@ LimitBlockInputStream::LimitBlockInputStream( size_t limit_, const String & req_id) : log(Logger::get(req_id)) - , limit_transform_action(input->getHeader(), limit_) - + , action(input->getHeader(), limit_) { children.push_back(input); } @@ -35,7 +34,7 @@ Block LimitBlockInputStream::readImpl() { Block res = children.back()->read(); - if (limit_transform_action.transform(res)) + if (action.transform(res)) { return res; } @@ -47,6 +46,6 @@ Block LimitBlockInputStream::readImpl() void LimitBlockInputStream::appendInfo(FmtBuffer & buffer) const { - buffer.fmtAppend(", limit = {}", limit_transform_action.getLimit()); + buffer.fmtAppend(", limit = {}", action.getLimit()); } } // namespace DB diff --git a/dbms/src/DataStreams/LimitBlockInputStream.h b/dbms/src/DataStreams/LimitBlockInputStream.h index d90cb19ca4d..c749ef30800 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.h +++ b/dbms/src/DataStreams/LimitBlockInputStream.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -46,7 +46,7 @@ class LimitBlockInputStream : public IProfilingBlockInputStream private: LoggerPtr log; - LimitTransformAction limit_transform_action; + LocalLimitTransformAction action; }; } // namespace DB diff --git a/dbms/src/DataStreams/LimitTransformAction.cpp b/dbms/src/DataStreams/LimitTransformAction.cpp index 7542d49e859..86343477add 100644 --- a/dbms/src/DataStreams/LimitTransformAction.cpp +++ b/dbms/src/DataStreams/LimitTransformAction.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -17,51 +17,57 @@ namespace DB { -LimitTransformAction::LimitTransformAction( - const Block & header_, - size_t limit_) - : header(header_) - , limit(limit_) +namespace { -} - -Block LimitTransformAction::getHeader() const +// Removes all rows outside of specified range of Block. +void cut(Block & block, size_t rows, size_t limit, size_t pos) { - return header; + assert(rows + limit > pos); + size_t pop_back_cnt = pos - limit; + for (auto & col : block) + { + auto mutate_col = (*std::move(col.column)).mutate(); + mutate_col->popBack(pop_back_cnt); + col.column = std::move(mutate_col); + } } +} // namespace -size_t LimitTransformAction::getLimit() const +bool LocalLimitTransformAction::transform(Block & block) { - return limit; + if (unlikely(!block)) + return true; + + /// pos - how many lines were read, including the last read block + if (pos >= limit) + return false; + + auto rows = block.rows(); + pos += rows; + if (pos > limit) + cut(block, rows, limit, pos); + // for pos <= limit, give away the whole block + return true; } -bool LimitTransformAction::transform(Block & block) +bool GlobalLimitTransformAction::transform(Block & block) { if (unlikely(!block)) return true; /// pos - how many lines were read, including the last read block if (pos >= limit) - { return false; - } auto rows = block.rows(); - pos += rows; - if (pos >= rows && pos <= limit) - { - // give away the whole block - return true; - } - else - { - // pos > limit - // give away a piece of the block - assert(rows + limit > pos); - size_t length = rows + limit - pos; - for (size_t i = 0; i < block.columns(); ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->cut(0, length); - return true; - } + size_t prev_pos = pos.fetch_add(rows); + if (prev_pos >= limit) + return false; + + size_t cur_pos = prev_pos + rows; + if (cur_pos > limit) + cut(block, rows, limit, cur_pos); + // for pos <= limit, give away the whole block + return true; } } // namespace DB diff --git a/dbms/src/DataStreams/LimitTransformAction.h b/dbms/src/DataStreams/LimitTransformAction.h index 36defe52602..e158f826c4a 100644 --- a/dbms/src/DataStreams/LimitTransformAction.h +++ b/dbms/src/DataStreams/LimitTransformAction.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,22 +15,54 @@ #include +#include +#include + namespace DB { -struct LimitTransformAction +struct LocalLimitTransformAction { public: - LimitTransformAction( + LocalLimitTransformAction( const Block & header_, - size_t limit_); + size_t limit_) + : header(header_) + , limit(limit_) + { + } bool transform(Block & block); - Block getHeader() const; - size_t getLimit() const; + + Block getHeader() const { return header; } + size_t getLimit() const { return limit; } private: - Block header; - size_t limit; + const Block header; + const size_t limit; size_t pos = 0; }; + +struct GlobalLimitTransformAction +{ +public: + GlobalLimitTransformAction( + const Block & header_, + size_t limit_) + : header(header_) + , limit(limit_) + { + } + + bool transform(Block & block); + + Block getHeader() const { return header; } + size_t getLimit() const { return limit; } + +private: + const Block header; + const size_t limit; + std::atomic_size_t pos{0}; +}; + +using GlobalLimitPtr = std::shared_ptr; } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_squashing_hash_join_transform.cpp b/dbms/src/DataStreams/tests/gtest_squashing_hash_join_transform.cpp similarity index 98% rename from dbms/src/Flash/tests/gtest_squashing_hash_join_transform.cpp rename to dbms/src/DataStreams/tests/gtest_squashing_hash_join_transform.cpp index 87fa98558bb..7e38db3f60e 100644 --- a/dbms/src/Flash/tests/gtest_squashing_hash_join_transform.cpp +++ b/dbms/src/DataStreams/tests/gtest_squashing_hash_join_transform.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/dbms/src/Flash/CMakeLists.txt b/dbms/src/Flash/CMakeLists.txt index 9e2e6b94f37..06bdb34f194 100644 --- a/dbms/src/Flash/CMakeLists.txt +++ b/dbms/src/Flash/CMakeLists.txt @@ -1,4 +1,4 @@ -# Copyright 2022 PingCAP, Ltd. +# Copyright 2023 PingCAP, Ltd. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -18,8 +18,14 @@ add_headers_and_sources(flash_service .) add_headers_and_sources(flash_service ./Coprocessor) add_headers_and_sources(flash_service ./Mpp) add_headers_and_sources(flash_service ./Executor) +add_headers_and_sources(flash_service ./Pipeline) +add_headers_and_sources(flash_service ./Pipeline/Exec) +add_headers_and_sources(flash_service ./Pipeline/Schedule) +add_headers_and_sources(flash_service ./Pipeline/Schedule/Events) +add_headers_and_sources(flash_service ./Pipeline/Schedule/Tasks) +add_headers_and_sources(flash_service ./Pipeline/Schedule/TaskQueues) add_headers_and_sources(flash_service ./Planner) -add_headers_and_sources(flash_service ./Planner/plans) +add_headers_and_sources(flash_service ./Planner/Plans) add_headers_and_sources(flash_service ./Statistics) add_headers_and_sources(flash_service ./Management) @@ -28,6 +34,12 @@ target_link_libraries(flash_service dbms) if (ENABLE_TESTS) add_subdirectory(Coprocessor/tests) + add_subdirectory(Executor/tests) + add_subdirectory(Pipeline/tests) + add_subdirectory(Pipeline/Exec/tests) + add_subdirectory(Pipeline/Schedule/tests) + add_subdirectory(Pipeline/Schedule/Events/tests) + add_subdirectory(Pipeline/Schedule/TaskQueues/tests) add_subdirectory(Planner/tests) add_subdirectory(tests) endif () diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index f181f5329ca..2b7ae2d6d1c 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -92,7 +92,7 @@ try auto start_time = Clock::now(); DAGContext & dag_context = *context.getDAGContext(); - BlockIO streams = executeQuery(context, internal); + BlockIO streams = executeAsBlockIO(context, internal); if (!streams.in || streams.out) // Only query is allowed, so streams.in must not be null and streams.out must be null throw TiFlashException("DAG is not query.", Errors::Coprocessor::Internal); diff --git a/dbms/src/Flash/Executor/DataStreamExecutor.cpp b/dbms/src/Flash/Executor/DataStreamExecutor.cpp index eee368b31b8..00069ff6e9d 100644 --- a/dbms/src/Flash/Executor/DataStreamExecutor.cpp +++ b/dbms/src/Flash/Executor/DataStreamExecutor.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -48,7 +48,7 @@ void DataStreamExecutor::cancel() p_stream->cancel(/*kill=*/false); } -String DataStreamExecutor::dump() const +String DataStreamExecutor::toString() const { FmtBuffer fb; data_stream->dumpTree(fb); diff --git a/dbms/src/Flash/Executor/DataStreamExecutor.h b/dbms/src/Flash/Executor/DataStreamExecutor.h index f9963c848fb..fdcc8431d1a 100644 --- a/dbms/src/Flash/Executor/DataStreamExecutor.h +++ b/dbms/src/Flash/Executor/DataStreamExecutor.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -30,7 +30,7 @@ class DataStreamExecutor : public QueryExecutor assert(data_stream); } - String dump() const override; + String toString() const override; void cancel() override; diff --git a/dbms/src/Flash/Executor/ExecutionResult.h b/dbms/src/Flash/Executor/ExecutionResult.h new file mode 100644 index 00000000000..0cc40e92cf3 --- /dev/null +++ b/dbms/src/Flash/Executor/ExecutionResult.h @@ -0,0 +1,43 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +struct ExecutionResult +{ + bool is_success; + String err_msg; + + void verify() + { + RUNTIME_CHECK(is_success, err_msg); + } + + static ExecutionResult success() + { + return {true, ""}; + } + + static ExecutionResult fail(const String & err_msg) + { + RUNTIME_CHECK(!err_msg.empty()); + return {false, err_msg}; + } +}; +} // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutor.cpp b/dbms/src/Flash/Executor/PipelineExecutor.cpp new file mode 100644 index 00000000000..5fda4fb5605 --- /dev/null +++ b/dbms/src/Flash/Executor/PipelineExecutor.cpp @@ -0,0 +1,83 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include + +namespace DB +{ +PipelineExecutor::PipelineExecutor( + const ProcessListEntryPtr & process_list_entry_, + Context & context_, + const PipelinePtr & root_pipeline_) + : QueryExecutor(process_list_entry_) + , context(context_) + , root_pipeline(root_pipeline_) +{ + assert(root_pipeline); +} + +ExecutionResult PipelineExecutor::execute(ResultHandler result_handler) +{ + assert(root_pipeline); + // for result_handler.isIgnored(), the sink plan of root_pipeline must be nullptr. + // Now only used by unit tests. + if (unlikely(!result_handler.isIgnored())) + root_pipeline->addGetResultSink(result_handler); + + { + auto events = root_pipeline->toEvents(status, context, context.getMaxStreams()); + Events without_input_events; + for (const auto & event : events) + { + if (event->withoutInput()) + without_input_events.push_back(event); + } + for (const auto & event : without_input_events) + event->schedule(); + } + + if (unlikely(context.isTest())) + { + // In test mode, a single query should take no more than 15 seconds to execute. + std::chrono::seconds timeout(15); + status.waitFor(timeout); + } + else + { + status.wait(); + } + return status.toExecutionResult(); +} + +void PipelineExecutor::cancel() +{ + status.cancel(); +} + +String PipelineExecutor::toString() const +{ + assert(root_pipeline); + FmtBuffer buffer; + root_pipeline->toTreeString(buffer); + return buffer.toString(); +} + +int PipelineExecutor::estimateNewThreadCount() +{ + return 0; +} +} // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutor.h b/dbms/src/Flash/Executor/PipelineExecutor.h new file mode 100644 index 00000000000..5fff4fdd1c3 --- /dev/null +++ b/dbms/src/Flash/Executor/PipelineExecutor.h @@ -0,0 +1,72 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class Context; + +class Pipeline; +using PipelinePtr = std::shared_ptr; +using Pipelines = std::vector; + +/** + * PipelineExecutor is the implementation of the pipeline-based execution model. + * + * ┌─────────────────────┐ + * │ task scheduler │ + * generate submit tasks │ │ + * pipeline ────────► event1 ─────────────► │ │ + * │ ┌────────────────┐ │ + * │ trigger │ │task thread pool│ │ + * ▼ │ └──────▲──┬──────┘ │ + * submit tasks │ │ │ │ + * event2 ─────────────► │ ┌────┴──▼────┐ │ + * │ │wait reactor│ │ + * │ └────────────┘ │ + * │ │ + * └─────────────────────┘ + * + * As shown above, the pipeline generates a number of events, which are executed in dependency order, + * and the events generate a number of tasks that will be submitted to the TaskScheduler for execution. + */ +class PipelineExecutor : public QueryExecutor +{ +public: + PipelineExecutor( + const ProcessListEntryPtr & process_list_entry_, + Context & context_, + const PipelinePtr & root_pipeline_); + + String toString() const override; + + void cancel() override; + + int estimateNewThreadCount() override; + +protected: + ExecutionResult execute(ResultHandler result_handler) override; + +private: + Context & context; + + PipelinePtr root_pipeline; + + PipelineExecutorStatus status; +}; +} // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutorStatus.cpp b/dbms/src/Flash/Executor/PipelineExecutorStatus.cpp new file mode 100644 index 00000000000..2667c07fa04 --- /dev/null +++ b/dbms/src/Flash/Executor/PipelineExecutorStatus.cpp @@ -0,0 +1,72 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +namespace DB +{ +ExecutionResult PipelineExecutorStatus::toExecutionResult() +{ + auto get_err_msg = getErrMsg(); + return get_err_msg.empty() + ? ExecutionResult::success() + : ExecutionResult::fail(get_err_msg); +} + +String PipelineExecutorStatus::getErrMsg() +{ + std::lock_guard lock(mu); + return err_msg; +} + +void PipelineExecutorStatus::onErrorOccurred(String && err_msg_) +{ + { + std::lock_guard lock(mu); + if (!err_msg.empty()) + return; + err_msg = err_msg_.empty() ? empty_err_msg : std::move(err_msg_); + } + cancel(); +} + +void PipelineExecutorStatus::wait() +{ + std::unique_lock lock(mu); + cv.wait(lock, [&] { return 0 == active_event_count; }); +} + +void PipelineExecutorStatus::onEventSchedule() +{ + std::lock_guard lock(mu); + ++active_event_count; +} + +void PipelineExecutorStatus::onEventFinish() +{ + bool notify = false; + { + std::lock_guard lock(mu); + notify = (0 == --active_event_count); + } + if (notify) + cv.notify_all(); +} + +void PipelineExecutorStatus::cancel() +{ + is_cancelled.store(true, std::memory_order_release); +} +} // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutorStatus.h b/dbms/src/Flash/Executor/PipelineExecutorStatus.h new file mode 100644 index 00000000000..72938a0d3e4 --- /dev/null +++ b/dbms/src/Flash/Executor/PipelineExecutorStatus.h @@ -0,0 +1,72 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include + +namespace DB +{ +class PipelineExecutorStatus : private boost::noncopyable +{ +public: + static constexpr auto empty_err_msg = "error without err msg"; + static constexpr auto timeout_err_msg = "error with timeout"; + + ExecutionResult toExecutionResult(); + + String getErrMsg(); + + void onEventSchedule(); + + void onEventFinish(); + + void onErrorOccurred(String && err_msg_); + + void wait(); + + template + void waitFor(const Duration & timeout_duration) + { + bool is_timeout = false; + { + std::unique_lock lock(mu); + is_timeout = !cv.wait_for(lock, timeout_duration, [&] { return 0 == active_event_count; }); + } + if (is_timeout) + { + onErrorOccurred(timeout_err_msg); + throw Exception(timeout_err_msg); + } + } + + void cancel(); + + bool isCancelled() + { + return is_cancelled.load(std::memory_order_acquire); + } + +private: + std::mutex mu; + std::condition_variable cv; + String err_msg; + UInt32 active_event_count{0}; + + std::atomic_bool is_cancelled{false}; +}; +} // namespace DB diff --git a/dbms/src/Flash/Executor/QueryExecutor.h b/dbms/src/Flash/Executor/QueryExecutor.h index b35dc0d8116..c1295a04298 100644 --- a/dbms/src/Flash/Executor/QueryExecutor.h +++ b/dbms/src/Flash/Executor/QueryExecutor.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,36 +14,14 @@ #pragma once +#include #include #include #include -#include namespace DB { -struct ExecutionResult -{ - bool is_success; - String err_msg; - - void verify() - { - RUNTIME_CHECK(is_success, err_msg); - } - - static ExecutionResult success() - { - return {true, ""}; - } - - static ExecutionResult fail(const String & err_msg) - { - RUNTIME_CHECK(!err_msg.empty()); - return {false, err_msg}; - } -}; - class ProcessListEntry; using ProcessListEntryPtr = std::shared_ptr; @@ -61,7 +39,7 @@ class QueryExecutor virtual void cancel() = 0; - virtual String dump() const = 0; + virtual String toString() const = 0; virtual int estimateNewThreadCount() = 0; diff --git a/dbms/src/Flash/Executor/tests/CMakeLists.txt b/dbms/src/Flash/Executor/tests/CMakeLists.txt new file mode 100644 index 00000000000..d030c5c82e7 --- /dev/null +++ b/dbms/src/Flash/Executor/tests/CMakeLists.txt @@ -0,0 +1,15 @@ +# Copyright 2023 PingCAP, Ltd. +# +# 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. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Flash/Executor/tests/gtest_pipeline_executor_status.cpp b/dbms/src/Flash/Executor/tests/gtest_pipeline_executor_status.cpp new file mode 100644 index 00000000000..3833c146d54 --- /dev/null +++ b/dbms/src/Flash/Executor/tests/gtest_pipeline_executor_status.cpp @@ -0,0 +1,84 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include + +namespace DB::tests +{ +class PipelineExecutorStatusTestRunner : public ::testing::Test +{ +}; + +TEST_F(PipelineExecutorStatusTestRunner, timeout) +try +{ + PipelineExecutorStatus status; + try + { + status.onEventSchedule(); + std::chrono::milliseconds timeout(10); + status.waitFor(timeout); + GTEST_FAIL(); + } + catch (Exception & e) + { + GTEST_ASSERT_EQ(e.message(), PipelineExecutorStatus::timeout_err_msg); + auto err_msg = status.getErrMsg(); + ASSERT_EQ(err_msg, PipelineExecutorStatus::timeout_err_msg); + } +} +CATCH + +TEST_F(PipelineExecutorStatusTestRunner, run) +try +{ + PipelineExecutorStatus status; + status.onEventSchedule(); + auto thread_manager = newThreadManager(); + thread_manager->schedule(false, "run", [&status]() mutable { status.onEventFinish(); }); + status.wait(); + auto err_msg = status.getErrMsg(); + ASSERT_TRUE(err_msg.empty()) << err_msg; + thread_manager->wait(); +} +CATCH + +TEST_F(PipelineExecutorStatusTestRunner, to_err) +try +{ + auto test = [](std::string && err_msg) { + auto expect_err_msg = err_msg.empty() ? PipelineExecutorStatus::empty_err_msg : err_msg; + PipelineExecutorStatus status; + status.onEventSchedule(); + auto thread_manager = newThreadManager(); + thread_manager->schedule(false, "err", [&status, &err_msg]() mutable { + status.onErrorOccurred(std::move(err_msg)); + status.onEventFinish(); + }); + status.wait(); + status.onErrorOccurred("unexpect exception"); + auto actual_err_msg = status.getErrMsg(); + ASSERT_TRUE(!actual_err_msg.empty()); + ASSERT_EQ(actual_err_msg, expect_err_msg); + thread_manager->wait(); + }; + test("throw exception"); + test(""); +} +CATCH + +} // namespace DB::tests diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp new file mode 100644 index 00000000000..d4d86e6be56 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.cpp @@ -0,0 +1,121 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include + +namespace DB +{ +#define CHECK_IS_CANCELLED \ + if (unlikely(exec_status.isCancelled())) \ + return OperatorStatus::CANCELLED; + +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, {OperatorStatus::NEED_INPUT}); +#endif + return op_status; +} +/** + * sink_op transform_op ... transform_op source_op + * + * prepare────►tryOutput───► ... ───►tryOutput────►read────┐ + * │ block + * write◄────transform◄─── ... ◄───transform◄────────────┘ + */ +OperatorStatus 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) + return op_status; + + // start from the next transform op after fetched block transform op. + for (size_t transform_op_index = start_transform_op_index; transform_op_index < transform_ops.size(); ++transform_op_index) + { + CHECK_IS_CANCELLED; + const auto & transform_op = transform_ops[transform_op_index]; + op_status = transform_op->transform(block); + if (op_status != OperatorStatus::HAS_OUTPUT) + return op_status; + } + CHECK_IS_CANCELLED; + op_status = sink_op->write(std::move(block)); + return op_status; +} + +// try fetch block from transform_ops and source_op. +OperatorStatus PipelineExec::fetchBlock( + Block & block, + size_t & start_transform_op_index) +{ + CHECK_IS_CANCELLED; + auto op_status = sink_op->prepare(); + if (op_status != OperatorStatus::NEED_INPUT) + return op_status; + for (int64_t index = transform_ops.size() - 1; index >= 0; --index) + { + CHECK_IS_CANCELLED; + const auto & transform_op = transform_ops[index]; + op_status = transform_op->tryOutput(block); + if (op_status != OperatorStatus::NEED_INPUT) + { + // Once the transform op tryOutput has succeeded, execution will begin with the next transform op. + start_transform_op_index = index + 1; + return op_status; + } + } + CHECK_IS_CANCELLED; + start_transform_op_index = 0; + op_status = source_op->read(block); + return op_status; +} + +OperatorStatus 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`. + assertOperatorStatus(op_status, {OperatorStatus::HAS_OUTPUT}); +#endif + return op_status; +} +OperatorStatus PipelineExec::awaitImpl() +{ + CHECK_IS_CANCELLED; + + auto op_status = sink_op->await(); + if (op_status != OperatorStatus::NEED_INPUT) + return op_status; + for (auto it = transform_ops.rbegin(); it != transform_ops.rend(); ++it) + { + // If the transform_op returns `NEED_INPUT`, + // we need to call the upstream transform_op until a transform_op returns something other than `NEED_INPUT`. + op_status = (*it)->await(); + if (op_status != OperatorStatus::NEED_INPUT) + return op_status; + } + op_status = source_op->await(); + return op_status; +} + +#undef CHECK_IS_CANCELLED + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.h b/dbms/src/Flash/Pipeline/Exec/PipelineExec.h new file mode 100644 index 00000000000..e1ec3e8c14e --- /dev/null +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.h @@ -0,0 +1,65 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class PipelineExecutorStatus; + +// The executor for push model operator. +// A pipeline will generate multiple pipeline_execs. +// data flow: source --> transform --> .. --> transform --> sink +class PipelineExec +{ +public: + PipelineExec( + PipelineExecutorStatus & exec_status_, + SourceOpPtr && source_op_, + TransformOps && transform_ops_, + SinkOpPtr && sink_op_) + : exec_status(exec_status_) + , source_op(std::move(source_op_)) + , transform_ops(std::move(transform_ops_)) + , sink_op(std::move(sink_op_)) + {} + + OperatorStatus execute(); + + OperatorStatus await(); + +private: + OperatorStatus executeImpl(); + + OperatorStatus awaitImpl(); + + OperatorStatus fetchBlock( + Block & block, + size_t & start_transform_op_index); + +private: + PipelineExecutorStatus & exec_status; + + SourceOpPtr source_op; + TransformOps transform_ops; + SinkOpPtr sink_op; +}; +using PipelineExecPtr = std::unique_ptr; +// a set of pipeline_execs running in parallel. +using PipelineExecGroup = std::vector; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.cpp b/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.cpp new file mode 100644 index 00000000000..eb47d5f9eef --- /dev/null +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.cpp @@ -0,0 +1,84 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include + +namespace DB +{ +void PipelineExecBuilder::setSourceOp(SourceOpPtr && source_op_) +{ + assert(!source_op && source_op_); + source_op = std::move(source_op_); +} +void PipelineExecBuilder::appendTransformOp(TransformOpPtr && transform_op) +{ + assert(source_op && transform_op); + Block header = getCurrentHeader(); + transform_op->transformHeader(header); + transform_ops.push_back(std::move(transform_op)); +} +void PipelineExecBuilder::setSinkOp(SinkOpPtr && sink_op_) +{ + assert(!sink_op && sink_op_); + Block header = getCurrentHeader(); + sink_op_->setHeader(header); + sink_op = std::move(sink_op_); +} + +PipelineExecPtr PipelineExecBuilder::build(PipelineExecutorStatus & exec_status) +{ + assert(source_op && sink_op); + return std::make_unique( + exec_status, + std::move(source_op), + std::move(transform_ops), + std::move(sink_op)); +} + +Block PipelineExecBuilder::getCurrentHeader() const +{ + if (sink_op) + return sink_op->getHeader(); + else if (!transform_ops.empty()) + return transform_ops.back()->getHeader(); + else + { + assert(source_op); + return source_op->getHeader(); + } +} + +void PipelineExecGroupBuilder::init(size_t init_concurrency) +{ + assert(concurrency == 0); + assert(init_concurrency > 0); + concurrency = init_concurrency; + group.resize(concurrency); +} + +PipelineExecGroup PipelineExecGroupBuilder::build(PipelineExecutorStatus & exec_status) +{ + assert(concurrency > 0); + PipelineExecGroup pipeline_exec_group; + for (auto & builder : group) + pipeline_exec_group.push_back(builder.build(exec_status)); + return pipeline_exec_group; +} + +Block PipelineExecGroupBuilder::getCurrentHeader() +{ + assert(!group.empty()); + return group.back().getCurrentHeader(); +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.h b/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.h new file mode 100644 index 00000000000..1526e610cd9 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.h @@ -0,0 +1,63 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class PipelineExecutorStatus; + +struct PipelineExecBuilder +{ + SourceOpPtr source_op; + TransformOps transform_ops; + SinkOpPtr sink_op; + + void setSourceOp(SourceOpPtr && source_op_); + void appendTransformOp(TransformOpPtr && transform_op); + void setSinkOp(SinkOpPtr && sink_op_); + + Block getCurrentHeader() const; + + PipelineExecPtr build(PipelineExecutorStatus & exec_status); +}; + +struct PipelineExecGroupBuilder +{ + // A Group generates a set of pipeline_execs running in parallel. + using BuilderGroup = std::vector; + BuilderGroup group; + + size_t concurrency = 0; + + void init(size_t init_concurrency); + + /// ff: [](PipelineExecBuilder & builder) {} + template + void transform(FF && ff) + { + assert(concurrency > 0); + for (auto & builder : group) + { + ff(builder); + } + } + + PipelineExecGroup build(PipelineExecutorStatus & exec_status); + + Block getCurrentHeader(); +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Exec/tests/CMakeLists.txt b/dbms/src/Flash/Pipeline/Exec/tests/CMakeLists.txt new file mode 100644 index 00000000000..d030c5c82e7 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Exec/tests/CMakeLists.txt @@ -0,0 +1,15 @@ +# Copyright 2023 PingCAP, Ltd. +# +# 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. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp new file mode 100644 index 00000000000..3d1494211f3 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp @@ -0,0 +1,187 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include +#include +#include +#include + +namespace DB::tests +{ +class SimpleOperatorTestRunner : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + + context.context.setExecutorTest(); + + context.addMockTable({"test_db", "test_table"}, + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + context.addExchangeReceiver("exchange1", + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + context.addExchangeReceiver("exchange3", + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeLongLong}, {"s4", TiDB::TP::TypeLongLong}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"}), + toNullableVec("s3", {1, {}, 1}), + toNullableVec("s4", {1, 1, {}})}); + } + + std::pair build( + const std::shared_ptr & request, + ResultHandler result_handler, + PipelineExecutorStatus & exec_status) + { + DAGContext dag_context(*request, "operator_test", /*concurrency=*/1); + context.context.setDAGContext(&dag_context); + context.context.setMockStorage(context.mockStorage()); + + PhysicalPlan physical_plan{context.context, ""}; + physical_plan.build(request.get()); + assert(!result_handler.isIgnored()); + auto plan_tree = PhysicalGetResultSink::build(result_handler, physical_plan.outputAndOptimize()); + + PipelineExecGroupBuilder group_builder; + PhysicalPlanVisitor::visitPostOrder(plan_tree, [&](const PhysicalPlanNodePtr & plan) { + assert(plan); + plan->buildPipelineExec(group_builder, context.context, /*concurrency=*/1); + }); + auto result = group_builder.build(exec_status); + assert(result.size() == 1); + return {std::move(plan_tree), std::move(result.back())}; + } + + void executeAndAssert( + const std::shared_ptr & request, + const ColumnsWithTypeAndName & expect_columns) + { + Blocks blocks; + ResultHandler result_handler{[&blocks](const Block & block) { + blocks.push_back(block); + }}; + PipelineExecutorStatus exec_status; + auto [plan, op_pipeline] = build(request, result_handler, exec_status); + while (op_pipeline->execute() != OperatorStatus::FINISHED) + { + } + ASSERT_COLUMNS_EQ_UR(expect_columns, mergeBlocks(std::move(blocks)).getColumnsWithTypeAndName()); + } +}; + +TEST_F(SimpleOperatorTestRunner, cancel) +try +{ + auto request = context.receive("exchange1") + .project({col("s1"), col("s2")}) + .filter(eq(col("s1"), col("s2"))) + .limit(1) + .build(context); + + ResultHandler result_handler{[](const Block &) { + }}; + PipelineExecutorStatus exec_status; + auto [_, op_pipeline] = build(request, result_handler, exec_status); + exec_status.cancel(); + ASSERT_EQ(op_pipeline->execute(), OperatorStatus::CANCELLED); +} +CATCH + +TEST_F(SimpleOperatorTestRunner, Filter) +try +{ + auto request = context.receive("exchange1") + .filter(eq(col("s1"), col("s2"))) + .build(context); + + executeAndAssert( + request, + {toNullableVec({"banana"}), + toNullableVec({"banana"})}); +} +CATCH + +TEST_F(SimpleOperatorTestRunner, Limit) +try +{ + auto request = context.receive("exchange1") + .limit(1) + .build(context); + + executeAndAssert( + request, + {toNullableVec({"banana"}), + toNullableVec({"apple"})}); +} +CATCH + +TEST_F(SimpleOperatorTestRunner, Projection) +try +{ + auto request = context.receive("exchange1") + .project({concat(col("s1"), col("s2"))}) + .build(context); + + executeAndAssert( + request, + {toNullableVec({"bananaapple", {}, "bananabanana"})}); + + request = context.receive("exchange3") + .project({concat(col("s1"), col("s2")), concat(col("s1"), col("s2")), And(col("s3"), col("s4")), NOT(col("s3"))}) + .build(context); + + executeAndAssert( + request, + {toNullableVec({"bananaapple", {}, "bananabanana"}), + toNullableVec({"bananaapple", {}, "bananabanana"}), + toNullableVec({1, {}, {}}), + toNullableVec({0, {}, 0})}); +} +CATCH + +TEST_F(SimpleOperatorTestRunner, MockExchangeReceiver) +try +{ + auto request = context.receive("exchange1") + .build(context); + + executeAndAssert( + request, + {toNullableVec({"banana", {}, "banana"}), + toNullableVec({"apple", {}, "banana"})}); +} +CATCH + +TEST_F(SimpleOperatorTestRunner, MockTableScan) +try +{ + auto request = context.scan("test_db", "test_table") + .build(context); + + executeAndAssert( + request, + {toNullableVec({"banana", {}, "banana"}), + toNullableVec({"apple", {}, "banana"})}); +} +CATCH + +} // namespace DB::tests diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp new file mode 100644 index 00000000000..50e00d609ec --- /dev/null +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -0,0 +1,147 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace +{ +FmtBuffer & addPrefix(FmtBuffer & buffer, size_t level) +{ + return buffer.append(String(level, ' ')); +} +} // namespace + +void Pipeline::addPlanNode(const PhysicalPlanNodePtr & plan_node) +{ + plan_nodes.push_back(plan_node); +} + +void Pipeline::addChild(const PipelinePtr & child) +{ + assert(child); + children.push_back(child); +} + +void Pipeline::toSelfString(FmtBuffer & buffer, size_t level) const +{ + if (level > 0) + addPrefix(buffer, level).append("|- "); + buffer.fmtAppend("pipeline#{}: ", id); + buffer.joinStr( + plan_nodes.cbegin(), + plan_nodes.cend(), + [](const auto & plan_node, FmtBuffer & buf) { buf.append(plan_node->toSimpleString()); }, + " -> "); +} + +void Pipeline::toTreeString(FmtBuffer & buffer, size_t level) const +{ + toSelfString(buffer, level); + if (!children.empty()) + buffer.append("\n"); + ++level; + for (const auto & child : children) + child->toTreeString(buffer, level); +} + +void Pipeline::addGetResultSink(ResultHandler result_handler) +{ + assert(!plan_nodes.empty()); + auto get_result_sink = PhysicalGetResultSink::build(result_handler, plan_nodes.back()); + addPlanNode(get_result_sink); +} + +PipelineExecGroup Pipeline::buildExecGroup(PipelineExecutorStatus & exec_status, Context & context, size_t concurrency) +{ + assert(!plan_nodes.empty()); + PipelineExecGroupBuilder builder; + for (const auto & plan_node : plan_nodes) + plan_node->buildPipelineExec(builder, context, concurrency); + return builder.build(exec_status); +} + +Events Pipeline::toEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency) +{ + Events all_events; + toEvent(status, context, concurrency, all_events); + assert(!all_events.empty()); + return all_events; +} + +EventPtr Pipeline::toEvent(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events) +{ + // TODO support fine grained shuffle + // - a fine grained partition maps to an event + // - the event flow will be + // ``` + // disable fine grained partition pipeline enable fine grained partition pipeline enable fine grained partition pipeline + // ┌───────────────FineGrainedPipelineEvent<────────────────FineGrainedPipelineEvent + // PlainPipelineEvent<────────┼───────────────FineGrainedPipelineEvent<────────────────FineGrainedPipelineEvent + // ├───────────────FineGrainedPipelineEvent<────────────────FineGrainedPipelineEvent + // └───────────────FineGrainedPipelineEvent<────────────────FineGrainedPipelineEvent + // ``` + auto memory_tracker = current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr; + + auto plain_pipeline_event = std::make_shared(status, memory_tracker, context, shared_from_this(), concurrency); + for (const auto & child : children) + { + auto input = child->toEvent(status, context, concurrency, all_events); + assert(input); + plain_pipeline_event->addInput(input); + } + all_events.push_back(plain_pipeline_event); + return plain_pipeline_event; +} + +bool Pipeline::isSupported(const tipb::DAGRequest & dag_request) +{ + bool is_supported = true; + traverseExecutors( + &dag_request, + [&](const tipb::Executor & executor) { + if (FineGrainedShuffle(&executor).enable()) + { + is_supported = false; + return false; + } + switch (executor.tp()) + { + case tipb::ExecType::TypeProjection: + case tipb::ExecType::TypeSelection: + case tipb::ExecType::TypeLimit: + // Only support mock table_scan/exchange_sender/exchange_receiver in test mode now. + case tipb::ExecType::TypeTableScan: + case tipb::ExecType::TypeExchangeSender: + case tipb::ExecType::TypeExchangeReceiver: + return true; + default: + is_supported = false; + return false; + } + }); + return is_supported; +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Pipeline.h b/dbms/src/Flash/Pipeline/Pipeline.h new file mode 100644 index 00000000000..3e02457da02 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Pipeline.h @@ -0,0 +1,78 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 + +#include +#include + +namespace tipb +{ +class DAGRequest; +} + +namespace DB +{ +class Pipeline; +using PipelinePtr = std::shared_ptr; +using Pipelines = std::vector; + +class Event; +using EventPtr = std::shared_ptr; +using Events = std::vector; + +class PhysicalPlanNode; +using PhysicalPlanNodePtr = std::shared_ptr; + +class PipelineExecutorStatus; + +class Pipeline : public std::enable_shared_from_this +{ +public: + explicit Pipeline(UInt32 id_) + : id(id_) + {} + + void addPlanNode(const PhysicalPlanNodePtr & plan_node); + + void addChild(const PipelinePtr & child); + + void toTreeString(FmtBuffer & buffer, size_t level = 0) const; + + // only used for test to get the result blocks. + void addGetResultSink(ResultHandler result_handler); + + PipelineExecGroup buildExecGroup(PipelineExecutorStatus & exec_status, Context & context, size_t concurrency); + + Events toEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency); + + static bool isSupported(const tipb::DAGRequest & dag_request); + +private: + void toSelfString(FmtBuffer & buffer, size_t level) const; + + EventPtr toEvent(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events); + +private: + const UInt32 id; + + // data flow: plan_nodes.begin() --> plan_nodes.end() + std::deque plan_nodes; + + std::vector children; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/PipelineBuilder.h b/dbms/src/Flash/Pipeline/PipelineBuilder.h new file mode 100644 index 00000000000..8f7b532b1f7 --- /dev/null +++ b/dbms/src/Flash/Pipeline/PipelineBuilder.h @@ -0,0 +1,102 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class PipelineIdGenerator +{ +public: + UInt32 nextID() + { + return current_id++; + } + +private: + UInt32 current_id = 0; +}; +using PipelineIdGeneratorPtr = std::shared_ptr; + +// PipelineBuilder is used to build pipelines, +// and PipelineBreaker is used to build the tree struct relationship between pipelines. +class PipelineBuilder +{ +public: + PipelineBuilder() + : id_generator(std::make_shared()) + , pipeline_breaker(std::nullopt) + { + pipeline = std::make_shared(id_generator->nextID()); + } + +private: + struct PipelineBreaker + { + PipelineBreaker(const PipelinePtr & pipeline_, const PhysicalPlanNodePtr & breaker_node_) + : pipeline(pipeline_) + , breaker_node(breaker_node_) + { + assert(pipeline); + assert(breaker_node); + } + + // the broken pipeline. + const PipelinePtr pipeline; + const PhysicalPlanNodePtr breaker_node; + }; + + PipelineBuilder(const PipelineIdGeneratorPtr & id_generator_, PipelineBreaker && pipeline_breaker_) + : id_generator(id_generator_) + , pipeline_breaker(std::move(pipeline_breaker_)) + { + pipeline = std::make_shared(id_generator->nextID()); + } + +public: + void addPlanNode(const PhysicalPlanNodePtr & node) + { + pipeline->addPlanNode(node); + } + + /// Break the current pipeline and return a new builder for the broken pipeline. + PipelineBuilder breakPipeline(const PhysicalPlanNodePtr & breaker_node) + { + return PipelineBuilder(id_generator, PipelineBreaker{pipeline, breaker_node}); + } + + PipelinePtr build() + { + assert(pipeline); + if (pipeline_breaker) + { + // First add the breaker node as the last node in this pipeline. + pipeline->addPlanNode(pipeline_breaker->breaker_node); + // Then set this pipeline as a child of the broken pipeline. + pipeline_breaker->pipeline->addChild(pipeline); + pipeline_breaker.reset(); + } + return pipeline; + } + +private: + PipelineIdGeneratorPtr id_generator; + PipelinePtr pipeline; + std::optional pipeline_breaker; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/Event.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/Event.cpp new file mode 100644 index 00000000000..66d9019ef49 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/Event.cpp @@ -0,0 +1,131 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include +#include +#include + +namespace DB +{ +// if any exception throw here, we should record err msg and then cancel the query. +#define CATCH \ + catch (...) \ + { \ + exec_status.onErrorOccurred(getCurrentExceptionMessage(true, true)); \ + } + +void Event::addInput(const EventPtr & input) +{ + assert(status == EventStatus::INIT); + input->addOutput(shared_from_this()); + ++unfinished_inputs; +} + +bool Event::withoutInput() +{ + assert(status == EventStatus::INIT); + return 0 == unfinished_inputs; +} + +void Event::addOutput(const EventPtr & output) +{ + assert(status == EventStatus::INIT); + outputs.push_back(output); +} + +void Event::onInputFinish() +{ + auto cur_value = unfinished_inputs.fetch_sub(1); + assert(cur_value >= 1); + if (1 == cur_value) + schedule(); +} + +void Event::schedule() noexcept +{ + switchStatus(EventStatus::INIT, EventStatus::SCHEDULED); + assert(0 == unfinished_inputs); + exec_status.onEventSchedule(); + MemoryTrackerSetter setter{true, mem_tracker.get()}; + // if err throw here, we should call finish directly. + bool direct_finish = true; + try + { + // if no task is scheduled here, we can call finish directly. + direct_finish = scheduleImpl(); + } + CATCH + if (direct_finish) + finish(); +} + +void Event::finish() noexcept +{ + switchStatus(EventStatus::SCHEDULED, EventStatus::FINISHED); + MemoryTrackerSetter setter{true, mem_tracker.get()}; + try + { + finishImpl(); + } + CATCH + // If query has already been cancelled, it will not trigger outputs. + if (likely(!exec_status.isCancelled())) + { + // finished processing the event, now we can schedule output events. + for (auto & output : outputs) + { + assert(output); + output->onInputFinish(); + output.reset(); + } + } + // Release all output, so that the event that did not call `finishImpl` + // because of `exec_status.isCancelled()` will be destructured before the end of `exec_status.wait`. + outputs.clear(); + // In order to ensure that `exec_status.wait()` doesn't finish when there is an active event, + // we have to call `exec_status.onEventFinish()` here, + // since `exec_status.onEventSchedule()` will have been called by outputs. + // The call order will be `eventA++ ───► eventB++ ───► eventA-- ───► eventB-- ───► exec_status.await finished`. + exec_status.onEventFinish(); +} + +void Event::scheduleTasks(std::vector & tasks) +{ + assert(!tasks.empty()); + assert(0 == unfinished_tasks); + unfinished_tasks = tasks.size(); + assert(status != EventStatus::FINISHED); + TaskScheduler::instance->submit(tasks); +} + +void Event::onTaskFinish() noexcept +{ + assert(status != EventStatus::FINISHED); + auto cur_value = unfinished_tasks.fetch_sub(1); + assert(cur_value >= 1); + if (1 == cur_value) + finish(); +} + +void Event::switchStatus(EventStatus from, EventStatus to) +{ + RUNTIME_ASSERT(status.compare_exchange_strong(from, to)); +} + +#undef CATCH + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/Event.h b/dbms/src/Flash/Pipeline/Schedule/Events/Event.h new file mode 100644 index 00000000000..5067101a088 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/Event.h @@ -0,0 +1,91 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 + +#include +#include +#include + +namespace DB +{ +enum class EventStatus +{ + INIT, + SCHEDULED, + FINISHED, +}; + +class Event; +using EventPtr = std::shared_ptr; +using Events = std::vector; + +class PipelineExecutorStatus; + +class Event : public std::enable_shared_from_this +{ +public: + Event(PipelineExecutorStatus & exec_status_, MemoryTrackerPtr mem_tracker_) + : exec_status(exec_status_) + , mem_tracker(std::move(mem_tracker_)) + {} + virtual ~Event() = default; + + void addInput(const EventPtr & input); + + // schedule, onTaskFinish and finish maybe called directly in TaskScheduler, + // so these functions must be noexcept. + void schedule() noexcept; + + void onTaskFinish() noexcept; + + bool withoutInput(); + +protected: + // Returns true meaning no task is scheduled. + virtual bool scheduleImpl() { return true; } + + // So far the ownership and the life-cycle of the resources are not very well-defined so we still rely on things like "A must be released before B". + // And this is the explicit place to release all the resources that need to be cleaned up before event destruction, so that we can satisfy the above constraints. + virtual void finishImpl() {} + + void scheduleTasks(std::vector & tasks); + +private: + void finish() noexcept; + + void addOutput(const EventPtr & output); + + void onInputFinish(); + + void switchStatus(EventStatus from, EventStatus to); + +protected: + PipelineExecutorStatus & exec_status; + + MemoryTrackerPtr mem_tracker; + +private: + Events outputs; + + std::atomic_int32_t unfinished_inputs{0}; + + std::atomic_int32_t unfinished_tasks{0}; + + std::atomic status{EventStatus::INIT}; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/PipelineEvent.h b/dbms/src/Flash/Pipeline/Schedule/Events/PipelineEvent.h new file mode 100644 index 00000000000..bc862370cea --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/PipelineEvent.h @@ -0,0 +1,49 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class Pipeline; +using PipelinePtr = std::shared_ptr; + +// The base class of pipeline related event. +class PipelineEvent : public Event +{ +public: + PipelineEvent( + PipelineExecutorStatus & exec_status_, + MemoryTrackerPtr mem_tracker_, + Context & context_, + const PipelinePtr & pipeline_) + : Event(exec_status_, std::move(mem_tracker_)) + , context(context_) + , pipeline(pipeline_) + {} + + void finishImpl() override + { + // Plan nodes in pipeline hold resources like hash table for join, when destruction they will operate memory tracker in MPP task. But MPP task may get destructed once `exec_status.onEventFinish()` is called. + // So pipeline needs to be released before `exec_status.onEventFinish()` is called. + pipeline.reset(); + } + +protected: + Context & context; + PipelinePtr pipeline; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp new file mode 100644 index 00000000000..c3543a32806 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp @@ -0,0 +1,33 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include + +namespace DB +{ +bool PlainPipelineEvent::scheduleImpl() +{ + assert(pipeline); + auto pipeline_exec_group = pipeline->buildExecGroup(exec_status, context, concurrency); + assert(!pipeline_exec_group.empty()); + std::vector tasks; + tasks.reserve(pipeline_exec_group.size()); + for (auto & pipline_exec : pipeline_exec_group) + tasks.push_back(std::make_unique(mem_tracker, exec_status, shared_from_this(), std::move(pipline_exec))); + scheduleTasks(tasks); + return false; +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h new file mode 100644 index 00000000000..e42d2c78b0b --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h @@ -0,0 +1,40 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class PlainPipelineEvent : public PipelineEvent +{ +public: + PlainPipelineEvent( + PipelineExecutorStatus & exec_status_, + MemoryTrackerPtr mem_tracker_, + Context & context_, + const PipelinePtr & pipeline_, + size_t concurrency_) + : PipelineEvent(exec_status_, std::move(mem_tracker_), context_, pipeline_) + , concurrency(concurrency_) + {} + +protected: + bool scheduleImpl() override; + +private: + size_t concurrency; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/tests/CMakeLists.txt b/dbms/src/Flash/Pipeline/Schedule/Events/tests/CMakeLists.txt new file mode 100644 index 00000000000..d030c5c82e7 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/tests/CMakeLists.txt @@ -0,0 +1,15 @@ +# Copyright 2023 PingCAP, Ltd. +# +# 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. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp new file mode 100644 index 00000000000..733fa31f596 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/tests/gtest_event.cpp @@ -0,0 +1,519 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include +#include +#include +#include + +namespace DB::tests +{ +namespace +{ +class BaseTask : public EventTask +{ +public: + BaseTask( + PipelineExecutorStatus & exec_status_, + const EventPtr & event_, + std::atomic_int64_t & counter_) + : EventTask(nullptr, exec_status_, event_) + , counter(counter_) + {} + +protected: + ExecTaskStatus doExecuteImpl() override + { + --counter; + return ExecTaskStatus::FINISHED; + } + +private: + std::atomic_int64_t & counter; +}; + +class BaseEvent : public Event +{ +public: + BaseEvent( + PipelineExecutorStatus & exec_status_, + std::atomic_int64_t & counter_) + : Event(exec_status_, nullptr) + , counter(counter_) + {} + + static constexpr auto task_num = 10; + +protected: + // Returns true meaning no task is scheduled. + bool scheduleImpl() override + { + std::vector tasks; + for (size_t i = 0; i < task_num; ++i) + tasks.push_back(std::make_unique(exec_status, shared_from_this(), counter)); + scheduleTasks(tasks); + return false; + } + + void finishImpl() override + { + --counter; + } + +private: + std::atomic_int64_t & counter; +}; + +class RunTask : public EventTask +{ +public: + RunTask( + PipelineExecutorStatus & exec_status_, + const EventPtr & event_) + : EventTask(nullptr, exec_status_, event_) + {} + +protected: + ExecTaskStatus doExecuteImpl() override + { + while ((--loop_count) > 0) + return ExecTaskStatus::RUNNING; + return ExecTaskStatus::FINISHED; + } + +private: + int loop_count = 5; +}; + +class RunEvent : public Event +{ +public: + RunEvent( + PipelineExecutorStatus & exec_status_, + bool with_tasks_) + : Event(exec_status_, nullptr) + , with_tasks(with_tasks_) + {} + +protected: + // Returns true meaning no task is scheduled. + bool scheduleImpl() override + { + if (!with_tasks) + return true; + + std::vector tasks; + for (size_t i = 0; i < 10; ++i) + tasks.push_back(std::make_unique(exec_status, shared_from_this())); + scheduleTasks(tasks); + return false; + } + +private: + bool with_tasks; +}; + +class DeadLoopTask : public EventTask +{ +public: + DeadLoopTask( + PipelineExecutorStatus & exec_status_, + const EventPtr & event_) + : EventTask(nullptr, exec_status_, event_) + {} + +protected: + ExecTaskStatus doExecuteImpl() override + { + std::this_thread::sleep_for(std::chrono::milliseconds(1)); + return ExecTaskStatus::RUNNING; + } +}; + +class DeadLoopEvent : public Event +{ +public: + DeadLoopEvent( + PipelineExecutorStatus & exec_status_, + bool with_tasks_) + : Event(exec_status_, nullptr) + , with_tasks(with_tasks_) + {} + +protected: + // Returns true meaning no task is scheduled. + bool scheduleImpl() override + { + if (!with_tasks) + { + while (!exec_status.isCancelled()) + std::this_thread::sleep_for(std::chrono::milliseconds(1)); + return true; + } + + std::vector tasks; + for (size_t i = 0; i < 10; ++i) + tasks.push_back(std::make_unique(exec_status, shared_from_this())); + scheduleTasks(tasks); + return false; + } + +private: + bool with_tasks; +}; + +class OnErrEvent : public Event +{ +public: + explicit OnErrEvent(PipelineExecutorStatus & exec_status_) + : Event(exec_status_, nullptr) + {} + + static constexpr auto err_msg = "error from OnErrEvent"; + +protected: + // Returns true meaning no task is scheduled. + bool scheduleImpl() override + { + exec_status.onErrorOccurred(err_msg); + return true; + } +}; + +class AssertMemoryTraceEvent : public Event +{ +public: + AssertMemoryTraceEvent(PipelineExecutorStatus & exec_status_, MemoryTrackerPtr mem_tracker_) + : Event(exec_status_, std::move(mem_tracker_)) + {} + +protected: + // Returns true meaning no task is scheduled. + bool scheduleImpl() override + { + assert(mem_tracker.get() == current_memory_tracker); + return true; + } + + void finishImpl() override + { + assert(mem_tracker.get() == current_memory_tracker); + } +}; + +class ThrowExceptionTask : public EventTask +{ +public: + ThrowExceptionTask( + PipelineExecutorStatus & exec_status_, + const EventPtr & event_) + : EventTask(nullptr, exec_status_, event_) + {} + +protected: + ExecTaskStatus doExecuteImpl() override + { + throw Exception("throw exception in doExecuteImpl"); + } +}; + +class ThrowExceptionEvent : public Event +{ +public: + ThrowExceptionEvent( + PipelineExecutorStatus & exec_status_, + bool with_task_) + : Event(exec_status_, nullptr) + , with_task(with_task_) + {} + +protected: + bool scheduleImpl() override + { + if (!with_task) + throw Exception("throw exception in scheduleImpl"); + + std::vector tasks; + for (size_t i = 0; i < 10; ++i) + tasks.push_back(std::make_unique(exec_status, shared_from_this())); + scheduleTasks(tasks); + return false; + } + + void finishImpl() override + { + if (!with_task) + throw Exception("throw exception in finishImpl"); + } + +private: + bool with_task; +}; + +class ManyTasksEvent : public Event +{ +public: + ManyTasksEvent( + PipelineExecutorStatus & exec_status_, + size_t task_num_) + : Event(exec_status_, nullptr) + , task_num(task_num_) + {} + +protected: + // Returns true meaning no task is scheduled. + bool scheduleImpl() override + { + if (0 == task_num) + return true; + + std::vector tasks; + for (size_t i = 0; i < task_num; ++i) + tasks.push_back(std::make_unique(exec_status, shared_from_this())); + scheduleTasks(tasks); + return false; + } + +private: + size_t task_num; +}; +} // namespace + +class EventTestRunner : public ::testing::Test +{ +public: + void schedule(std::vector & events, std::shared_ptr thread_manager = nullptr) + { + Events without_input_events; + for (const auto & event : events) + { + if (event->withoutInput()) + without_input_events.push_back(event); + } + for (const auto & event : without_input_events) + { + if (thread_manager) + thread_manager->schedule(false, "event", [event]() { event->schedule(); }); + else + event->schedule(); + } + } + + void wait(PipelineExecutorStatus & exec_status) + { + std::chrono::seconds timeout(15); + exec_status.waitFor(timeout); + } + + void assertNoErr(PipelineExecutorStatus & exec_status) + { + auto err_msg = exec_status.getErrMsg(); + ASSERT_TRUE(err_msg.empty()) << err_msg; + } + +protected: + static constexpr size_t thread_num = 5; + + void SetUp() override + { + TaskSchedulerConfig config{thread_num}; + assert(!TaskScheduler::instance); + TaskScheduler::instance = std::make_unique(config); + } + + void TearDown() override + { + assert(TaskScheduler::instance); + TaskScheduler::instance.reset(); + } +}; + +TEST_F(EventTestRunner, base) +try +{ + auto do_test = [&](size_t group_num, size_t event_num) { + // group_num * (event_num * (`BaseEvent::finishImpl + BaseEvent::task_num * ~BaseTask()`)) + std::atomic_int64_t counter{static_cast(group_num * (event_num * (1 + BaseEvent::task_num)))}; + PipelineExecutorStatus exec_status; + { + std::vector all_events; + for (size_t i = 0; i < group_num; ++i) + { + std::vector events; + EventPtr start; + for (size_t j = 0; j < event_num; ++j) + { + auto event = std::make_shared(exec_status, counter); + if (!events.empty()) + event->addInput(events.back()); + events.push_back(event); + } + all_events.insert(all_events.end(), events.begin(), events.end()); + } + schedule(all_events); + } + wait(exec_status); + ASSERT_EQ(0, counter); + assertNoErr(exec_status); + }; + for (size_t group_num = 1; group_num < 50; group_num += 11) + { + for (size_t event_num = 1; event_num < 50; event_num += 11) + do_test(group_num, event_num); + } +} +CATCH + +TEST_F(EventTestRunner, run) +try +{ + auto do_test = [&](bool with_tasks, size_t event_num) { + PipelineExecutorStatus exec_status; + { + std::vector events; + for (size_t i = 0; i < event_num; ++i) + events.push_back(std::make_shared(exec_status, with_tasks)); + schedule(events); + } + wait(exec_status); + assertNoErr(exec_status); + }; + for (size_t i = 1; i < 100; i += 7) + { + do_test(false, i); + do_test(true, i); + } +} +CATCH + +TEST_F(EventTestRunner, cancel) +try +{ + auto do_test = [&](bool with_tasks, size_t event_batch_num) { + PipelineExecutorStatus exec_status; + auto thread_manager = newThreadManager(); + { + std::vector events; + for (size_t i = 0; i < event_batch_num; ++i) + { + auto dead_loop_event = std::make_shared(exec_status, with_tasks); + events.push_back(dead_loop_event); + // Expected on_err_event will not be triggered. + auto on_err_event = std::make_shared(exec_status); + on_err_event->addInput(dead_loop_event); + events.push_back(on_err_event); + } + schedule(events, with_tasks ? nullptr : thread_manager); + } + exec_status.cancel(); + wait(exec_status); + assertNoErr(exec_status); + thread_manager->wait(); + }; + for (size_t i = 1; i < 100; i += 7) + { + do_test(false, i); + do_test(true, i); + } +} +CATCH + +TEST_F(EventTestRunner, err) +try +{ + auto do_test = [&](bool with_tasks, size_t dead_loop_event_num) { + PipelineExecutorStatus exec_status; + auto thread_manager = newThreadManager(); + { + std::vector events; + for (size_t i = 0; i < dead_loop_event_num; ++i) + events.push_back(std::make_shared(exec_status, with_tasks)); + schedule(events, with_tasks ? nullptr : thread_manager); + } + { + auto on_err_event = std::make_shared(exec_status); + assert(on_err_event->withoutInput()); + on_err_event->schedule(); + } + wait(exec_status); + auto err_msg = exec_status.getErrMsg(); + ASSERT_EQ(err_msg, OnErrEvent::err_msg) << err_msg; + thread_manager->wait(); + }; + for (size_t i = 1; i < 100; i += 7) + { + do_test(false, i); + do_test(true, i); + } +} +CATCH + +TEST_F(EventTestRunner, memory_trace) +try +{ + PipelineExecutorStatus exec_status; + auto tracker = MemoryTracker::create(); + auto event = std::make_shared(exec_status, tracker); + event->schedule(); + wait(exec_status); + assertNoErr(exec_status); +} +CATCH + +TEST_F(EventTestRunner, throw_exception) +try +{ + std::vector with_tasks{false, true}; + for (auto with_task : with_tasks) + { + PipelineExecutorStatus exec_status; + std::vector events; + // throw_exception_event <-- run_event should run first, + // otherwise the thread pool will be filled up by DeadLoopEvent/DeadLoopTask, + // resulting in a period of time before RunEvent/RunTask/ThrowExceptionEvent will run. + auto run_event = std::make_shared(exec_status, /*with_tasks=*/true); + events.push_back(run_event); + auto crash_event = std::make_shared(exec_status, with_task); + crash_event->addInput(run_event); + events.push_back(crash_event); + + for (size_t i = 0; i < 100; ++i) + events.push_back(std::make_shared(exec_status, /*with_tasks=*/true)); + + schedule(events); + wait(exec_status); + auto err_msg = exec_status.getErrMsg(); + ASSERT_TRUE(!err_msg.empty()); + } +} +CATCH + +TEST_F(EventTestRunner, many_tasks) +try +{ + for (size_t i = 0; i < 200; i += 7) + { + PipelineExecutorStatus exec_status; + auto event = std::make_shared(exec_status, i); + event->schedule(); + wait(exec_status); + assertNoErr(exec_status); + } +} +CATCH + +} // namespace DB::tests diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/FiFOTaskQueue.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/FiFOTaskQueue.cpp new file mode 100644 index 00000000000..e145cc6ea5a --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/FiFOTaskQueue.cpp @@ -0,0 +1,80 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include + +namespace DB +{ +bool FIFOTaskQueue::take(TaskPtr & task) +{ + assert(!task); + { + std::unique_lock lock(mu); + while (true) + { + if (unlikely(is_closed)) + return false; + if (!task_queue.empty()) + break; + cv.wait(lock); + } + + task = std::move(task_queue.front()); + task_queue.pop_front(); + } + assert(task); + return true; +} + +bool FIFOTaskQueue::empty() +{ + std::lock_guard lock(mu); + return task_queue.empty(); +} + +void FIFOTaskQueue::close() +{ + { + std::lock_guard lock(mu); + is_closed = true; + } + cv.notify_all(); +} + +void FIFOTaskQueue::submit(TaskPtr && task) +{ + assert(task); + { + std::lock_guard lock(mu); + task_queue.push_back(std::move(task)); + } + cv.notify_one(); +} + +void FIFOTaskQueue::submit(std::vector & tasks) +{ + if (tasks.empty()) + return; + + std::lock_guard lock(mu); + for (auto & task : tasks) + { + assert(task); + task_queue.push_back(std::move(task)); + cv.notify_one(); + } +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/FiFOTaskQueue.h b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/FiFOTaskQueue.h new file mode 100644 index 00000000000..bcac9ed4d8a --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/FiFOTaskQueue.h @@ -0,0 +1,46 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 + +#include +#include + +namespace DB +{ +class FIFOTaskQueue : public TaskQueue +{ +public: + void submit(TaskPtr && task) override; + + void submit(std::vector & tasks) override; + + bool take(TaskPtr & task) override; + + bool empty() override; + + void close() override; + +private: + std::mutex mu; + std::condition_variable cv; + bool is_closed = false; + std::deque task_queue; + + LoggerPtr logger = Logger::get(); +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/TaskQueue.h b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/TaskQueue.h new file mode 100644 index 00000000000..079626b191a --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/TaskQueue.h @@ -0,0 +1,45 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include + +namespace DB +{ +// TODO support more kind of TaskQueue, such as +// - multi-level feedback queue +// - resource group queue +class TaskQueue +{ +public: + virtual ~TaskQueue() = default; + + virtual void submit(TaskPtr && task) = 0; + + virtual void submit(std::vector & tasks) = 0; + + // return false if the queue had been closed. + virtual bool take(TaskPtr & task) = 0; + + virtual bool empty() = 0; + + virtual void close() = 0; +}; +using TaskQueuePtr = std::unique_ptr; + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/CMakeLists.txt b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/CMakeLists.txt new file mode 100644 index 00000000000..d030c5c82e7 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/CMakeLists.txt @@ -0,0 +1,15 @@ +# Copyright 2023 PingCAP, Ltd. +# +# 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. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_fifo.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_fifo.cpp new file mode 100644 index 00000000000..cd03f1f9e5c --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskQueues/tests/gtest_fifo.cpp @@ -0,0 +1,83 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include + +#include + +namespace DB::tests +{ +namespace +{ +class IndexTask : public Task +{ +public: + explicit IndexTask(size_t index_) + : Task(nullptr) + , index(index_) + {} + + ExecTaskStatus executeImpl() override { return ExecTaskStatus::FINISHED; } + + size_t index; +}; +} // namespace + +class FIFOTestRunner : public ::testing::Test +{ +}; + +TEST_F(FIFOTestRunner, base) +try +{ + FIFOTaskQueue queue; + + auto thread_manager = newThreadManager(); + size_t valid_task_num = 1000; + + // submit valid task + thread_manager->schedule(false, "submit", [&]() { + for (size_t i = 0; i < valid_task_num; ++i) + queue.submit(std::make_unique(i)); + // Close the queue after all valid tasks have been consumed. + while (!queue.empty()) + std::this_thread::sleep_for(std::chrono::milliseconds(10)); + queue.close(); + }); + // take valid task + thread_manager->schedule(false, "take", [&]() { + TaskPtr task; + size_t expect_index = 0; + while (queue.take(task)) + { + ASSERT_TRUE(task); + auto * index_task = static_cast(task.get()); + ASSERT_EQ(index_task->index, expect_index++); + task.reset(); + } + ASSERT_EQ(expect_index, valid_task_num); + }); + thread_manager->wait(); + + // No tasks are taken after the queue is closed. + queue.submit(std::make_unique(valid_task_num)); + TaskPtr task; + ASSERT_FALSE(queue.take(task)); +} +CATCH + +} // namespace DB::tests diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskScheduler.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskScheduler.cpp new file mode 100644 index 00000000000..75d584b116c --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskScheduler.cpp @@ -0,0 +1,74 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +TaskScheduler::TaskScheduler(const TaskSchedulerConfig & config) + : task_thread_pool(*this, config.task_thread_pool_size) + , wait_reactor(*this) +{ +} + +TaskScheduler::~TaskScheduler() +{ + task_thread_pool.close(); + wait_reactor.close(); + + task_thread_pool.waitForStop(); + wait_reactor.waitForStop(); +} + +void TaskScheduler::submit(std::vector & tasks) +{ + if (unlikely(tasks.empty())) + return; + + // The memory tracker is set by the caller. + std::vector running_tasks; + std::list waiting_tasks; + for (auto & task : tasks) + { + assert(task); + // A quick check to avoid an unnecessary round into `running_tasks` then being scheduled out immediately. + auto status = task->await(); + switch (status) + { + case ExecTaskStatus::RUNNING: + running_tasks.push_back(std::move(task)); + break; + case ExecTaskStatus::WAITING: + waiting_tasks.push_back(std::move(task)); + break; + case FINISH_STATUS: + task.reset(); + break; + default: + UNEXPECTED_STATUS(logger, status); + } + } + tasks.clear(); + task_thread_pool.submit(running_tasks); + wait_reactor.submit(waiting_tasks); +} + +std::unique_ptr TaskScheduler::instance; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskScheduler.h b/dbms/src/Flash/Pipeline/Schedule/TaskScheduler.h new file mode 100644 index 00000000000..3d86cad8ac2 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskScheduler.h @@ -0,0 +1,68 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include +#include + +namespace DB +{ +struct TaskSchedulerConfig +{ + size_t task_thread_pool_size; +}; + +/** + * ┌─────────────────────┐ + * │ task scheduler │ + * │ │ + * │ ┌────────────────┐ │ + * │ │task thread pool│ │ + * │ └──────▲──┬──────┘ │ + * │ │ │ │ + * │ ┌────┴──▼────┐ │ + * │ │wait reactor│ │ + * │ └────────────┘ │ + * │ │ + * └─────────────────────┘ + * + * A globally shared execution scheduler, used by pipeline executor. + * - task thread pool: for operator compute. + * - wait reactor: for polling asynchronous io status, etc. + */ +class TaskScheduler +{ +public: + explicit TaskScheduler(const TaskSchedulerConfig & config); + + ~TaskScheduler(); + + void submit(std::vector & tasks); + + static std::unique_ptr instance; + +private: + TaskThreadPool task_thread_pool; + + WaitReactor wait_reactor; + + LoggerPtr logger = Logger::get(); + + friend class TaskThreadPool; + friend class WaitReactor; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskThreadPool.cpp b/dbms/src/Flash/Pipeline/Schedule/TaskThreadPool.cpp new file mode 100644 index 00000000000..3e6cceba437 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskThreadPool.cpp @@ -0,0 +1,108 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +TaskThreadPool::TaskThreadPool(TaskScheduler & scheduler_, size_t thread_num) + : task_queue(std::make_unique()) + , scheduler(scheduler_) +{ + RUNTIME_CHECK(thread_num > 0); + threads.reserve(thread_num); + for (size_t i = 0; i < thread_num; ++i) + threads.emplace_back(&TaskThreadPool::loop, this, i); +} + +void TaskThreadPool::close() +{ + task_queue->close(); +} + +void TaskThreadPool::waitForStop() +{ + for (auto & thread : threads) + thread.join(); + LOG_INFO(logger, "task thread pool is stopped"); +} + +void TaskThreadPool::loop(size_t thread_no) noexcept +{ + auto thread_no_str = fmt::format("thread_no={}", thread_no); + auto thread_logger = logger->getChild(thread_no_str); + setThreadName(thread_no_str.c_str()); + LOG_INFO(thread_logger, "start loop"); + ASSERT_MEMORY_TRACKER + + TaskPtr task; + while (likely(task_queue->take(task))) + { + handleTask(task, thread_logger); + assert(!task); + ASSERT_MEMORY_TRACKER + } + + LOG_INFO(thread_logger, "loop finished"); +} + +void TaskThreadPool::handleTask(TaskPtr & task, const LoggerPtr & log) +{ + assert(task); + TRACE_MEMORY(task); + + Stopwatch stopwatch{CLOCK_MONOTONIC_COARSE}; + ExecTaskStatus status; + while (true) + { + status = task->execute(); + // The executing task should yield if it takes more than `YIELD_MAX_TIME_SPENT_NS`. + if (status != ExecTaskStatus::RUNNING || stopwatch.elapsed() >= YIELD_MAX_TIME_SPENT_NS) + break; + } + + switch (status) + { + case ExecTaskStatus::RUNNING: + submit(std::move(task)); + break; + case ExecTaskStatus::WAITING: + scheduler.wait_reactor.submit(std::move(task)); + break; + case FINISH_STATUS: + task.reset(); + break; + default: + UNEXPECTED_STATUS(log, status); + } +} + +void TaskThreadPool::submit(TaskPtr && task) +{ + task_queue->submit(std::move(task)); +} + +void TaskThreadPool::submit(std::vector & tasks) +{ + task_queue->submit(tasks); +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/TaskThreadPool.h b/dbms/src/Flash/Pipeline/Schedule/TaskThreadPool.h new file mode 100644 index 00000000000..6e6df611003 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/TaskThreadPool.h @@ -0,0 +1,55 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include + +#include +#include + +namespace DB +{ +class TaskScheduler; + +class TaskThreadPool +{ +public: + TaskThreadPool(TaskScheduler & scheduler_, size_t thread_num); + + void close(); + + void waitForStop(); + + void submit(TaskPtr && task); + + void submit(std::vector & tasks); + +private: + void loop(size_t thread_no) noexcept; + + void handleTask(TaskPtr & task, const LoggerPtr & log); + +private: + TaskQueuePtr task_queue; + + LoggerPtr logger = Logger::get(); + + TaskScheduler & scheduler; + + std::vector threads; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp new file mode 100644 index 00000000000..f6e87a63a39 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp @@ -0,0 +1,47 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include + +namespace DB +{ +EventTask::EventTask( + MemoryTrackerPtr mem_tracker_, + PipelineExecutorStatus & exec_status_, + const EventPtr & event_) + : Task(std::move(mem_tracker_)) + , exec_status(exec_status_) + , event(event_) +{ + assert(event); +} + +EventTask::~EventTask() +{ + assert(event); + event->onTaskFinish(); + event.reset(); +} + +ExecTaskStatus EventTask::executeImpl() +{ + return doTaskAction([&] { return doExecuteImpl(); }); +} + +ExecTaskStatus EventTask::awaitImpl() +{ + return doTaskAction([&] { return doAwaitImpl(); }); +} + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h new file mode 100644 index 00000000000..a1f6cd9dae3 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h @@ -0,0 +1,79 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include +#include + +namespace DB +{ +// The base class of event related task. +class EventTask : public Task +{ +public: + EventTask( + MemoryTrackerPtr mem_tracker_, + PipelineExecutorStatus & exec_status_, + const EventPtr & event_); + + ~EventTask(); + +protected: + ExecTaskStatus executeImpl() override; + virtual ExecTaskStatus doExecuteImpl() = 0; + + ExecTaskStatus awaitImpl() override; + virtual ExecTaskStatus doAwaitImpl() { return ExecTaskStatus::RUNNING; }; + + // Used to release held resources, just like `Event::finishImpl`. + virtual void finalize(){}; + +private: + template + ExecTaskStatus doTaskAction(Action && action) + { + if (unlikely(exec_status.isCancelled())) + { + finalize(); + return ExecTaskStatus::CANCELLED; + } + try + { + auto status = action(); + switch (status) + { + case FINISH_STATUS: + finalize(); + default: + return status; + } + } + catch (...) + { + finalize(); + assert(event); + exec_status.onErrorOccurred(getCurrentExceptionMessage(true, true)); + return ExecTaskStatus::ERROR; + } + } + +private: + PipelineExecutorStatus & exec_status; + EventPtr event; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.cpp new file mode 100644 index 00000000000..b9c1b4dfdaa --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.cpp @@ -0,0 +1,90 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +#include + +namespace DB +{ +PipelineTask::PipelineTask( + MemoryTrackerPtr mem_tracker_, + PipelineExecutorStatus & exec_status_, + const EventPtr & event_, + PipelineExecPtr && pipeline_exec_) + : EventTask(std::move(mem_tracker_), exec_status_, event_) + , pipeline_exec(std::move(pipeline_exec_)) +{ + assert(pipeline_exec); +} + +void PipelineTask::finalize() +{ + pipeline_exec.reset(); +} + +#define HANDLE_NOT_RUNNING_STATUS \ + case OperatorStatus::FINISHED: \ + { \ + return ExecTaskStatus::FINISHED; \ + } \ + case OperatorStatus::CANCELLED: \ + { \ + return ExecTaskStatus::CANCELLED; \ + } \ + case OperatorStatus::WAITING: \ + { \ + return ExecTaskStatus::WAITING; \ + } + +#define UNEXPECTED_OP_STATUS(op_status, function_name) \ + throw Exception(fmt::format("Unexpected op state {} at {}", magic_enum::enum_name(op_status), (function_name))); + +ExecTaskStatus PipelineTask::doExecuteImpl() +{ + assert(pipeline_exec); + auto op_status = pipeline_exec->execute(); + switch (op_status) + { + HANDLE_NOT_RUNNING_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, "PipelineTask::execute"); + } +} + +ExecTaskStatus PipelineTask::doAwaitImpl() +{ + assert(pipeline_exec); + auto op_status = pipeline_exec->await(); + switch (op_status) + { + HANDLE_NOT_RUNNING_STATUS + // After `pipeline_exec->await`, `HAS_OUTPUT` means that pipeline_exec has data to do the calculations and expect the next call to `execute` + // And other states are unexpected. + case OperatorStatus::HAS_OUTPUT: + return ExecTaskStatus::RUNNING; + default: + UNEXPECTED_OP_STATUS(op_status, "PipelineTask::await"); + } +} + +#undef HANDLE_NOT_RUNNING_STATUS +#undef UNEXPECTED_OP_STATUS + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h new file mode 100644 index 00000000000..ec12fe48f63 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/PipelineTask.h @@ -0,0 +1,41 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class PipelineTask : public EventTask +{ +public: + PipelineTask( + MemoryTrackerPtr mem_tracker_, + PipelineExecutorStatus & exec_status_, + const EventPtr & event_, + PipelineExecPtr && pipeline_exec_); + +protected: + ExecTaskStatus doExecuteImpl() override; + + ExecTaskStatus doAwaitImpl() override; + + void finalize() override; + +private: + PipelineExecPtr pipeline_exec; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h new file mode 100644 index 00000000000..9d8b4164aff --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/Task.h @@ -0,0 +1,74 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +/** + * CANCELLED/ERROR/FINISHED + * ▲ + * │ + * ┌────────────────────────┐ + * │ WATITING◄─────►RUNNING │ + * └────────────────────────┘ + */ +enum class ExecTaskStatus +{ + WAITING, + RUNNING, + FINISHED, + ERROR, + CANCELLED, +}; + +class Task +{ +public: + explicit Task(MemoryTrackerPtr mem_tracker_) + : mem_tracker(std::move(mem_tracker_)) + {} + + virtual ~Task() = default; + + MemoryTrackerPtr getMemTracker() + { + return mem_tracker; + } + + ExecTaskStatus execute() noexcept + { + assert(getMemTracker().get() == current_memory_tracker); + return executeImpl(); + } + // Avoid allocating memory in `await` if possible. + ExecTaskStatus await() noexcept + { + assert(getMemTracker().get() == current_memory_tracker); + return awaitImpl(); + } + +protected: + virtual ExecTaskStatus executeImpl() = 0; + virtual ExecTaskStatus awaitImpl() { return ExecTaskStatus::RUNNING; } + +private: + MemoryTrackerPtr mem_tracker; +}; +using TaskPtr = std::unique_ptr; + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/TaskHelper.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/TaskHelper.h new file mode 100644 index 00000000000..598305db124 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/TaskHelper.h @@ -0,0 +1,45 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include +#include + +#include + +namespace DB +{ +// Hold the shared_ptr of memory tracker. +// To avoid the current_memory_tracker being an illegal pointer. +#define TRACE_MEMORY(task) \ + assert(nullptr == current_memory_tracker); \ + auto memory_tracker = (task)->getMemTracker(); \ + MemoryTrackerSetter memory_tracker_setter{true, memory_tracker.get()}; + +#define ASSERT_MEMORY_TRACKER \ + assert(nullptr == current_memory_tracker); \ + assert(0 == CurrentMemoryTracker::getLocalDeltaMemory()); + +#define FINISH_STATUS \ + ExecTaskStatus::FINISHED : case ExecTaskStatus::ERROR : case ExecTaskStatus::CANCELLED + +#define UNEXPECTED_STATUS(logger, status) \ + RUNTIME_ASSERT(false, (logger), "Unexpected task status {}", magic_enum::enum_name(status)); + +static constexpr int64_t YIELD_MAX_TIME_SPENT_NS = 100'000'000L; + +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/WaitReactor.cpp b/dbms/src/Flash/Pipeline/Schedule/WaitReactor.cpp new file mode 100644 index 00000000000..b3fdbe4d088 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/WaitReactor.cpp @@ -0,0 +1,160 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace +{ +class Spinner +{ +public: + Spinner(TaskThreadPool & task_thread_pool_, const LoggerPtr & logger_) + : task_thread_pool(task_thread_pool_) + , logger(logger_->getChild("Spinner")) + {} + + // return true if the task is not in waiting status. + bool awaitAndPushReadyTask(TaskPtr && task) + { + assert(task); + TRACE_MEMORY(task); + auto status = task->await(); + switch (status) + { + case ExecTaskStatus::RUNNING: + running_tasks.push_back(std::move(task)); + return true; + case ExecTaskStatus::WAITING: + return false; + case FINISH_STATUS: + task.reset(); + return true; + default: + UNEXPECTED_STATUS(logger, status); + } + } + + // return false if there are no ready task to submit. + bool submitReadyTasks() + { + if (running_tasks.empty()) + return false; + + task_thread_pool.submit(running_tasks); + running_tasks.clear(); + spin_count = 0; + return true; + } + + void tryYield() + { + assert(running_tasks.empty()); + ++spin_count; + + if (spin_count != 0 && spin_count % 64 == 0) + { + sched_yield(); + if (spin_count == 640) + { + spin_count = 0; + sched_yield(); + } + } + } + +private: + TaskThreadPool & task_thread_pool; + + LoggerPtr logger; + + int16_t spin_count = 0; + + std::vector running_tasks; +}; +} // namespace + +WaitReactor::WaitReactor(TaskScheduler & scheduler_) + : scheduler(scheduler_) +{ + thread = std::thread(&WaitReactor::loop, this); +} + +void WaitReactor::close() +{ + waiting_task_list.close(); +} + +void WaitReactor::waitForStop() +{ + thread.join(); + LOG_INFO(logger, "wait reactor is stopped"); +} + +void WaitReactor::submit(TaskPtr && task) +{ + waiting_task_list.submit(std::move(task)); +} + +void WaitReactor::submit(std::list & tasks) +{ + waiting_task_list.submit(tasks); +} + +void WaitReactor::loop() noexcept +{ + setThreadName("WaitReactor"); + LOG_INFO(logger, "start wait reactor loop"); + ASSERT_MEMORY_TRACKER + + Spinner spinner{scheduler.task_thread_pool, logger}; + std::list local_waiting_tasks; + // Get the incremental tasks from waiting_task_list. + // return false if waiting_task_list has been closed. + auto take_from_waiting_task_list = [&]() { + return local_waiting_tasks.empty() + ? waiting_task_list.take(local_waiting_tasks) + // If the local waiting tasks are not empty, there is no need to be blocked here + // and we can continue to process the leftover tasks in the local waiting tasks + : waiting_task_list.tryTake(local_waiting_tasks); + }; + while (take_from_waiting_task_list()) + { + assert(!local_waiting_tasks.empty()); + auto task_it = local_waiting_tasks.begin(); + while (task_it != local_waiting_tasks.end()) + { + if (spinner.awaitAndPushReadyTask(std::move(*task_it))) + task_it = local_waiting_tasks.erase(task_it); + else + ++task_it; + ASSERT_MEMORY_TRACKER + } + + if (!spinner.submitReadyTasks()) + spinner.tryYield(); + } + + LOG_INFO(logger, "wait reactor loop finished"); +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/WaitReactor.h b/dbms/src/Flash/Pipeline/Schedule/WaitReactor.h new file mode 100644 index 00000000000..95a8e819edd --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/WaitReactor.h @@ -0,0 +1,53 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include + +#include +#include + +namespace DB +{ +class TaskScheduler; + +class WaitReactor +{ +public: + explicit WaitReactor(TaskScheduler & scheduler_); + + void close(); + + void waitForStop(); + + void submit(TaskPtr && task); + + void submit(std::list & tasks); + +private: + void loop() noexcept; + +private: + WaitingTaskList waiting_task_list; + + LoggerPtr logger = Logger::get(); + + TaskScheduler & scheduler; + + std::thread thread; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/WaitingTaskList.cpp b/dbms/src/Flash/Pipeline/Schedule/WaitingTaskList.cpp new file mode 100644 index 00000000000..e3358d3e50c --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/WaitingTaskList.cpp @@ -0,0 +1,78 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include + +namespace DB +{ +bool WaitingTaskList::take(std::list & local_waiting_tasks) +{ + { + std::unique_lock lock(mu); + while (true) + { + if (unlikely(is_closed)) + return false; + if (!waiting_tasks.empty()) + break; + cv.wait(lock); + } + + local_waiting_tasks.splice(local_waiting_tasks.end(), waiting_tasks); + } + assert(!local_waiting_tasks.empty()); + return true; +} + +bool WaitingTaskList::tryTake(std::list & local_waiting_tasks) +{ + std::lock_guard lock(mu); + if (unlikely(is_closed)) + return false; + local_waiting_tasks.splice(local_waiting_tasks.end(), waiting_tasks); + return true; +} + +void WaitingTaskList::close() +{ + { + std::lock_guard lock(mu); + is_closed = true; + } + cv.notify_all(); +} + +void WaitingTaskList::submit(TaskPtr && task) +{ + assert(task); + { + std::lock_guard lock(mu); + waiting_tasks.emplace_back(std::move(task)); + } + cv.notify_one(); +} + +void WaitingTaskList::submit(std::list & tasks) +{ + if (tasks.empty()) + return; + { + std::lock_guard lock(mu); + waiting_tasks.splice(waiting_tasks.end(), tasks); + } + cv.notify_one(); +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/WaitingTaskList.h b/dbms/src/Flash/Pipeline/Schedule/WaitingTaskList.h new file mode 100644 index 00000000000..1e8aea67646 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/WaitingTaskList.h @@ -0,0 +1,45 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include + +namespace DB +{ +class WaitingTaskList +{ +public: + /// return false if the waiting task list had been closed. + // this function will wait until `!waiting_tasks.empty()` + bool take(std::list & local_waiting_tasks); + // this function will return immediately. + bool tryTake(std::list & local_waiting_tasks); + + void submit(TaskPtr && task); + + void submit(std::list & tasks); + + void close(); + +private: + std::mutex mu; + std::condition_variable cv; + std::list waiting_tasks; + bool is_closed = false; +}; +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/tests/CMakeLists.txt b/dbms/src/Flash/Pipeline/Schedule/tests/CMakeLists.txt new file mode 100644 index 00000000000..d030c5c82e7 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/tests/CMakeLists.txt @@ -0,0 +1,15 @@ +# Copyright 2023 PingCAP, Ltd. +# +# 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. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp new file mode 100644 index 00000000000..f18f0f3e97a --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/tests/gtest_task_scheduler.cpp @@ -0,0 +1,282 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include +#include + +namespace DB::tests +{ +namespace +{ +class Waiter +{ +public: + Waiter(size_t init_value) + : counter(init_value) + {} + void notify() + { + bool last_notify = false; + { + std::lock_guard lock(mu); + last_notify = (--counter) == 0; + } + if (last_notify) + cv.notify_one(); + } + void wait() + { + std::chrono::seconds timeout(15); + std::unique_lock lock(mu); + RUNTIME_CHECK(cv.wait_for(lock, timeout, [&] { return 0 == counter; })); + } + +private: + mutable std::mutex mu; + int64_t counter; + std::condition_variable cv; +}; + +class SimpleTask : public Task +{ +public: + explicit SimpleTask(Waiter & waiter_) + : Task(nullptr) + , waiter(waiter_) + {} + + ~SimpleTask() + { + waiter.notify(); + } + +protected: + ExecTaskStatus executeImpl() override + { + while ((--loop_count) > 0) + return ExecTaskStatus::RUNNING; + return ExecTaskStatus::FINISHED; + } + +private: + Waiter & waiter; + int loop_count = 5; +}; + +class SimpleWaitingTask : public Task +{ +public: + explicit SimpleWaitingTask(Waiter & waiter_) + : Task(nullptr) + , waiter(waiter_) + {} + + ~SimpleWaitingTask() + { + waiter.notify(); + } + +protected: + ExecTaskStatus executeImpl() override + { + if (loop_count > 0) + { + if ((loop_count % 2) == 0) + return ExecTaskStatus::WAITING; + else + { + --loop_count; + return ExecTaskStatus::RUNNING; + } + } + return ExecTaskStatus::FINISHED; + } + + ExecTaskStatus awaitImpl() override + { + if (loop_count > 0) + { + if ((loop_count % 2) == 0) + { + --loop_count; + return ExecTaskStatus::WAITING; + } + else + return ExecTaskStatus::RUNNING; + } + return ExecTaskStatus::FINISHED; + } + +private: + int loop_count = 10 + random() % 10; + Waiter & waiter; +}; + +enum class TraceTaskStatus +{ + initing, + running, + waiting, +}; +class MemoryTraceTask : public Task +{ +public: + MemoryTraceTask(MemoryTrackerPtr mem_tracker_, Waiter & waiter_) + : Task(std::move(mem_tracker_)) + , waiter(waiter_) + {} + + ~MemoryTraceTask() + { + waiter.notify(); + } + + // From CurrentMemoryTracker::MEMORY_TRACER_SUBMIT_THRESHOLD + static constexpr Int64 MEMORY_TRACER_SUBMIT_THRESHOLD = 1024 * 1024; // 1 MiB + +protected: + ExecTaskStatus executeImpl() override + { + switch (status) + { + case TraceTaskStatus::initing: + status = TraceTaskStatus::waiting; + return ExecTaskStatus::WAITING; + case TraceTaskStatus::waiting: + { + status = TraceTaskStatus::running; + CurrentMemoryTracker::alloc(MEMORY_TRACER_SUBMIT_THRESHOLD); + return ExecTaskStatus::FINISHED; + } + default: + __builtin_unreachable(); + } + } + + ExecTaskStatus awaitImpl() override + { + if (status == TraceTaskStatus::waiting) + CurrentMemoryTracker::alloc(MEMORY_TRACER_SUBMIT_THRESHOLD - 10); + return ExecTaskStatus::RUNNING; + } + +private: + TraceTaskStatus status{TraceTaskStatus::initing}; + Waiter & waiter; +}; + +class DeadLoopTask : public Task +{ +public: + DeadLoopTask() + : Task(nullptr) + {} + +protected: + ExecTaskStatus executeImpl() override + { + return ExecTaskStatus::WAITING; + } + + ExecTaskStatus awaitImpl() override + { + return ExecTaskStatus::RUNNING; + } +}; +} // namespace + +class TaskSchedulerTestRunner : public ::testing::Test +{ +public: + static constexpr size_t thread_num = 5; + + void submitAndWait(std::vector & tasks, Waiter & waiter) + { + TaskSchedulerConfig config{thread_num}; + TaskScheduler task_scheduler{config}; + task_scheduler.submit(tasks); + waiter.wait(); + } +}; + +TEST_F(TaskSchedulerTestRunner, simple_task) +try +{ + for (size_t task_num = 1; task_num < 100; ++task_num) + { + Waiter waiter(task_num); + std::vector tasks; + for (size_t i = 0; i < task_num; ++i) + tasks.push_back(std::make_unique(waiter)); + submitAndWait(tasks, waiter); + } +} +CATCH + +TEST_F(TaskSchedulerTestRunner, simple_waiting_task) +try +{ + for (size_t task_num = 1; task_num < 100; ++task_num) + { + Waiter waiter(task_num); + std::vector tasks; + for (size_t i = 0; i < task_num; ++i) + tasks.push_back(std::make_unique(waiter)); + submitAndWait(tasks, waiter); + } +} +CATCH + +TEST_F(TaskSchedulerTestRunner, test_memory_trace) +try +{ + for (size_t task_num = 1; task_num < 100; ++task_num) + { + auto tracker = MemoryTracker::create(); + MemoryTrackerSetter memory_tracker_setter{true, tracker.get()}; + Waiter waiter(task_num); + std::vector tasks; + for (size_t i = 0; i < task_num; ++i) + tasks.push_back(std::make_unique(tracker, waiter)); + submitAndWait(tasks, waiter); + // The value of the memory tracer is not checked here because of `std::memory_order_relaxed`. + } +} +CATCH + +TEST_F(TaskSchedulerTestRunner, shutdown) +try +{ + auto do_test = [](size_t task_thread_pool_size, size_t task_num) { + TaskSchedulerConfig config{task_thread_pool_size}; + TaskScheduler task_scheduler{config}; + std::vector tasks; + for (size_t i = 0; i < task_num; ++i) + tasks.push_back(std::make_unique()); + task_scheduler.submit(tasks); + }; + std::vector thread_nums{1, 5, 10, 100}; + for (auto task_thread_pool_size : thread_nums) + { + std::vector task_nums{0, 1, 5, 10, 100, 200}; + for (auto task_num : task_nums) + do_test(task_thread_pool_size, task_num); + } +} +CATCH + +} // namespace DB::tests diff --git a/dbms/src/Flash/Pipeline/tests/CMakeLists.txt b/dbms/src/Flash/Pipeline/tests/CMakeLists.txt new file mode 100644 index 00000000000..d030c5c82e7 --- /dev/null +++ b/dbms/src/Flash/Pipeline/tests/CMakeLists.txt @@ -0,0 +1,15 @@ +# Copyright 2023 PingCAP, Ltd. +# +# 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. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Flash/Pipeline/tests/gtest_pipeline.cpp b/dbms/src/Flash/Pipeline/tests/gtest_pipeline.cpp new file mode 100644 index 00000000000..edcfd191d56 --- /dev/null +++ b/dbms/src/Flash/Pipeline/tests/gtest_pipeline.cpp @@ -0,0 +1,88 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include +#include + +namespace DB +{ +namespace tests +{ +class PipelineTestRunner : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + + context.context.setExecutorTest(); + + context.addMockTable({"test_db", "test_table"}, + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + context.addExchangeReceiver("exchange", + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + } + + void assertEquals( + const std::shared_ptr & request, + const String & expected_pipeline) + { + DAGContext dag_context(*request, "executor_test", /*concurrency=*/1); + context.context.setDAGContext(&dag_context); + context.context.setMockStorage(context.mockStorage()); + + PhysicalPlan physical_plan{context.context, ""}; + assert(request); + physical_plan.build(request.get()); + physical_plan.outputAndOptimize(); + auto root_pipeline = physical_plan.toPipeline(); + assert(root_pipeline); + FmtBuffer buffer; + root_pipeline->toTreeString(buffer); + ASSERT_EQ(Poco::trim(expected_pipeline), Poco::trim(buffer.toString())); + } +}; + +TEST_F(PipelineTestRunner, simple_pipeline) +try +{ + { + auto request = context.receive("exchange") + .filter(eq(col("s1"), col("s2"))) + .project({concat(col("s1"), col("s2"))}) + .limit(1) + .exchangeSender(tipb::Hash) + .build(context); + assertEquals( + request, + "pipeline#0: MockExchangeReceiver|exchange_receiver_0 -> Filter|selection_1 -> Projection|project_2 -> Limit|limit_3 -> Projection|NonTiDBOperator -> MockExchangeSender|exchange_sender_4"); + } + + { + auto request = context.scan("test_db", "test_table").build(context); + assertEquals( + request, + "pipeline#0: MockTableScan|table_scan_0 -> Projection|NonTiDBOperator"); + } +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 5f171fe4fd8..f69a5a6d71e 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,24 +16,26 @@ #include #include #include +#include +#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include @@ -252,7 +254,7 @@ void PhysicalPlan::addRootFinalProjectionIfNeed() } } -void PhysicalPlan::outputAndOptimize() +PhysicalPlanNodePtr PhysicalPlan::outputAndOptimize() { RUNTIME_ASSERT(!root_node, log, "root_node should be nullptr before `outputAndOptimize`"); RUNTIME_ASSERT(cur_plan_nodes.size() == 1, log, "There can only be one plan node output, but here are {}", cur_plan_nodes.size()); @@ -274,6 +276,8 @@ void PhysicalPlan::outputAndOptimize() if (!dagContext().return_executor_id) fillOrderForListBasedExecutors(dagContext(), root_node); + + return root_node; } String PhysicalPlan::toString() const @@ -282,9 +286,28 @@ String PhysicalPlan::toString() const return PhysicalPlanVisitor::visitToString(root_node); } -void PhysicalPlan::transform(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalPlan::buildBlockInputStream(DAGPipeline & pipeline, Context & context, size_t max_streams) { assert(root_node); - root_node->transform(pipeline, context, max_streams); + root_node->buildBlockInputStream(pipeline, context, max_streams); +} + +PipelinePtr PhysicalPlan::toPipeline() +{ + assert(root_node); + PipelineBuilder builder; + root_node->buildPipeline(builder); + root_node.reset(); + auto pipeline = builder.build(); + auto to_string = [&]() -> String { + FmtBuffer buffer; + pipeline->toTreeString(buffer); + return buffer.toString(); + }; + LOG_DEBUG( + log, + "build pipeline dag: \n{}", + to_string()); + return pipeline; } } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index 643bf4d2deb..cafe404a0e3 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -24,10 +24,14 @@ namespace DB { +class Pipeline; +using PipelinePtr = std::shared_ptr; +using Pipelines = std::vector; + class PhysicalPlan { public: - explicit PhysicalPlan(Context & context_, const String & req_id) + PhysicalPlan(Context & context_, const String & req_id) : context(context_) , log(Logger::get(req_id)) {} @@ -35,11 +39,13 @@ class PhysicalPlan void build(const tipb::DAGRequest * dag_request); // after outputAndOptimize, the physical plan node tree is done. - void outputAndOptimize(); + PhysicalPlanNodePtr outputAndOptimize(); String toString() const; - void transform(DAGPipeline & pipeline, Context & context, size_t max_streams); + void buildBlockInputStream(DAGPipeline & pipeline, Context & context, size_t max_streams); + + PipelinePtr toPipeline(); private: void addRootFinalProjectionIfNeed(); diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp index b16a2e544b7..413aa556fd4 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,6 +16,8 @@ #include #include #include +#include +#include #include #include #include @@ -52,6 +54,11 @@ String PhysicalPlanNode::toString() schema_to_string()); } +String PhysicalPlanNode::toSimpleString() +{ + return fmt::format("{}|{}", type.toString(), isTiDBOperator() ? executor_id : "NonTiDBOperator"); +} + void PhysicalPlanNode::finalize() { finalize(DB::toNames(schema)); @@ -69,9 +76,9 @@ void PhysicalPlanNode::recordProfileStreams(DAGPipeline & pipeline, const Contex } } -void PhysicalPlanNode::transform(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalPlanNode::buildBlockInputStream(DAGPipeline & pipeline, Context & context, size_t max_streams) { - transformImpl(pipeline, context, max_streams); + buildBlockInputStreamImpl(pipeline, context, max_streams); if (is_tidb_operator) recordProfileStreams(pipeline, context); if (is_restore_concurrency) @@ -80,4 +87,17 @@ void PhysicalPlanNode::transform(DAGPipeline & pipeline, Context & context, size restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); } } + +void PhysicalPlanNode::buildPipelineExec(PipelineExecGroupBuilder & /*group_builder*/, Context & /*context*/, size_t /*concurrency*/) +{ + throw Exception("Unsupport"); +} + +void PhysicalPlanNode::buildPipeline(PipelineBuilder & builder) +{ + assert(childrenSize() <= 1); + if (childrenSize() == 1) + children(0)->buildPipeline(builder); + builder.addPlanNode(shared_from_this()); +} } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.h b/dbms/src/Flash/Planner/PhysicalPlanNode.h index 397f7e037f9..4d903c2ba59 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.h +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -28,10 +28,16 @@ struct DAGPipeline; class Context; class DAGContext; +struct PipelineExecGroupBuilder; + +class Pipeline; +using PipelinePtr = std::shared_ptr; +class PipelineBuilder; + class PhysicalPlanNode; using PhysicalPlanNodePtr = std::shared_ptr; -class PhysicalPlanNode +class PhysicalPlanNode : public std::enable_shared_from_this { public: PhysicalPlanNode( @@ -44,8 +50,6 @@ class PhysicalPlanNode virtual PhysicalPlanNodePtr children(size_t /*i*/) const = 0; - virtual void setChild(size_t /*i*/, const PhysicalPlanNodePtr & /*new_child*/) = 0; - const PlanType & tp() const { return type; } const String & execId() const { return executor_id; } @@ -54,7 +58,11 @@ class PhysicalPlanNode virtual size_t childrenSize() const = 0; - virtual void transform(DAGPipeline & pipeline, Context & context, size_t max_streams); + virtual void buildBlockInputStream(DAGPipeline & pipeline, Context & context, size_t max_streams); + + virtual void buildPipelineExec(PipelineExecGroupBuilder & /*group_builder*/, Context & /*context*/, size_t /*concurrency*/); + + virtual void buildPipeline(PipelineBuilder & builder); virtual void finalize(const Names & parent_require) = 0; void finalize(); @@ -70,8 +78,10 @@ class PhysicalPlanNode String toString(); + String toSimpleString(); + protected: - virtual void transformImpl(DAGPipeline & /*pipeline*/, Context & /*context*/, size_t /*max_streams*/){}; + virtual void buildBlockInputStreamImpl(DAGPipeline & /*pipeline*/, Context & /*context*/, size_t /*max_streams*/){}; void recordProfileStreams(DAGPipeline & pipeline, const Context & context); diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index be71f5c3f48..8f347716b2d 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -36,6 +36,7 @@ struct PlanType TableScan = 11, MockTableScan = 12, Join = 13, + GetResult = 14, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 4cdececbba0..a61fc496334 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -52,6 +52,6 @@ void Planner::executeImpl(DAGPipeline & pipeline) physical_plan.build(&plan_source.getDAGRequest()); physical_plan.outputAndOptimize(); - physical_plan.transform(pipeline, context, max_streams); + physical_plan.buildBlockInputStream(pipeline, context, max_streams); } } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp similarity index 89% rename from dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp index c7322251fc5..90a4f441e35 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -21,9 +21,10 @@ #include #include #include +#include #include #include -#include +#include #include namespace DB @@ -85,9 +86,9 @@ PhysicalPlanNodePtr PhysicalAggregation::build( return physical_agg; } -void PhysicalAggregation::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalAggregation::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); executeExpression(pipeline, before_agg_actions, log, "before aggregation"); @@ -151,6 +152,20 @@ void PhysicalAggregation::transformImpl(DAGPipeline & pipeline, Context & contex executeExpression(pipeline, expr_after_agg, log, "expr after aggregation"); } +void PhysicalAggregation::buildPipeline(PipelineBuilder & builder) +{ + // Break the pipeline for pre-agg. + // FIXME: Should be newly created PhysicalPreAgg. + auto pre_agg_builder = builder.breakPipeline(shared_from_this()); + // Pre-agg pipeline. + child->buildPipeline(pre_agg_builder); + pre_agg_builder.build(); + // Final-agg pipeline. + // FIXME: Should be newly created PhysicalFinalAgg. + builder.addPlanNode(shared_from_this()); + throw Exception("Unsupport"); +} + void PhysicalAggregation::finalize(const Names & parent_require) { // schema.size() >= parent_require.size() diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.h similarity index 90% rename from dbms/src/Flash/Planner/plans/PhysicalAggregation.h rename to dbms/src/Flash/Planner/Plans/PhysicalAggregation.h index 4497c478d59..b97492e286f 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ #pragma once -#include +#include #include #include #include @@ -54,12 +54,14 @@ class PhysicalAggregation : public PhysicalUnary , fine_grained_shuffle(fine_grained_shuffle_) {} + void buildPipeline(PipelineBuilder & builder) override; + void finalize(const Names & parent_require) override; const Block & getSampleBlock() const override; private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; ExpressionActionsPtr before_agg_actions; Names aggregation_keys; diff --git a/dbms/src/Flash/Planner/plans/PhysicalBinary.h b/dbms/src/Flash/Planner/Plans/PhysicalBinary.h similarity index 77% rename from dbms/src/Flash/Planner/plans/PhysicalBinary.h rename to dbms/src/Flash/Planner/Plans/PhysicalBinary.h index 1ae4810e9c2..22300e2f087 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalBinary.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalBinary.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -47,15 +47,6 @@ class PhysicalBinary : public PhysicalPlanNode return i == 0 ? left : right; } - void setChild(size_t i, const PhysicalPlanNodePtr & new_child) override - { - RUNTIME_ASSERT(i <= 1, log, "child_index({}) shouldn't >= childrenSize({})", i, childrenSize()); - RUNTIME_ASSERT(new_child, log, "new_child for child_index({}) shouldn't be nullptr", i); - RUNTIME_ASSERT(new_child.get() != this, log, "new_child for child_index({}) shouldn't be itself", i); - auto & child = i == 0 ? left : right; - child = new_child; - } - size_t childrenSize() const override { return 2; }; protected: diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp similarity index 94% rename from dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp index 272dd635c6d..1dd99163426 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include @@ -58,7 +58,7 @@ PhysicalPlanNodePtr PhysicalExchangeReceiver::build( return physical_exchange_receiver; } -void PhysicalExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalExchangeReceiver::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { assert(pipeline.streams.empty()); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.h similarity index 88% rename from dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h rename to dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.h index da9459cca20..52fe31ab61f 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,7 +15,7 @@ #pragma once #include -#include +#include namespace DB { @@ -46,7 +46,7 @@ class PhysicalExchangeReceiver : public PhysicalLeaf } private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; Block sample_block; diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp similarity index 92% rename from dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp index 3ed6ef51d4e..5efae216198 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include namespace DB @@ -50,9 +50,9 @@ PhysicalPlanNodePtr PhysicalExchangeSender::build( return physical_exchange_sender; } -void PhysicalExchangeSender::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalExchangeSender::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); auto & dag_context = *context.getDAGContext(); restoreConcurrency(pipeline, dag_context.final_concurrency, log); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.h similarity index 91% rename from dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h rename to dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.h index 536cd1e3164..3829882b13b 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,7 +15,7 @@ #pragma once #include -#include +#include #include #include @@ -52,7 +52,7 @@ class PhysicalExchangeSender : public PhysicalUnary const Block & getSampleBlock() const override; private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; std::vector partition_col_ids; TiDB::TiDBCollators partition_col_collators; diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp b/dbms/src/Flash/Planner/Plans/PhysicalFilter.cpp similarity index 77% rename from dbms/src/Flash/Planner/plans/PhysicalFilter.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalFilter.cpp index ecf527f465b..fcb8a028ecb 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalFilter.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,10 +16,12 @@ #include #include #include +#include #include #include -#include +#include #include +#include namespace DB { @@ -51,13 +53,21 @@ PhysicalPlanNodePtr PhysicalFilter::build( return physical_filter; } -void PhysicalFilter::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalFilter::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, before_filter_actions, filter_column, log->identifier()); }); } +void PhysicalFilter::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +{ + auto input_header = group_builder.getCurrentHeader(); + group_builder.transform([&](auto & builder) { + builder.appendTransformOp(std::make_unique(input_header, before_filter_actions, filter_column, log->identifier())); + }); +} + void PhysicalFilter::finalize(const Names & parent_require) { Names required_output = parent_require; diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.h b/dbms/src/Flash/Planner/Plans/PhysicalFilter.h similarity index 83% rename from dbms/src/Flash/Planner/plans/PhysicalFilter.h rename to dbms/src/Flash/Planner/Plans/PhysicalFilter.h index 27d050b4a61..08f420530e2 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalFilter.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ #pragma once -#include +#include #include #include @@ -46,8 +46,10 @@ class PhysicalFilter : public PhysicalUnary const Block & getSampleBlock() const override; + void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; String filter_column; ExpressionActionsPtr before_filter_actions; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.cpp new file mode 100644 index 00000000000..56ca2f6a325 --- /dev/null +++ b/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.cpp @@ -0,0 +1,34 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include +#include + +namespace DB +{ +PhysicalPlanNodePtr PhysicalGetResultSink::build( + ResultHandler result_handler, + const PhysicalPlanNodePtr & child) +{ + return std::make_shared("get_result_sink", child->getSchema(), "", child, result_handler); +} + +void PhysicalGetResultSink::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +{ + group_builder.transform([&](auto & builder) { + builder.setSinkOp(std::make_unique(*this)); + }); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.h b/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.h new file mode 100644 index 00000000000..dbf7e56c3fb --- /dev/null +++ b/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.h @@ -0,0 +1,67 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class GetResultSinkOp; + +class PhysicalGetResultSink : public PhysicalUnary +{ +public: + static PhysicalPlanNodePtr build( + ResultHandler result_handler, + const PhysicalPlanNodePtr & child); + + PhysicalGetResultSink( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_, + ResultHandler result_handler_) + : PhysicalUnary(executor_id_, PlanType::GetResult, schema_, req_id, child_) + , result_handler(result_handler_) + { + assert(!result_handler.isIgnored()); + } + + void finalize(const Names &) override + { + throw Exception("Unsupport"); + } + + const Block & getSampleBlock() const override + { + throw Exception("Unsupport"); + } + + void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + +private: + friend class GetResultSinkOp; + + std::mutex mu; + ResultHandler result_handler; + +private: + void buildBlockInputStreamImpl(DAGPipeline &, Context &, size_t) override + { + throw Exception("Unsupport"); + } +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp b/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp similarity index 90% rename from dbms/src/Flash/Planner/plans/PhysicalJoin.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp index eeabcb38a42..c6eec3eaf6a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp @@ -1,18 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. -// -// 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. - -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -37,9 +23,10 @@ #include #include #include +#include #include #include -#include +#include #include #include #include @@ -213,18 +200,18 @@ void PhysicalJoin::buildSideTransform(DAGPipeline & build_pipeline, Context & co dag_context.addSubquery(execId(), std::move(build_query)); } -void PhysicalJoin::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalJoin::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { /// The build side needs to be transformed first. { DAGPipeline build_pipeline; - build()->transform(build_pipeline, context, max_streams); + build()->buildBlockInputStream(build_pipeline, context, max_streams); buildSideTransform(build_pipeline, context, max_streams); } { DAGPipeline & probe_pipeline = pipeline; - probe()->transform(probe_pipeline, context, max_streams); + probe()->buildBlockInputStream(probe_pipeline, context, max_streams); probeSideTransform(probe_pipeline, context); } @@ -247,6 +234,21 @@ void PhysicalJoin::doSchemaProject(DAGPipeline & pipeline) executeExpression(pipeline, schema_project, log, "remove useless column after join"); } +void PhysicalJoin::buildPipeline(PipelineBuilder & builder) +{ + // Break the pipeline for join build. + // FIXME: Should be newly created PhysicalJoinBuild. + auto join_build_builder = builder.breakPipeline(shared_from_this()); + // Join build pipeline. + build()->buildPipeline(join_build_builder); + join_build_builder.build(); + // Join probe pipeline. + probe()->buildPipeline(builder); + // FIXME: Should be newly created PhysicalJoinProbe. + builder.addPlanNode(shared_from_this()); + throw Exception("Unsupport"); +} + void PhysicalJoin::finalize(const Names & parent_require) { // schema.size() >= parent_require.size() diff --git a/dbms/src/Flash/Planner/plans/PhysicalJoin.h b/dbms/src/Flash/Planner/Plans/PhysicalJoin.h similarity index 91% rename from dbms/src/Flash/Planner/plans/PhysicalJoin.h rename to dbms/src/Flash/Planner/Plans/PhysicalJoin.h index b8c21def761..4a1f29fb5e8 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalJoin.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalJoin.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ #pragma once -#include +#include #include #include #include @@ -52,6 +52,8 @@ class PhysicalJoin : public PhysicalBinary , fine_grained_shuffle(fine_grained_shuffle_) {} + void buildPipeline(PipelineBuilder & builder) override; + void finalize(const Names & parent_require) override; const Block & getSampleBlock() const override; @@ -63,7 +65,7 @@ class PhysicalJoin : public PhysicalBinary void doSchemaProject(DAGPipeline & pipeline); - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; /// the right side is the build side. const PhysicalPlanNodePtr & probe() const { return left; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h b/dbms/src/Flash/Planner/Plans/PhysicalLeaf.h similarity index 85% rename from dbms/src/Flash/Planner/plans/PhysicalLeaf.h rename to dbms/src/Flash/Planner/Plans/PhysicalLeaf.h index 343ab66625c..b2e0ded27d8 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalLeaf.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -38,11 +38,6 @@ class PhysicalLeaf : public PhysicalPlanNode throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); } - void setChild(size_t, const PhysicalPlanNodePtr &) override - { - throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); - } - size_t childrenSize() const override { return 0; }; }; } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp b/dbms/src/Flash/Planner/Plans/PhysicalLimit.cpp similarity index 68% rename from dbms/src/Flash/Planner/plans/PhysicalLimit.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalLimit.cpp index 374a1007af5..3bc65878fe5 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalLimit.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,11 +14,14 @@ #include #include +#include #include #include #include -#include +#include +#include #include +#include namespace DB { @@ -38,9 +41,9 @@ PhysicalPlanNodePtr PhysicalLimit::build( return physical_limit; } -void PhysicalLimit::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalLimit::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, log->identifier()); }); if (pipeline.hasMoreThanOneStream()) @@ -50,6 +53,15 @@ void PhysicalLimit::transformImpl(DAGPipeline & pipeline, Context & context, siz } } +void PhysicalLimit::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +{ + auto input_header = group_builder.getCurrentHeader(); + auto global_limit = std::make_shared(input_header, limit); + group_builder.transform([&](auto & builder) { + builder.appendTransformOp(std::make_unique(global_limit, log->identifier())); + }); +} + void PhysicalLimit::finalize(const Names & parent_require) { child->finalize(parent_require); diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.h b/dbms/src/Flash/Planner/Plans/PhysicalLimit.h similarity index 80% rename from dbms/src/Flash/Planner/plans/PhysicalLimit.h rename to dbms/src/Flash/Planner/Plans/PhysicalLimit.h index 24f6fe83044..b8e026a6b49 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalLimit.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ #pragma once -#include +#include #include namespace DB @@ -42,8 +42,10 @@ class PhysicalLimit : public PhysicalUnary const Block & getSampleBlock() const override; + void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; size_t limit; }; diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.cpp similarity index 78% rename from dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.cpp index 0765ed82d6c..b5fef14a1df 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,10 +16,12 @@ #include #include #include +#include #include #include -#include +#include #include +#include namespace DB { @@ -55,12 +57,21 @@ PhysicalPlanNodePtr PhysicalMockExchangeReceiver::build( return physical_mock_exchange_receiver; } -void PhysicalMockExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) +void PhysicalMockExchangeReceiver::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) { assert(pipeline.streams.empty()); pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); } +void PhysicalMockExchangeReceiver::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +{ + group_builder.init(mock_streams.size()); + size_t i = 0; + group_builder.transform([&](auto & builder) { + builder.setSourceOp(std::make_unique(mock_streams[i++])); + }); +} + void PhysicalMockExchangeReceiver::finalize(const Names & parent_require) { FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.h similarity index 84% rename from dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h rename to dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.h index 3311a0a7d39..26b2c72caed 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,7 +15,7 @@ #pragma once #include -#include +#include #include namespace DB @@ -49,8 +49,10 @@ class PhysicalMockExchangeReceiver : public PhysicalLeaf size_t getSourceNum() const { return source_num; }; + void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + private: - void transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; private: Block sample_block; diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.cpp similarity index 85% rename from dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.cpp index 5e6e83eeb0b..490ec164ae6 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include namespace DB @@ -37,9 +37,9 @@ PhysicalPlanNodePtr PhysicalMockExchangeSender::build( return physical_mock_exchange_sender; } -void PhysicalMockExchangeSender::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalMockExchangeSender::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, log->identifier()); }); } diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.h similarity index 80% rename from dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h rename to dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.h index bfebf34c1ea..445222d094d 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ #pragma once -#include +#include #include #include @@ -40,7 +40,9 @@ class PhysicalMockExchangeSender : public PhysicalUnary const Block & getSampleBlock() const override; + void buildPipelineExec(PipelineExecGroupBuilder & /*group_builder*/, Context & /*context*/, size_t /*concurrency*/) override {} + private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp similarity index 88% rename from dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp index 74d470ebf1b..8bdfe53d504 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -17,10 +17,12 @@ #include #include #include +#include #include #include -#include +#include #include +#include namespace DB { @@ -101,12 +103,21 @@ PhysicalPlanNodePtr PhysicalMockTableScan::build( return physical_mock_table_scan; } -void PhysicalMockTableScan::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) +void PhysicalMockTableScan::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) { assert(pipeline.streams.empty()); pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); } +void PhysicalMockTableScan::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +{ + group_builder.init(mock_streams.size()); + size_t i = 0; + group_builder.transform([&](auto & builder) { + builder.setSourceOp(std::make_unique(mock_streams[i++])); + }); +} + void PhysicalMockTableScan::finalize(const Names & parent_require) { FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.h b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h similarity index 86% rename from dbms/src/Flash/Planner/plans/PhysicalMockTableScan.h rename to dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h index 6df9e41d801..4a961c8904b 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include namespace DB @@ -48,6 +48,8 @@ class PhysicalMockTableScan : public PhysicalLeaf const Block & getSampleBlock() const override; + void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void initStreams(Context & context); // for delta-merge test @@ -62,7 +64,7 @@ class PhysicalMockTableScan : public PhysicalLeaf void updateStreams(Context & context); private: - void transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; private: PushDownFilter push_down_filter; diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp similarity index 88% rename from dbms/src/Flash/Planner/plans/PhysicalProjection.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp index 3543da76f2d..18f7fcb81c7 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -18,10 +18,12 @@ #include #include #include +#include #include #include -#include +#include #include +#include namespace DB { @@ -134,13 +136,23 @@ PhysicalPlanNodePtr PhysicalProjection::buildRootFinal( return physical_projection; } -void PhysicalProjection::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalProjection::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); executeExpression(pipeline, project_actions, log, extra_info); } +void PhysicalProjection::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +{ + if (project_actions && !project_actions->getActions().empty()) + { + group_builder.transform([&](auto & builder) { + builder.appendTransformOp(std::make_unique(project_actions, log->identifier())); + }); + } +} + void PhysicalProjection::finalize(const Names & parent_require) { FinalizeHelper::checkSampleBlockContainsParentRequire(getSampleBlock(), parent_require); diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.h b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h similarity index 88% rename from dbms/src/Flash/Planner/plans/PhysicalProjection.h rename to dbms/src/Flash/Planner/Plans/PhysicalProjection.h index bd5e8140f28..ab1e1936a86 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ #pragma once -#include +#include #include #include @@ -66,8 +66,10 @@ class PhysicalProjection : public PhysicalUnary const Block & getSampleBlock() const override; + void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; const String extra_info; diff --git a/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp similarity index 95% rename from dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp index 9351215951c..ffae24ccc9f 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include namespace DB @@ -51,7 +51,7 @@ PhysicalPlanNodePtr PhysicalTableScan::build( return physical_table_scan; } -void PhysicalTableScan::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalTableScan::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { assert(pipeline.streams.empty()); diff --git a/dbms/src/Flash/Planner/plans/PhysicalTableScan.h b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h similarity index 89% rename from dbms/src/Flash/Planner/plans/PhysicalTableScan.h rename to dbms/src/Flash/Planner/Plans/PhysicalTableScan.h index a4795f920c4..3998e531b82 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTableScan.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ #include #include -#include +#include #include namespace DB @@ -45,7 +45,7 @@ class PhysicalTableScan : public PhysicalLeaf const String & getPushDownFilterId() const; private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; void buildProjection(DAGPipeline & pipeline, const NamesAndTypes & storage_schema); private: diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/Plans/PhysicalTopN.cpp similarity index 91% rename from dbms/src/Flash/Planner/plans/PhysicalTopN.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalTopN.cpp index 10f25c3f3cd..b92b68cc975 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalTopN.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include namespace DB @@ -56,9 +56,9 @@ PhysicalPlanNodePtr PhysicalTopN::build( return physical_top_n; } -void PhysicalTopN::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalTopN::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); executeExpression(pipeline, before_sort_actions, log, "before TopN"); diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.h b/dbms/src/Flash/Planner/Plans/PhysicalTopN.h similarity index 90% rename from dbms/src/Flash/Planner/plans/PhysicalTopN.h rename to dbms/src/Flash/Planner/Plans/PhysicalTopN.h index bfabb5f4261..1de947b50f9 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalTopN.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,7 +15,7 @@ #pragma once #include -#include +#include #include #include @@ -50,7 +50,7 @@ class PhysicalTopN : public PhysicalUnary const Block & getSampleBlock() const override; private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; SortDescription order_descr; ExpressionActionsPtr before_sort_actions; diff --git a/dbms/src/Flash/Planner/plans/PhysicalUnary.h b/dbms/src/Flash/Planner/Plans/PhysicalUnary.h similarity index 76% rename from dbms/src/Flash/Planner/plans/PhysicalUnary.h rename to dbms/src/Flash/Planner/Plans/PhysicalUnary.h index e418653135f..47f32643be4 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalUnary.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalUnary.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -45,14 +45,6 @@ class PhysicalUnary : public PhysicalPlanNode return child; } - void setChild(size_t i, const PhysicalPlanNodePtr & new_child) override - { - RUNTIME_ASSERT(i == 0, log, "child_index({}) shouldn't >= childrenSize({})", i, childrenSize()); - RUNTIME_ASSERT(new_child, log, "new_child for child_index({}) shouldn't be nullptr", i); - RUNTIME_ASSERT(new_child.get() != this, log, "new_child for child_index({}) shouldn't be itself", i); - child = new_child; - } - size_t childrenSize() const override { return 1; }; protected: diff --git a/dbms/src/Flash/Planner/plans/PhysicalWindow.cpp b/dbms/src/Flash/Planner/Plans/PhysicalWindow.cpp similarity index 93% rename from dbms/src/Flash/Planner/plans/PhysicalWindow.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalWindow.cpp index 9a38a14ff3d..c6134574571 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalWindow.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindow.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include namespace DB @@ -61,9 +61,9 @@ PhysicalPlanNodePtr PhysicalWindow::build( return physical_window; } -void PhysicalWindow::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalWindow::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); executeExpression(pipeline, window_description.before_window, log, "before window"); window_description.fillArgColumnNumbers(); diff --git a/dbms/src/Flash/Planner/plans/PhysicalWindow.h b/dbms/src/Flash/Planner/Plans/PhysicalWindow.h similarity index 90% rename from dbms/src/Flash/Planner/plans/PhysicalWindow.h rename to dbms/src/Flash/Planner/Plans/PhysicalWindow.h index 6d37e11a597..05a7b5a6eca 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalWindow.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindow.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,7 +15,7 @@ #pragma once #include -#include +#include #include #include @@ -49,7 +49,7 @@ class PhysicalWindow : public PhysicalUnary const Block & getSampleBlock() const override; private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; private: WindowDescription window_description; diff --git a/dbms/src/Flash/Planner/plans/PhysicalWindowSort.cpp b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.cpp similarity index 89% rename from dbms/src/Flash/Planner/plans/PhysicalWindowSort.cpp rename to dbms/src/Flash/Planner/Plans/PhysicalWindowSort.cpp index 2f58fb95e58..a35578fa3e2 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalWindowSort.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include namespace DB @@ -50,9 +50,9 @@ PhysicalPlanNodePtr PhysicalWindowSort::build( return physical_window_sort; } -void PhysicalWindowSort::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +void PhysicalWindowSort::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) { - child->transform(pipeline, context, max_streams); + child->buildBlockInputStream(pipeline, context, max_streams); orderStreams(pipeline, max_streams, order_descr, 0, fine_grained_shuffle.enable(), context, log); } diff --git a/dbms/src/Flash/Planner/plans/PhysicalWindowSort.h b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h similarity index 90% rename from dbms/src/Flash/Planner/plans/PhysicalWindowSort.h rename to dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h index 39ea96abbd2..3ad4251a19a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalWindowSort.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,7 +16,7 @@ #include #include -#include +#include #include namespace DB @@ -49,7 +49,7 @@ class PhysicalWindowSort : public PhysicalUnary const Block & getSampleBlock() const override; private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; private: SortDescription order_descr; diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index d2798548806..5bb71f219a7 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -109,7 +109,7 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest BlockInputStreamPtr final_stream; { DAGPipeline pipeline; - physical_plan.transform(pipeline, context.context, max_streams); + physical_plan.buildBlockInputStream(pipeline, context.context, max_streams); executeCreatingSets(pipeline, context.context, max_streams, log); final_stream = pipeline.firstStream(); FmtBuffer fb; diff --git a/dbms/src/Flash/executeQuery.cpp b/dbms/src/Flash/executeQuery.cpp index 9a42ead72a4..dba7f904082 100644 --- a/dbms/src/Flash/executeQuery.cpp +++ b/dbms/src/Flash/executeQuery.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,9 +14,14 @@ #include #include +#include #include #include #include +#include +#include +#include +#include #include #include #include @@ -35,6 +40,7 @@ namespace FailPoints { extern const char random_interpreter_failpoint[]; } // namespace FailPoints + namespace { void prepareForExecute(Context & context) @@ -67,7 +73,7 @@ ProcessList::EntryPtr getProcessListEntry(Context & context, DAGContext & dag_co } } -BlockIO executeDAG(IQuerySource & dag, Context & context, bool internal) +BlockIO doExecuteAsBlockIO(IQuerySource & dag, Context & context, bool internal) { RUNTIME_ASSERT(context.getDAGContext()); auto & dag_context = *context.getDAGContext(); @@ -98,24 +104,63 @@ BlockIO executeDAG(IQuerySource & dag, Context & context, bool internal) return res; } + +std::optional executeAsPipeline(Context & context, bool internal) +{ + RUNTIME_ASSERT(context.getDAGContext()); + auto & dag_context = *context.getDAGContext(); + const auto & logger = dag_context.log; + RUNTIME_ASSERT(logger); + + if (!TaskScheduler::instance || !Pipeline::isSupported(*dag_context.dag_request)) + return {}; + + prepareForExecute(context); + + ProcessList::EntryPtr process_list_entry; + if (likely(!internal)) + { + process_list_entry = getProcessListEntry(context, dag_context); + logQuery(dag_context.dummy_query_string, context, logger); + } + + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_interpreter_failpoint); + + PhysicalPlan physical_plan{context, logger->identifier()}; + physical_plan.build(dag_context.dag_request); + physical_plan.outputAndOptimize(); + auto pipeline = physical_plan.toPipeline(); + auto executor = std::make_unique(process_list_entry, context, pipeline); + if (likely(!internal)) + LOG_DEBUG(logger, fmt::format("Query pipeline:\n{}", executor->toString())); + return {std::move(executor)}; +} } // namespace -BlockIO executeQuery(Context & context, bool internal) +BlockIO executeAsBlockIO(Context & context, bool internal) { if (context.getSettingsRef().enable_planner) { PlanQuerySource plan(context); - return executeDAG(plan, context, internal); + return doExecuteAsBlockIO(plan, context, internal); } else { DAGQuerySource dag(context); - return executeDAG(dag, context, internal); + return doExecuteAsBlockIO(dag, context, internal); } } QueryExecutorPtr queryExecute(Context & context, bool internal) { - return std::make_unique(executeQuery(context, internal)); + // now only support pipeline model in executor/interpreter test. + if ((context.isExecutorTest() || context.isInterpreterTest()) + && context.getSettingsRef().enable_planner + && context.getSettingsRef().enable_pipeline) + { + if (auto res = executeAsPipeline(context, internal); res) + return std::move(*res); + } + return std::make_unique(executeAsBlockIO(context, internal)); } } // namespace DB diff --git a/dbms/src/Flash/executeQuery.h b/dbms/src/Flash/executeQuery.h index 2421b50a34b..b0541a9941a 100644 --- a/dbms/src/Flash/executeQuery.h +++ b/dbms/src/Flash/executeQuery.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,13 +14,15 @@ #pragma once -#include #include #include -#include namespace DB { -BlockIO executeQuery(Context & context, bool internal = false); +class Context; + +// Use BlockInputStream-based pull model directly. +BlockIO executeAsBlockIO(Context & context, bool internal = false); + QueryExecutorPtr queryExecute(Context & context, bool internal = false); } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_executors_with_dm.cpp b/dbms/src/Flash/tests/gtest_executors_with_dm.cpp index a4c802547ac..f2db4ee6acc 100644 --- a/dbms/src/Flash/tests/gtest_executors_with_dm.cpp +++ b/dbms/src/Flash/tests/gtest_executors_with_dm.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -64,11 +64,6 @@ class ExecutorsWithDMTestRunner : public DB::tests::ExecutorTest toNullableVec("col9", col_string)}); } - void TearDown() override - { - context.mockStorage()->clear(); - } - ColumnWithInt64 col_id{1, 2, 3, 4, 5, 6, 7, 8, 9}; ColumnWithNullableInt8 col_tinyint{1, 2, 3, {}, {}, 0, 0, -1, -2}; ColumnWithNullableInt16 col_smallint{2, 3, {}, {}, 0, -1, -2, 4, 0}; diff --git a/dbms/src/Flash/tests/gtest_filter_executor.cpp b/dbms/src/Flash/tests/gtest_filter_executor.cpp index 5794faedd70..96ad0094395 100644 --- a/dbms/src/Flash/tests/gtest_filter_executor.cpp +++ b/dbms/src/Flash/tests/gtest_filter_executor.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -52,6 +52,19 @@ class FilterExecutorTestRunner : public DB::tests::ExecutorTest toNullableVec("double_col", {0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7}), toNullableVec("string_col", {"", "a", "1", "0", "ab", " ", "\t", "\n"}), }); + + // with 200 rows. + std::vector::FieldType>> key(200); + std::vector> value(200); + for (size_t i = 0; i < 200; ++i) + { + key[i] = i % 15; + value[i] = {fmt::format("val_{}", i)}; + } + context.addMockTable( + {"test_db", "big_table"}, + {{"key", TiDB::TP::TypeLong}, {"value", TiDB::TP::TypeString}}, + {toNullableVec("key", key), toNullableVec("value", value)}); } }; @@ -252,6 +265,18 @@ try } CATCH +TEST_F(FilterExecutorTestRunner, BigTable) +try +{ + auto request = context + .scan("test_db", "big_table") + .filter(gt(col("key"), lit(Field(static_cast(7))))) + .build(context); + auto expect = executeStreams(request, 1); + executeAndAssertColumnsEqual(request, expect); +} +CATCH + TEST_F(FilterExecutorTestRunner, PushDownFilter) try { @@ -336,7 +361,5 @@ Expression: } CATCH -/// TODO: more functions. - } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_limit_executor.cpp b/dbms/src/Flash/tests/gtest_limit_executor.cpp index 00c48d8d779..79dc56ee6b5 100644 --- a/dbms/src/Flash/tests/gtest_limit_executor.cpp +++ b/dbms/src/Flash/tests/gtest_limit_executor.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -19,10 +19,11 @@ namespace DB { namespace tests { - class LimitExecutorTestRunner : public DB::tests::ExecutorTest { public: + static constexpr size_t big_table_rows = 200; + void initializeContext() override { ExecutorTest::initializeContext(); @@ -30,6 +31,13 @@ class LimitExecutorTestRunner : public DB::tests::ExecutorTest context.addMockTable({db_name, table_name}, {{col_name, TiDB::TP::TypeString}}, {toNullableVec(col_name, col0)}); + + ColumnWithNullableString col; + for (size_t i = 0; i < big_table_rows; ++i) + col.emplace_back("a"); + context.addMockTable({"test", "bigtable"}, + {{"col", TiDB::TP::TypeString}}, + {toNullableVec("col", col)}); } std::shared_ptr buildDAGRequest(size_t limit_num) @@ -65,9 +73,9 @@ try else expect_cols = {toNullableVec(col_name, ColumnWithNullableString(col0.begin(), col0.begin() + limit_num))}; - WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN + WRAP_FOR_TEST_BEGIN ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols); - WRAP_FOR_DIS_ENABLE_PLANNER_END + WRAP_FOR_TEST_END executeAndAssertRowsEqual(request, std::min(limit_num, col_data_num)); } @@ -83,5 +91,16 @@ try } CATCH +TEST_F(LimitExecutorTestRunner, BigTable) +try +{ + for (size_t limit = 1; limit < 2 * big_table_rows; limit += 7) + { + auto request = context.scan("test", "bigtable").limit(limit).build(context); + executeAndAssertRowsEqual(request, std::min(limit, big_table_rows)); + } +} +CATCH + } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_pipeline_interpreter.cpp b/dbms/src/Flash/tests/gtest_pipeline_interpreter.cpp new file mode 100644 index 00000000000..022c06d0a20 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_pipeline_interpreter.cpp @@ -0,0 +1,1034 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +namespace DB +{ +namespace tests +{ +class PipelineInterpreterExecuteTest : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + + enablePlanner(true); + enablePipeline(true); + + context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); + context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); + context.addMockTable({"test_db", "r_table"}, {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addMockTable({"test_db", "l_table"}, {{"l_a", TiDB::TP::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeReceiver("sender_1", {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); + context.addExchangeReceiver("sender_l", {{"l_a", TiDB::TP::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeReceiver("sender_r", {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + } +}; + +TEST_F(PipelineInterpreterExecuteTest, StrangeQuery) +try +{ + auto request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .filter(eq(col("s1"), col("s3"))) + .filter(eq(col("s1"), col("s2"))) + .build(context); + { + String expected = "pipeline#0: MockTableScan|table_scan_0 -> Filter|selection_1 -> Filter|selection_2 -> Filter|selection_3 -> Projection|NonTiDBOperator"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .limit(10) + .limit(9) + .limit(8) + .build(context); + { + String expected = "pipeline#0: MockTableScan|table_scan_0 -> Limit|limit_1 -> Limit|limit_2 -> Limit|limit_3 -> Projection|NonTiDBOperator"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .topN("s3", false, 10) + .topN("s2", false, 9) + .topN("s1", false, 8) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + MergeSorting, limit = 8 + Union: + PartialSorting x 10: limit = 8 + SharedQuery: + MergeSorting, limit = 9 + Union: + PartialSorting x 10: limit = 9 + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, SingleQueryBlock) +try +{ + auto request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .filter(eq(col("s2"), col("s3"))) + .topN("s2", false, 10) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Filter + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Filter + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .filter(eq(col("s2"), col("s3"))) + .limit(10) + .build(context); + + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Filter + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Filter + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, ParallelQuery) +try +{ + /// executor with table scan + auto request = context.scan("test_db", "test_table_1") + .limit(10) + .build(context); + { + String expected = "pipeline#0: MockTableScan|table_scan_0 -> Limit|limit_1 -> Projection|NonTiDBOperator"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .build(context); + { + String expected = "pipeline#0: MockTableScan|table_scan_0 -> Projection|project_1 -> Projection|NonTiDBOperator"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected = R"( +Expression: + Expression: + Aggregating + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 5, final: true + MockTableScan x 5)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .build(context); + { + String expected = R"( +Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 5: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .build(context); + { + String expected = "pipeline#0: MockTableScan|table_scan_0 -> Filter|selection_1 -> Projection|NonTiDBOperator"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + /// other cases + request = context.scan("test_db", "test_table_1") + .limit(10) + .project({"s1", "s2", "s3"}) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + SharedQuery x 10: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +Expression: + Expression: + Aggregating + Limit, limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .project({"s1", "s2", "s3"}) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + SharedQuery x 10: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +Expression: + Expression: + Aggregating + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .project({"s2", "s3"}) + .aggregation({Max(col("s2"))}, {col("s3")}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + MockTableScan x 10)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +Expression: + Expression: + Aggregating + Expression: + Expression: + Aggregating + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected = R"( +Union: + MockExchangeSender x 10 + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + MockTableScan x 10)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +MockExchangeSender + Expression: + Expression: + Aggregating + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected = R"( +Union: + MockExchangeSender x 10 + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +MockExchangeSender + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .limit(10) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected = "pipeline#0: MockTableScan|table_scan_0 -> Limit|limit_1 -> Projection|NonTiDBOperator -> MockExchangeSender|exchange_sender_2"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + request = table1.join(table2.limit(1), tipb::JoinType::TypeLeftOuterJoin, {col("join_c")}).build(context); + { + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + SharedQuery: + Limit, limit = 1 + Union: + Limit x 10, limit = 1 + MockTableScan + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, MultipleQueryBlockWithSource) +try +{ + auto request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project({"s1"}) + .build(context); + { + String expected = "pipeline#0: MockTableScan|table_scan_0 -> Projection|project_1 -> Projection|project_2 -> Projection|project_3 -> Projection|NonTiDBOperator"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .aggregation({Max(col("s1"))}, {col("s1"), col("s2")}) + .project({"max(s1)", "s1", "s2"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .aggregation({Max(col("s1"))}, {col("s1"), col("s2")}) + .project({"max(s1)", "s1", "s2"}) + .filter(eq(col("s1"), col("s2"))) + .project({"max(s1)", "s1"}) + .limit(10) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + Filter + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.receive("sender_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project({"s1"}) + .build(context); + { + String expected = "pipeline#0: MockExchangeReceiver|exchange_receiver_0 -> Projection|project_1 -> Projection|project_2 -> Projection|project_3 -> Projection|NonTiDBOperator"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.receive("sender_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project({"s1"}) + .exchangeSender(tipb::Broadcast) + .build(context); + { + String expected = "pipeline#0: MockExchangeReceiver|exchange_receiver_0 -> Projection|project_1 -> Projection|project_2 -> Projection|project_3 -> Projection|NonTiDBOperator -> MockExchangeSender|exchange_sender_4"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, Window) +try +{ + auto request = context + .scan("test_db", "test_table") + .sort({{"s1", true}, {"s2", false}}, true) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table") + .sort({{"s1", true}, {"s2", false}}, true) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) + .project({"s1", "s2", "RowNumber()"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .sort({{"s1", true}, {"s2", false}}, true) + .project({"s1", "s2", "s3"}) + .window(RowNumber(), {"s1", true}, {"s1", false}, buildDefaultRowsFrame()) + .project({"s1", "s2", "s3", "RowNumber()"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Union: + SharedQuery x 10: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, FineGrainedShuffle) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + auto request = context + .receive("sender_1", enable) + .sort({{"s1", true}, {"s2", false}}, true, enable) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame(), enable) + .build(context); + { + String expected = R"( +Union: + Expression x 8: + Expression: + Window: , function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting: , limit = 0 + PartialSorting: : limit = 0 + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + auto topn_request = context + .receive("sender_1") + .topN("s2", false, 10) + .build(context); + String topn_expected = R"( +Union: + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(topn_expected, topn_request, 10); + + // fine-grained shuffle is disabled. + request = context + .receive("sender_1", disable) + .sort({{"s1", true}, {"s2", false}}, true, disable) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame(), disable) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + topn_request = context + .receive("sender_1") + .topN("s2", false, 10) + .build(context); + ASSERT_BLOCKINPUTSTREAM_EQAUL(topn_expected, topn_request, 10); +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, FineGrainedShuffleJoin) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + { + // Join Source. + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r", enable); + + auto request = receiver1.join( + receiver2, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}, + enable) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 8: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + { + // Join Source. + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r", disable); + + auto request = receiver1.join( + receiver2, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}, + disable) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, FineGrainedShuffleAgg) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + { + DAGRequestBuilder receiver1 = context.receive("sender_1", enable); + auto request = receiver1 + .aggregation({Max(col("s1"))}, {col("s2")}, enable) + .build(context); + String expected = R"( +Union: + Expression x 8: + Expression: + Aggregating: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + DAGRequestBuilder receiver1 = context.receive("sender_1", disable); + auto request = receiver1 + .aggregation({Max(col("s1"))}, {col("s2")}, disable) + .build(context); + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, Join) +try +{ + // TODO: Find a way to write the request easier. + { + // Join Source. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + DAGRequestBuilder table3 = context.scan("test_db", "r_table"); + DAGRequestBuilder table4 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2.join( + table3.join(table4, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union x 2: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // only join + ExchangeReceiver + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + DAGRequestBuilder receiver3 = context.receive("sender_l"); + DAGRequestBuilder receiver4 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union x 2: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // join + receiver + sender + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + DAGRequestBuilder receiver3 = context.receive("sender_l"); + DAGRequestBuilder receiver4 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}) + .exchangeSender(tipb::PassThrough) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union x 2: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver + Union: + MockExchangeSender x 10 + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, JoinThenAgg) +try +{ + { + // Left Join. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // Right Join + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + tipb::JoinType::TypeRightOuterJoin, + {col("join_c")}) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Right + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + HashJoinProbe: + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // Right join + receiver + sender + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2, + tipb::JoinType::TypeRightOuterJoin, + {col("join_c")}) + .aggregation({Sum(col("r_a"))}, {col("join_c")}) + .limit(10) + .exchangeSender(tipb::PassThrough) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 20: , join_kind = Right + Expression: + Expression: + MockExchangeReceiver + Union: + MockExchangeSender x 20 + Expression: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 20, limit = 10 + Expression: + SharedQuery: + ParallelAggregating, max_threads: 20, final: true + Expression x 20: + Expression: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 20); + } +} +CATCH + +TEST_F(PipelineInterpreterExecuteTest, ListBase) +try +{ + { + auto request = context + .scan("test_db", "test_table") + .filter(eq(col("s1"), col("s2"))) + .aggregation(Max(col("s1")), col("s2")) + .filter(eq(col("s2"), lit(Field("1", 1)))) + .limit(10) + .build(context, DAGRequestType::list); + String expected = R"( +Expression: + Limit, limit = 10 + Filter + Expression: + Aggregating + Expression: + Filter + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + { + auto request = context + .scan("test_db", "test_table") + .filter(eq(col("s1"), col("s2"))) + .aggregation(Max(col("s1")), col("s2")) + .filter(eq(col("s2"), lit(Field("1", 1)))) + .topN("s2", false, 10) + .build(context, DAGRequestType::list); + String expected = R"( +Union: + Expression x 20: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 20: limit = 10 + Expression: + Filter + Expression: + SharedQuery: + ParallelAggregating, max_threads: 20, final: true + Expression x 20: + Filter + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 20); + } +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp index 3492ef101a3..f8edd651008 100644 --- a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -27,6 +27,7 @@ class PlannerInterpreterExecuteTest : public DB::tests::ExecutorTest ExecutorTest::initializeContext(); enablePlanner(true); + enablePipeline(false); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); diff --git a/dbms/src/Flash/tests/gtest_projection_executor.cpp b/dbms/src/Flash/tests/gtest_projection_executor.cpp index b4d5e4e1aa0..bff714e1887 100644 --- a/dbms/src/Flash/tests/gtest_projection_executor.cpp +++ b/dbms/src/Flash/tests/gtest_projection_executor.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -58,6 +58,19 @@ class ProjectionExecutorTestRunner : public DB::tests::ExecutorTest toNullableVec("s2", {1, 1, 1, 1, 1}), toNullableVec("s3", {1, 1, 1, 1, 1}), toNullableVec("s4", {1, 1, 1, 1, 1})}); + + // with 200 rows. + std::vector::FieldType>> key(200); + std::vector> value(200); + for (size_t i = 0; i < 200; ++i) + { + key[i] = i % 15; + value[i] = {fmt::format("val_{}", i)}; + } + context.addMockTable( + {"test_db", "big_table"}, + {{"key", TiDB::TP::TypeLongLong}, {"value", TiDB::TP::TypeString}}, + {toNullableVec("key", key), toNullableVec("value", value)}); } template @@ -421,5 +434,17 @@ try } CATCH +TEST_F(ProjectionExecutorTestRunner, BigTable) +try +{ + auto request = context + .scan("test_db", "big_table") + .project({plusInt(col("key"), lit(Field(static_cast(7)))), concat(col("value"), col("value"))}) + .build(context); + auto expect = executeStreams(request, 1); + executeAndAssertColumnsEqual(request, expect); +} +CATCH + } // namespace tests } // namespace DB diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 44d6c3e887c..dc79161f356 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -298,7 +298,9 @@ struct Settings M(SettingUInt64, manual_compact_more_until_ms, 60000, "Continuously compact more segments until reaching specified elapsed time. If 0 is specified, only one segment will be compacted each round.") \ M(SettingUInt64, max_spilled_size_per_spill, 1024ULL * 1024 * 1024, "Max spilled data size per spill, 1GB as the default value.") \ \ - M(SettingBool, enable_planner, true, "Enable planner") + M(SettingBool, enable_planner, true, "Enable planner") \ + M(SettingBool, enable_pipeline, false, "Enable pipeline model") \ + M(SettingUInt64, pipeline_task_thread_pool_size, 0, "The size of task thread pool. 0 means using number_of_logical_cpu_cores.") // clang-format on #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) TYPE NAME{DEFAULT}; diff --git a/dbms/src/Operators/BlockInputStreamSourceOp.cpp b/dbms/src/Operators/BlockInputStreamSourceOp.cpp new file mode 100644 index 00000000000..44fd9426069 --- /dev/null +++ b/dbms/src/Operators/BlockInputStreamSourceOp.cpp @@ -0,0 +1,41 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +namespace DB +{ +BlockInputStreamSourceOp::BlockInputStreamSourceOp( + const BlockInputStreamPtr & impl_) + : impl(impl_) +{ + impl->readPrefix(); + setHeader(impl->getHeader()); +} + +OperatorStatus BlockInputStreamSourceOp::readImpl(Block & block) +{ + if (unlikely(finished)) + return OperatorStatus::HAS_OUTPUT; + + block = impl->read(); + if (unlikely(!block)) + { + impl->readSuffix(); + finished = true; + } + return OperatorStatus::HAS_OUTPUT; +} +} // namespace DB diff --git a/dbms/src/Operators/BlockInputStreamSourceOp.h b/dbms/src/Operators/BlockInputStreamSourceOp.h new file mode 100644 index 00000000000..d61f0f77451 --- /dev/null +++ b/dbms/src/Operators/BlockInputStreamSourceOp.h @@ -0,0 +1,43 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class IBlockInputStream; +using BlockInputStreamPtr = std::shared_ptr; + +// Wrap the BlockInputStream of pull model as the source operator of push model. +// Now it is used by `PhysicalMockExchangeReceiver/PhysicalMockTableScan` which are only used in unit test. +class BlockInputStreamSourceOp : public SourceOp +{ +public: + explicit BlockInputStreamSourceOp(const BlockInputStreamPtr & impl_); + + String getName() const override + { + return "BlockInputStreamSourceOp"; + } + + OperatorStatus readImpl(Block & block) override; + +private: + BlockInputStreamPtr impl; + bool finished = false; +}; +} // namespace DB diff --git a/dbms/src/Operators/CMakeLists.txt b/dbms/src/Operators/CMakeLists.txt new file mode 100644 index 00000000000..09b08aa6a4c --- /dev/null +++ b/dbms/src/Operators/CMakeLists.txt @@ -0,0 +1,17 @@ +# Copyright 2023 PingCAP, Ltd. +# +# 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. + +if (ENABLE_TESTS) + add_subdirectory (tests EXCLUDE_FROM_ALL) +endif () diff --git a/dbms/src/Operators/ExpressionTransformOp.cpp b/dbms/src/Operators/ExpressionTransformOp.cpp new file mode 100644 index 00000000000..f9c69257677 --- /dev/null +++ b/dbms/src/Operators/ExpressionTransformOp.cpp @@ -0,0 +1,31 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +namespace DB +{ +OperatorStatus ExpressionTransformOp::transformImpl(Block & block) +{ + if (likely(block)) + expression->execute(block); + return OperatorStatus::HAS_OUTPUT; +} + +void ExpressionTransformOp::transformHeaderImpl(Block & header_) +{ + expression->execute(header_); +} +} // namespace DB diff --git a/dbms/src/Operators/ExpressionTransformOp.h b/dbms/src/Operators/ExpressionTransformOp.h new file mode 100644 index 00000000000..918e90d76bf --- /dev/null +++ b/dbms/src/Operators/ExpressionTransformOp.h @@ -0,0 +1,48 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class ExpressionActions; +using ExpressionActionsPtr = std::shared_ptr; + +class ExpressionTransformOp : public TransformOp +{ +public: + ExpressionTransformOp( + const ExpressionActionsPtr & expression_, + const String & req_id) + : expression(expression_) + , log(Logger::get(req_id)) + {} + + String getName() const override + { + return "ExpressionTransformOp"; + } + + OperatorStatus transformImpl(Block & block) override; + + void transformHeaderImpl(Block & header_) override; + +private: + ExpressionActionsPtr expression; + const LoggerPtr log; +}; +} // namespace DB diff --git a/dbms/src/Operators/FilterTransformOp.cpp b/dbms/src/Operators/FilterTransformOp.cpp new file mode 100644 index 00000000000..526c50a3424 --- /dev/null +++ b/dbms/src/Operators/FilterTransformOp.cpp @@ -0,0 +1,39 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include + +namespace DB +{ +OperatorStatus FilterTransformOp::transformImpl(Block & block) +{ + if (unlikely(filter_transform_action.alwaysFalse())) + { + block = {}; + return OperatorStatus::HAS_OUTPUT; + } + + if (likely(block)) + return filter_transform_action.transform(block, /*child_filter=*/nullptr) + ? OperatorStatus::HAS_OUTPUT + : OperatorStatus::NEED_INPUT; + + return OperatorStatus::HAS_OUTPUT; +} + +void FilterTransformOp::transformHeaderImpl(Block & header_) +{ + header_ = filter_transform_action.getHeader(); +} +} // namespace DB diff --git a/dbms/src/Operators/FilterTransformOp.h b/dbms/src/Operators/FilterTransformOp.h new file mode 100644 index 00000000000..dbc1714678c --- /dev/null +++ b/dbms/src/Operators/FilterTransformOp.h @@ -0,0 +1,48 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +#include + +namespace DB +{ +class FilterTransformOp : public TransformOp +{ +public: + FilterTransformOp( + const Block & input_header, + const ExpressionActionsPtr & expression, + const String & filter_column_name, + const String & req_id) + : filter_transform_action(input_header, expression, filter_column_name) + , log(Logger::get(req_id)) + {} + + String getName() const override + { + return "FilterTransformOp"; + } + + OperatorStatus transformImpl(Block & block) override; + + void transformHeaderImpl(Block & header_) override; + +private: + FilterTransformAction filter_transform_action; + const LoggerPtr log; +}; +} // namespace DB diff --git a/dbms/src/Operators/GetResultSinkOp.cpp b/dbms/src/Operators/GetResultSinkOp.cpp new file mode 100644 index 00000000000..f73c8c718b8 --- /dev/null +++ b/dbms/src/Operators/GetResultSinkOp.cpp @@ -0,0 +1,29 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +namespace DB +{ +OperatorStatus GetResultSinkOp::writeImpl(Block && block) +{ + if (!block) + return OperatorStatus::FINISHED; + + std::lock_guard lock(physical_sink.mu); + physical_sink.result_handler(block); + return OperatorStatus::NEED_INPUT; +} +} // namespace DB diff --git a/dbms/src/Operators/GetResultSinkOp.h b/dbms/src/Operators/GetResultSinkOp.h new file mode 100644 index 00000000000..fd013c2cbd6 --- /dev/null +++ b/dbms/src/Operators/GetResultSinkOp.h @@ -0,0 +1,43 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +class PhysicalGetResultSink; +// The sink operator for getting the execution results. +// Now it is used in unit tests. +class GetResultSinkOp : public SinkOp +{ +public: + explicit GetResultSinkOp(PhysicalGetResultSink & physical_sink_) + : physical_sink(physical_sink_) + { + } + + String getName() const override + { + return "GetResultSinkOp"; + } + + OperatorStatus writeImpl(Block && block) override; + +private: + PhysicalGetResultSink & physical_sink; +}; +} // namespace DB diff --git a/dbms/src/Operators/LimitTransformOp.cpp b/dbms/src/Operators/LimitTransformOp.cpp new file mode 100644 index 00000000000..fe64a0fc7d9 --- /dev/null +++ b/dbms/src/Operators/LimitTransformOp.cpp @@ -0,0 +1,31 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +namespace DB +{ +OperatorStatus LimitTransformOp::transformImpl(Block & block) +{ + if (!action->transform(block)) + block = {}; + return OperatorStatus::HAS_OUTPUT; +} + +void LimitTransformOp::transformHeaderImpl(Block & header_) +{ + header_ = action->getHeader(); +} +} // namespace DB diff --git a/dbms/src/Operators/LimitTransformOp.h b/dbms/src/Operators/LimitTransformOp.h new file mode 100644 index 00000000000..716c34b8b1c --- /dev/null +++ b/dbms/src/Operators/LimitTransformOp.h @@ -0,0 +1,48 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +struct GlobalLimitTransformAction; +using GlobalLimitPtr = std::shared_ptr; + +class LimitTransformOp : public TransformOp +{ +public: + LimitTransformOp( + const GlobalLimitPtr & action_, + const String & req_id) + : action(action_) + , log(Logger::get(req_id)) + {} + + String getName() const override + { + return "LimitTransformOp"; + } + + OperatorStatus transformImpl(Block & block) override; + + void transformHeaderImpl(Block & header_) override; + +private: + GlobalLimitPtr action; + const LoggerPtr log; +}; +} // namespace DB diff --git a/dbms/src/Operators/Operator.cpp b/dbms/src/Operators/Operator.cpp new file mode 100644 index 00000000000..03aed506df5 --- /dev/null +++ b/dbms/src/Operators/Operator.cpp @@ -0,0 +1,103 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +namespace DB +{ +OperatorStatus Operator::await() +{ + // TODO collect operator profile info here. + auto op_status = awaitImpl(); +#ifndef NDEBUG + assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); +#endif + return op_status; +} + +OperatorStatus SourceOp::read(Block & block) +{ + // TODO collect operator profile info here. + assert(!block); + auto op_status = readImpl(block); +#ifndef NDEBUG + if (block) + { + Block header = getHeader(); + assertBlocksHaveEqualStructure(block, header, getName()); + } + assertOperatorStatus(op_status, {OperatorStatus::HAS_OUTPUT}); +#endif + return op_status; +} + +OperatorStatus TransformOp::transform(Block & block) +{ + // TODO collect operator profile info here. + auto op_status = transformImpl(block); +#ifndef NDEBUG + if (block) + { + Block header = getHeader(); + assertBlocksHaveEqualStructure(block, header, getName()); + } + assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); +#endif + return op_status; +} + +OperatorStatus TransformOp::tryOutput(Block & block) +{ + // TODO collect operator profile info here. + assert(!block); + auto op_status = tryOutputImpl(block); +#ifndef NDEBUG + if (block) + { + Block header = getHeader(); + assertBlocksHaveEqualStructure(block, header, getName()); + } + assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT, OperatorStatus::HAS_OUTPUT}); +#endif + return op_status; +} + +OperatorStatus SinkOp::prepare() +{ + // TODO collect operator profile info here. + auto op_status = prepareImpl(); +#ifndef NDEBUG + assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT}); +#endif + return op_status; +} + +OperatorStatus SinkOp::write(Block && block) +{ +#ifndef NDEBUG + if (block) + { + Block header = getHeader(); + assertBlocksHaveEqualStructure(block, header, getName()); + } +#endif + // TODO collect operator profile info here. + auto op_status = writeImpl(std::move(block)); +#ifndef NDEBUG + assertOperatorStatus(op_status, {OperatorStatus::NEED_INPUT}); +#endif + return op_status; +} +} // namespace DB diff --git a/dbms/src/Operators/Operator.h b/dbms/src/Operators/Operator.h new file mode 100644 index 00000000000..c2cc81e863f --- /dev/null +++ b/dbms/src/Operators/Operator.h @@ -0,0 +1,127 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +/** + * All interfaces of the operator may return the following state. + * - finish status and waiting status can be returned in all method of operator. + * - operator may return a different running status depending on the method. +*/ +enum class OperatorStatus +{ + /// finish status + FINISHED, + CANCELLED, + /// waiting status + WAITING, + /// running status + // means that TransformOp/SinkOp needs to input a block to do the calculation, + NEED_INPUT, + // means that SourceOp/TransformOp outputs a block as input to the subsequent operators. + HAS_OUTPUT, +}; + +// TODO support operator profile info like `BlockStreamProfileInfo`. + +class Operator +{ +public: + virtual ~Operator() = default; + // running status may return are + // - `NEED_INPUT` means that the data that the operator is waiting for has been prepared. + OperatorStatus await(); + virtual OperatorStatus awaitImpl() { throw Exception("Unsupport"); } + + virtual String getName() const = 0; + + /** Get data structure of the operator in a form of "header" block (it is also called "sample block"). + * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values. + */ + Block getHeader() const + { + assert(header); + return header; + } + void setHeader(const Block & header_) + { + assert(header_ && !header); + header = header_; + } + +protected: + Block header; +}; + +// The running status returned by Source can only be `HAS_OUTPUT`. +class SourceOp : public Operator +{ +public: + // 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; + + OperatorStatus awaitImpl() override { return OperatorStatus::HAS_OUTPUT; } +}; +using SourceOpPtr = std::unique_ptr; + +class TransformOp : public Operator +{ +public: + // 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; } + // 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; + + virtual void transformHeaderImpl(Block & header_) = 0; + void transformHeader(Block & header_) + { + assert(header_); + transformHeaderImpl(header_); + setHeader(header_); + } + + OperatorStatus awaitImpl() override { return OperatorStatus::NEED_INPUT; } +}; +using TransformOpPtr = std::unique_ptr; +using TransformOps = std::vector; + +// The running status returned by Sink can only be `NEED_INPUT`. +class SinkOp : public Operator +{ +public: + OperatorStatus prepare(); + virtual OperatorStatus prepareImpl() { return OperatorStatus::NEED_INPUT; } + + OperatorStatus write(Block && block); + virtual OperatorStatus writeImpl(Block && block) = 0; + + OperatorStatus awaitImpl() override { return OperatorStatus::NEED_INPUT; } +}; +using SinkOpPtr = std::unique_ptr; +} // namespace DB diff --git a/dbms/src/Operators/OperatorHelper.cpp b/dbms/src/Operators/OperatorHelper.cpp new file mode 100644 index 00000000000..8ed5a4e6dde --- /dev/null +++ b/dbms/src/Operators/OperatorHelper.cpp @@ -0,0 +1,44 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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. + +#include +#include + +#include + +namespace DB +{ +void assertOperatorStatus( + OperatorStatus status, + std::initializer_list expect_running_statuses) +{ + switch (status) + { + // finish status and waiting status can be returned in all method of operator. + case OperatorStatus::FINISHED: + case OperatorStatus::CANCELLED: + case OperatorStatus::WAITING: + return; + default: + { + for (const auto & expect_running_status : expect_running_statuses) + { + if (expect_running_status == status) + return; + } + RUNTIME_ASSERT(false, "Unexpected operator status {}", magic_enum::enum_name(status)); + } + } +} +} // namespace DB diff --git a/dbms/src/Operators/OperatorHelper.h b/dbms/src/Operators/OperatorHelper.h new file mode 100644 index 00000000000..7cf9f89d1c2 --- /dev/null +++ b/dbms/src/Operators/OperatorHelper.h @@ -0,0 +1,24 @@ +// Copyright 2023 PingCAP, Ltd. +// +// 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 +{ +void assertOperatorStatus( + OperatorStatus status, + std::initializer_list expect_running_statuses); +} // namespace DB diff --git a/dbms/src/Operators/tests/CMakeLists.txt b/dbms/src/Operators/tests/CMakeLists.txt new file mode 100644 index 00000000000..d030c5c82e7 --- /dev/null +++ b/dbms/src/Operators/tests/CMakeLists.txt @@ -0,0 +1,15 @@ +# Copyright 2023 PingCAP, Ltd. +# +# 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. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 981b2ef8ee2..50c7140077a 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -42,6 +42,7 @@ #include #include #include +#include #include #include #include @@ -1277,6 +1278,25 @@ int Server::main(const std::vector & /*args*/) settings.elastic_threadpool_init_cap, std::chrono::milliseconds(settings.elastic_threadpool_shrink_period_ms)); + // For test mode, TaskScheduler is controlled by test case. + bool enable_pipeline = settings.enable_pipeline && !global_context->isTest(); + if (enable_pipeline) + { + auto get_pool_size = [](const auto & setting) { + return setting == 0 ? getNumberOfLogicalCPUCores() : static_cast(setting); + }; + TaskSchedulerConfig config{get_pool_size(settings.pipeline_task_thread_pool_size)}; + assert(!TaskScheduler::instance); + TaskScheduler::instance = std::make_unique(config); + } + SCOPE_EXIT({ + if (enable_pipeline) + { + assert(TaskScheduler::instance); + TaskScheduler::instance.reset(); + } + }); + if (settings.enable_async_grpc_client) { auto size = settings.grpc_completion_queue_pool_size; diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 699c9e43907..fb5592c4083 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -53,6 +54,24 @@ TiDB::TP dataTypeToTP(const DataTypePtr & type) } } +void ExecutorTest::SetUp() +{ + initializeContext(); + initializeClientInfo(); + TaskSchedulerConfig config{8}; + assert(!TaskScheduler::instance); + TaskScheduler::instance = std::make_unique(config); +} + +void ExecutorTest::TearDown() +{ + if (context.mockStorage()) + context.mockStorage()->clear(); + + assert(TaskScheduler::instance); + TaskScheduler::instance.reset(); +} + DAGContext & ExecutorTest::getDAGContext() { assert(dag_context_ptr != nullptr); @@ -101,7 +120,7 @@ void ExecutorTest::executeInterpreter(const String & expected_string, const std: // Don't care regions information in interpreter tests. auto query_executor = queryExecute(context.context, /*internal=*/true); - ASSERT_EQ(Poco::trim(expected_string), Poco::trim(query_executor->dump())); + ASSERT_EQ(Poco::trim(expected_string), Poco::trim(query_executor->toString())); } void ExecutorTest::executeInterpreterWithDeltaMerge(const String & expected_string, const std::shared_ptr & request, size_t concurrency) @@ -110,14 +129,14 @@ void ExecutorTest::executeInterpreterWithDeltaMerge(const String & expected_stri TiFlashTestEnv::setUpTestContext(context.context, &dag_context, context.mockStorage(), TestType::EXECUTOR_TEST); // Don't care regions information in interpreter tests. auto query_executor = queryExecute(context.context, /*internal=*/true); - ASSERT_EQ(Poco::trim(expected_string), Poco::trim(query_executor->dump())); + ASSERT_EQ(Poco::trim(expected_string), Poco::trim(query_executor->toString())); } void ExecutorTest::executeExecutor( const std::shared_ptr & request, std::function<::testing::AssertionResult(const ColumnsWithTypeAndName &)> assert_func) { - WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN + WRAP_FOR_TEST_BEGIN std::vector concurrencies{1, 2, 10}; for (auto concurrency : concurrencies) { @@ -153,7 +172,7 @@ void ExecutorTest::executeExecutor( ASSERT_TRUE(assert_func(res)) << test_info_msg(); } } - WRAP_FOR_DIS_ENABLE_PLANNER_END + WRAP_FOR_TEST_END } void ExecutorTest::executeAndAssertColumnsEqual(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns) @@ -205,6 +224,11 @@ void ExecutorTest::enablePlanner(bool is_enable) context.context.setSetting("enable_planner", is_enable ? "true" : "false"); } +void ExecutorTest::enablePipeline(bool is_enable) +{ + context.context.setSetting("enable_pipeline", is_enable ? "true" : "false"); +} + DB::ColumnsWithTypeAndName ExecutorTest::executeStreams( const std::shared_ptr & request, size_t concurrency, diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index 556aebc6ee1..85caf2358a4 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -31,22 +31,28 @@ TiDB::TP dataTypeToTP(const DataTypePtr & type); ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream); ColumnsWithTypeAndName readBlocks(std::vector streams); -#define WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN \ - std::vector bools{false, true}; \ - for (auto enable_planner : bools) \ - { \ - enablePlanner(enable_planner); - -#define WRAP_FOR_DIS_ENABLE_PLANNER_END } +#define WRAP_FOR_TEST_BEGIN \ + std::vector planner_bools{false, true}; \ + for (auto enable_planner : planner_bools) \ + { \ + enablePlanner(enable_planner); \ + std::vector pipeline_bools{false}; \ + if (enable_planner) \ + pipeline_bools.push_back(true); \ + for (auto enable_pipeline : pipeline_bools) \ + { \ + enablePipeline(enable_pipeline); + +#define WRAP_FOR_TEST_END \ + } \ + } class ExecutorTest : public ::testing::Test { protected: - void SetUp() override - { - initializeContext(); - initializeClientInfo(); - } + void SetUp() override; + + void TearDown() override; public: ExecutorTest() @@ -63,6 +69,8 @@ class ExecutorTest : public ::testing::Test void enablePlanner(bool is_enable); + void enablePipeline(bool is_enable); + static void dagRequestEqual(const String & expected_string, const std::shared_ptr & actual); void executeInterpreter(const String & expected_string, const std::shared_ptr & request, size_t concurrency); diff --git a/dbms/src/TestUtils/MPPTaskTestUtils.cpp b/dbms/src/TestUtils/MPPTaskTestUtils.cpp index b187f3e6f5a..08117f45b31 100644 --- a/dbms/src/TestUtils/MPPTaskTestUtils.cpp +++ b/dbms/src/TestUtils/MPPTaskTestUtils.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -42,6 +42,7 @@ void MPPTaskTestUtils::SetUpTestCase() void MPPTaskTestUtils::TearDown() { MockComputeServerManager::instance().reset(); + ExecutorTest::TearDown(); } void MPPTaskTestUtils::startServers() diff --git a/dbms/src/WindowFunctions/tests/gtest_lead_lag.cpp b/dbms/src/WindowFunctions/tests/gtest_lead_lag.cpp index 1a8d9a7bcfc..911ff2869b6 100644 --- a/dbms/src/WindowFunctions/tests/gtest_lead_lag.cpp +++ b/dbms/src/WindowFunctions/tests/gtest_lead_lag.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -35,7 +35,7 @@ class LeadLag : public DB::tests::ExecutorTest void executeWithConcurrencyAndBlockSize(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns) { - WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN + WRAP_FOR_TEST_BEGIN std::vector block_sizes{1, 2, 3, 4, DEFAULT_BLOCK_SIZE}; for (auto block_size : block_sizes) { @@ -44,7 +44,7 @@ class LeadLag : public DB::tests::ExecutorTest ASSERT_COLUMNS_EQ_UR(expect_columns, executeStreams(request, 2)); ASSERT_COLUMNS_EQ_UR(expect_columns, executeStreams(request, max_concurrency_level)); } - WRAP_FOR_DIS_ENABLE_PLANNER_END + WRAP_FOR_TEST_END } void executeFunctionAndAssert(