diff --git a/.github/workflows/license-checker.yml b/.github/workflows/license-checker.yml index e156c1b2b4c..2b23cca7eaa 100644 --- a/.github/workflows/license-checker.yml +++ b/.github/workflows/license-checker.yml @@ -15,7 +15,7 @@ jobs: steps: - uses: actions/checkout@v2 - name: Check License Header - uses: apache/skywalking-eyes@main + uses: apache/skywalking-eyes@v0.3.0 env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} with: diff --git a/CMakeLists.txt b/CMakeLists.txt index f2ec9f3316b..2e33a127807 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -239,10 +239,8 @@ else () set (CMAKE_CXX_STANDARD_REQUIRED ON) endif () -if (NOT ARCH_ARM) - set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3") - set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3") -endif () +set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3") +set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3") option (DEBUG_WITHOUT_DEBUG_INFO "Set to ON to build dev target without debug info (remove flag `-g` in order to accelerate compiling speed and reduce target binary size)" OFF) if (DEBUG_WITHOUT_DEBUG_INFO) diff --git a/contrib/kvproto b/contrib/kvproto index 12e2f5a9d16..a5d4ffd2ba3 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 12e2f5a9d167f46602804840857ddc8ff06dc695 +Subproject commit a5d4ffd2ba337dad0bc99e9fb53bf665864a3f3b diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 88332510ea5..573efc6d3d1 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 88332510ea560c97ec9aa558b15c75a42bc2ea97 +Subproject commit 573efc6d3d155a9a01da003e70f111485becf2bc diff --git a/contrib/tipb b/contrib/tipb index bfb5c2c5518..0f4f873beca 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit bfb5c2c55188c254018d3cf77bfad73b4d4b77ec +Subproject commit 0f4f873beca8d5078dde0a23d15ad5ce3188ed0d diff --git a/dbms/src/Common/MPMCQueue.h b/dbms/src/Common/MPMCQueue.h index 31dfc65a174..e005c363eae 100644 --- a/dbms/src/Common/MPMCQueue.h +++ b/dbms/src/Common/MPMCQueue.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include @@ -74,6 +75,11 @@ class MPMCQueue destruct(getObj(read_pos)); } + // Cannot to use copy/move constructor, + // because MPMCQueue maybe used by different threads. + // Copy and move it is dangerous. + DISALLOW_COPY_AND_MOVE(MPMCQueue); + /// Block until: /// 1. Pop succeeds with a valid T: return true. /// 2. The queue is cancelled or finished: return false. diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index f983de91b37..cd9d6235f52 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -24,7 +24,7 @@ namespace DB { ParallelAggregatingBlockInputStream::ParallelAggregatingBlockInputStream( const BlockInputStreams & inputs, - const BlockInputStreamPtr & additional_input_at_end, + const BlockInputStreams & additional_inputs_at_end, const Aggregator::Params & params_, const FileProviderPtr & file_provider_, bool final_, @@ -41,11 +41,10 @@ ParallelAggregatingBlockInputStream::ParallelAggregatingBlockInputStream( , keys_size(params.keys_size) , aggregates_size(params.aggregates_size) , handler(*this) - , processor(inputs, additional_input_at_end, max_threads, handler, log) + , processor(inputs, additional_inputs_at_end, max_threads, handler, log) { children = inputs; - if (additional_input_at_end) - children.push_back(additional_input_at_end); + children.insert(children.end(), additional_inputs_at_end.begin(), additional_inputs_at_end.end()); } diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h index 41e61786370..907622c8364 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -36,7 +36,7 @@ class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream */ ParallelAggregatingBlockInputStream( const BlockInputStreams & inputs, - const BlockInputStreamPtr & additional_input_at_end, + const BlockInputStreams & additional_inputs_at_end, const Aggregator::Params & params_, const FileProviderPtr & file_provider_, bool final_, diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 34c70a7085e..57ab37e1756 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -83,9 +84,8 @@ template class ParallelInputsProcessor { public: - /** additional_input_at_end - if not nullptr, - * then the blocks from this source will start to be processed only after all other sources are processed. - * This is done in the main thread. + /** additional_inputs_at_end - if not empty, + * then the blocks from the sources will start to be processed only after all other sources are processed. * * Intended for implementation of FULL and RIGHT JOIN * - where you must first make JOIN in parallel, while noting which keys are not found, @@ -93,19 +93,18 @@ class ParallelInputsProcessor */ ParallelInputsProcessor( const BlockInputStreams & inputs_, - const BlockInputStreamPtr & additional_input_at_end_, + const BlockInputStreams & additional_inputs_at_end_, size_t max_threads_, Handler & handler_, const LoggerPtr & log_) : inputs(inputs_) - , additional_input_at_end(additional_input_at_end_) - , max_threads(std::min(inputs_.size(), max_threads_)) + , additional_inputs_at_end(additional_inputs_at_end_) + , max_threads(std::min(std::max(inputs_.size(), additional_inputs_at_end_.size()), max_threads_)) , handler(handler_) + , working_inputs(inputs_) + , working_additional_inputs(additional_inputs_at_end_) , log(log_) - { - for (size_t i = 0; i < inputs_.size(); ++i) - unprepared_inputs.emplace(inputs_[i], i); - } + {} ~ParallelInputsProcessor() { @@ -132,36 +131,21 @@ class ParallelInputsProcessor /// Ask all sources to stop earlier than they run out. void cancel(bool kill) { - finish = true; + working_inputs.available_inputs.cancel(); + working_additional_inputs.available_inputs.cancel(); - for (auto & input : inputs) - { - if (IProfilingBlockInputStream * child = dynamic_cast(&*input)) - { - try - { - child->cancel(kill); - } - catch (...) - { - /** If you can not ask one or more sources to stop. - * (for example, the connection is broken for distributed query processing) - * - then do not care. - */ - LOG_FMT_ERROR(log, "Exception while cancelling {}", child->getName()); - } - } - } + cancelStreams(inputs, kill); + cancelStreams(additional_inputs_at_end, kill); } /// Wait until all threads are finished, before the destructor. void wait() { - if (joined_threads) - return; if (thread_manager) + { thread_manager->wait(); - joined_threads = true; + thread_manager.reset(); + } } size_t getNumActiveThreads() const @@ -181,13 +165,78 @@ class ParallelInputsProcessor BlockInputStreamPtr in; size_t i; /// The source number (for debugging). - InputData() {} + InputData() + : i(0) + {} InputData(const BlockInputStreamPtr & in_, size_t i_) : in(in_) , i(i_) {} }; + struct WorkingInputs + { + explicit WorkingInputs(const BlockInputStreams & inputs_) + : available_inputs(inputs_.size()) + , active_inputs(inputs_.size()) + , unprepared_inputs(inputs_.size()) + { + for (size_t i = 0; i < inputs_.size(); ++i) + unprepared_inputs.emplace(inputs_[i], i); + } + /** A set of available sources that are not currently processed by any thread. + * Each thread takes one source from this set, takes a block out of the source (at this moment the source does the calculations) + * and (if the source is not run out), puts it back into the set of available sources. + * + * The question arises what is better to use: + * - the queue (just processed source will be processed the next time later than the rest) + * - stack (just processed source will be processed as soon as possible). + * + * The stack is better than the queue when you need to do work on reading one source more consequentially, + * and theoretically, this allows you to achieve more consequent/consistent reads from the disk. + * + * But when using the stack, there is a problem with distributed query processing: + * data is read only from a part of the servers, and on the other servers + * a timeout occurs during send, and the request processing ends with an exception. + * + * Therefore, a queue is used. This can be improved in the future. + */ + using AvailableInputs = MPMCQueue; + AvailableInputs available_inputs; + + /// How many active input streams. + std::atomic active_inputs; + + /** For parallel preparing (readPrefix) child streams. + * First, streams are located here. + * After a stream was prepared, it is moved to "available_inputs" for reading. + */ + using UnpreparedInputs = MPMCQueue; + UnpreparedInputs unprepared_inputs; + }; + + void cancelStreams(const BlockInputStreams & streams, bool kill) + { + for (const auto & input : streams) + { + if (auto * p_child = dynamic_cast(&*input)) + { + try + { + p_child->cancel(kill); + } + catch (...) + { + /** If you can not ask one or more sources to stop. + * (for example, the connection is broken for distributed query processing) + * - then do not care. + */ + LOG_FMT_ERROR(log, "Exception while cancelling {}", p_child->getName()); + } + } + } + } + void publishPayload(BlockInputStreamPtr & stream, Block & block, size_t thread_num) { if constexpr (mode == StreamUnionMode::Basic) @@ -201,32 +250,24 @@ class ParallelInputsProcessor void thread(size_t thread_num) { - std::exception_ptr exception; + work(thread_num, working_inputs); + work(thread_num, working_additional_inputs); - try - { - while (!finish) - { - InputData unprepared_input; - { - std::lock_guard lock(unprepared_inputs_mutex); - - if (unprepared_inputs.empty()) - break; - - unprepared_input = unprepared_inputs.front(); - unprepared_inputs.pop(); - } + handler.onFinishThread(thread_num); - unprepared_input.in->readPrefix(); + if (0 == --active_threads) + { + handler.onFinish(); + } + } - { - std::lock_guard lock(available_inputs_mutex); - available_inputs.push(unprepared_input); - } - } + void work(size_t thread_num, WorkingInputs & work) + { + std::exception_ptr exception; - loop(thread_num); + try + { + loop(thread_num, work); } catch (...) { @@ -237,134 +278,63 @@ class ParallelInputsProcessor { handler.onException(exception, thread_num); } - - handler.onFinishThread(thread_num); - - /// The last thread on the output indicates that there is no more data. - if (0 == --active_threads) - { - /// And then it processes an additional source, if there is one. - if (additional_input_at_end) - { - try - { - additional_input_at_end->readPrefix(); - while (Block block = additional_input_at_end->read()) - publishPayload(additional_input_at_end, block, thread_num); - } - catch (...) - { - exception = std::current_exception(); - } - - if (exception) - { - handler.onException(exception, thread_num); - } - } - - handler.onFinish(); /// TODO If in `onFinish` or `onFinishThread` there is an exception, then std::terminate is called. - } } - void loop(size_t thread_num) + /// This function may be called in different threads. + /// If no exception occurs, we can ensure that the work is all done when the function + /// returns in any thread. + void loop(size_t thread_num, WorkingInputs & work) { - while (!finish) /// You may need to stop work earlier than all sources run out. + if (work.active_inputs == 0) { - InputData input; + return; + } - /// Select the next source. - { - std::lock_guard lock(available_inputs_mutex); + InputData input; - /// If there are no free sources, then this thread is no longer needed. (But other threads can work with their sources.) - if (available_inputs.empty()) - break; - - input = available_inputs.front(); + while (work.unprepared_inputs.tryPop(input)) + { + input.in->readPrefix(); - /// We remove the source from the queue of available sources. - available_inputs.pop(); - } + work.available_inputs.push(input); + } + // The condition is false when all input streams are exhausted or + // an exception occurred then the queue was cancelled. + while (work.available_inputs.pop(input)) + { /// The main work. Block block = input.in->read(); + if (block) { - if (finish) - break; - - /// If this source is not run out yet, then put the resulting block in the ready queue. + work.available_inputs.push(input); + publishPayload(input.in, block, thread_num); + } + else + { + if (0 == --work.active_inputs) { - std::lock_guard lock(available_inputs_mutex); - - if (block) - { - available_inputs.push(input); - } - else - { - if (available_inputs.empty()) - break; - } - } - - if (finish) + work.available_inputs.finish(); break; - - if (block) - publishPayload(input.in, block, thread_num); + } } } } - BlockInputStreams inputs; - BlockInputStreamPtr additional_input_at_end; + const BlockInputStreams inputs; + const BlockInputStreams additional_inputs_at_end; unsigned max_threads; Handler & handler; std::shared_ptr thread_manager; - /** A set of available sources that are not currently processed by any thread. - * Each thread takes one source from this set, takes a block out of the source (at this moment the source does the calculations) - * and (if the source is not run out), puts it back into the set of available sources. - * - * The question arises what is better to use: - * - the queue (just processed source will be processed the next time later than the rest) - * - stack (just processed source will be processed as soon as possible). - * - * The stack is better than the queue when you need to do work on reading one source more consequentially, - * and theoretically, this allows you to achieve more consequent/consistent reads from the disk. - * - * But when using the stack, there is a problem with distributed query processing: - * data is read only from a part of the servers, and on the other servers - * a timeout occurs during send, and the request processing ends with an exception. - * - * Therefore, a queue is used. This can be improved in the future. - */ - using AvailableInputs = std::queue; - AvailableInputs available_inputs; - - /** For parallel preparing (readPrefix) child streams. - * First, streams are located here. - * After a stream was prepared, it is moved to "available_inputs" for reading. - */ - using UnpreparedInputs = std::queue; - UnpreparedInputs unprepared_inputs; - - /// For operations with available_inputs. - std::mutex available_inputs_mutex; - - /// For operations with unprepared_inputs. - std::mutex unprepared_inputs_mutex; + WorkingInputs working_inputs; + WorkingInputs working_additional_inputs; /// How many sources ran out. std::atomic active_threads{0}; - /// Finish the threads work (before the sources run out). - std::atomic finish{false}; - /// Wait for the completion of all threads. - std::atomic joined_threads{false}; const LoggerPtr log; }; diff --git a/dbms/src/DataStreams/TiRemoteBlockInputStream.h b/dbms/src/DataStreams/TiRemoteBlockInputStream.h index f249bf1a0dc..c1afb1e9f4e 100644 --- a/dbms/src/DataStreams/TiRemoteBlockInputStream.h +++ b/dbms/src/DataStreams/TiRemoteBlockInputStream.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -58,6 +59,11 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream uint64_t total_rows; + // For fine grained shuffle, sender will partition data into muiltiple streams by hashing. + // ExchangeReceiverBlockInputStream only need to read its own stream, i.e., streams[stream_id]. + // CoprocessorBlockInputStream doesn't take care of this. + size_t stream_id; + void initRemoteExecutionSummaries(tipb::SelectResponse & resp, size_t index) { for (const auto & execution_summary : resp.execution_summaries()) @@ -120,7 +126,7 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream bool fetchRemoteResult() { - auto result = remote_reader->nextResult(block_queue, sample_block); + auto result = remote_reader->nextResult(block_queue, sample_block, stream_id); if (result.meet_error) { LOG_FMT_WARNING(log, "remote reader meets error: {}", result.error_msg); @@ -168,29 +174,22 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream } public: - TiRemoteBlockInputStream(std::shared_ptr remote_reader_, const String & req_id, const String & executor_id) + TiRemoteBlockInputStream(std::shared_ptr remote_reader_, const String & req_id, const String & executor_id, size_t stream_id_) : remote_reader(remote_reader_) , source_num(remote_reader->getSourceNum()) , name(fmt::format("TiRemoteBlockInputStream({})", RemoteReader::name)) , execution_summaries_inited(source_num) , log(Logger::get(name, req_id, executor_id)) , total_rows(0) + , stream_id(stream_id_) { - // generate sample block - ColumnsWithTypeAndName columns; - for (auto & dag_col : remote_reader->getOutputSchema()) - { - auto tp = getDataTypeByColumnInfoForComputingLayer(dag_col.second); - ColumnWithTypeAndName col(tp, dag_col.first); - columns.emplace_back(col); - } - for (size_t i = 0; i < source_num; i++) + for (size_t i = 0; i < source_num; ++i) { execution_summaries_inited[i].store(false); } execution_summaries.resize(source_num); connection_profile_infos.resize(source_num); - sample_block = Block(columns); + sample_block = Block(getColumnWithTypeAndName(toNamesAndTypes(remote_reader->getOutputSchema()))); } Block getHeader() const override { return sample_block; } diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index a782c3dd087..ffcc8d77c10 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -94,20 +94,19 @@ class UnionBlockInputStream final : public IProfilingBlockInputStream public: UnionBlockInputStream( BlockInputStreams inputs, - BlockInputStreamPtr additional_input_at_end, + BlockInputStreams additional_inputs_at_end, size_t max_threads, const String & req_id, ExceptionCallback exception_callback_ = ExceptionCallback()) - : output_queue(std::min(inputs.size(), max_threads) * 5) // reduce contention + : output_queue(std::min(std::max(inputs.size(), additional_inputs_at_end.size()), max_threads) * 5) // reduce contention , log(Logger::get(NAME, req_id)) , handler(*this) - , processor(inputs, additional_input_at_end, max_threads, handler, log) + , processor(inputs, additional_inputs_at_end, max_threads, handler, log) , exception_callback(exception_callback_) { // TODO: assert capacity of output_queue is not less than processor.getMaxThreads() children = inputs; - if (additional_input_at_end) - children.push_back(additional_input_at_end); + children.insert(children.end(), additional_inputs_at_end.begin(), additional_inputs_at_end.end()); size_t num_children = children.size(); if (num_children > 1) diff --git a/dbms/src/DataStreams/tests/union_stream2.cpp b/dbms/src/DataStreams/tests/union_stream2.cpp index f939cda4e14..fb3f7238414 100644 --- a/dbms/src/DataStreams/tests/union_stream2.cpp +++ b/dbms/src/DataStreams/tests/union_stream2.cpp @@ -51,7 +51,7 @@ try for (size_t i = 0, size = streams.size(); i < size; ++i) streams[i] = std::make_shared(streams[i]); - BlockInputStreamPtr stream = std::make_shared>(streams, nullptr, settings.max_threads, /*req_id=*/""); + BlockInputStreamPtr stream = std::make_shared>(streams, BlockInputStreams{}, settings.max_threads, /*req_id=*/""); stream = std::make_shared(stream, 10, 0, ""); WriteBufferFromFileDescriptor wb(STDERR_FILENO); diff --git a/dbms/src/Debug/astToExecutor.cpp b/dbms/src/Debug/astToExecutor.cpp index 7d1f3bc7209..61f4474f919 100644 --- a/dbms/src/Debug/astToExecutor.cpp +++ b/dbms/src/Debug/astToExecutor.cpp @@ -170,6 +170,7 @@ std::unordered_map func_name_to_sig({ {"cast_decimal_datetime", tipb::ScalarFuncSig::CastDecimalAsTime}, {"cast_time_datetime", tipb::ScalarFuncSig::CastTimeAsTime}, {"cast_string_datetime", tipb::ScalarFuncSig::CastStringAsTime}, + {"concat", tipb::ScalarFuncSig::Concat}, {"round_int", tipb::ScalarFuncSig::RoundInt}, {"round_uint", tipb::ScalarFuncSig::RoundInt}, {"round_dec", tipb::ScalarFuncSig::RoundDec}, @@ -461,6 +462,14 @@ void functionToPB(const DAGSchema & input, ASTFunction * func, tipb::Expr * expr ft->set_collate(collator_id); break; } + case tipb::ScalarFuncSig::Concat: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeString); + ft->set_collate(collator_id); + break; + } case tipb::ScalarFuncSig::RoundInt: case tipb::ScalarFuncSig::RoundWithFracInt: { @@ -851,6 +860,7 @@ bool ExchangeReceiver::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t c { tipb_executor->set_tp(tipb::ExecType::TypeExchangeReceiver); tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); tipb::ExchangeReceiver * exchange_receiver = tipb_executor->mutable_exchange_receiver(); for (auto & field : output_schema) { @@ -1354,6 +1364,7 @@ bool Window::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id { tipb_executor->set_tp(tipb::ExecType::TypeWindow); tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); tipb::Window * window = tipb_executor->mutable_window(); auto & input_schema = children[0]->output_schema; for (const auto & expr : func_descs) @@ -1430,6 +1441,7 @@ bool Sort::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, { tipb_executor->set_tp(tipb::ExecType::TypeSort); tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); tipb::Sort * sort = tipb_executor->mutable_sort(); sort->set_ispartialsort(is_partial_sort); @@ -1545,7 +1557,7 @@ ExecutorPtr compileAggregation(ExecutorPtr input, size_t & executor_index, ASTPt ci.tp = TiDB::TypeLongLong; ci.flag = TiDB::ColumnFlagUnsigned | TiDB::ColumnFlagNotNull; } - else if (func->name == "max" || func->name == "min" || func->name == "first_row") + else if (func->name == "max" || func->name == "min" || func->name == "first_row" || func->name == "sum") { ci = children_ci[0]; ci.flag &= ~TiDB::ColumnFlagNotNull; @@ -1665,13 +1677,13 @@ ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, ti return exchange_sender; } -ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema) +ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema, uint64_t fine_grained_shuffle_stream_count) { - ExecutorPtr exchange_receiver = std::make_shared(executor_index, schema); + ExecutorPtr exchange_receiver = std::make_shared(executor_index, schema, fine_grained_shuffle_stream_count); return exchange_receiver; } -ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame) +ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) { std::vector partition_columns; if (partition_by_expr_list != nullptr) @@ -1739,12 +1751,13 @@ ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr fun window_exprs, std::move(partition_columns), std::move(order_columns), - frame); + frame, + fine_grained_shuffle_stream_count); window->children.push_back(input); return window; } -ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort) +ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count) { std::vector order_columns; if (order_by_expr_list != nullptr) @@ -1758,8 +1771,8 @@ ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order compileExpr(input->output_schema, elem->children[0]); } } - ExecutorPtr sort = std::make_shared(executor_index, input->output_schema, std::move(order_columns), is_partial_sort); + ExecutorPtr sort = std::make_shared(executor_index, input->output_schema, std::move(order_columns), is_partial_sort, fine_grained_shuffle_stream_count); sort->children.push_back(input); return sort; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Debug/astToExecutor.h b/dbms/src/Debug/astToExecutor.h index 4d87c0db77e..f39f4059d26 100644 --- a/dbms/src/Debug/astToExecutor.h +++ b/dbms/src/Debug/astToExecutor.h @@ -139,8 +139,11 @@ struct ExchangeSender : Executor struct ExchangeReceiver : Executor { TaskMetas task_metas; - ExchangeReceiver(size_t & index, const DAGSchema & output) + uint64_t fine_grained_shuffle_stream_count; + + ExchangeReceiver(size_t & index, const DAGSchema & output, uint64_t fine_grained_shuffle_stream_count_ = 0) : Executor(index, "exchange_receiver_" + std::to_string(index), output) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) {} void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } bool toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, const MPPInfo & mpp_info, const Context &) override; @@ -292,13 +295,15 @@ struct Window : Executor std::vector partition_by_exprs; std::vector order_by_exprs; MockWindowFrame frame; + uint64_t fine_grained_shuffle_stream_count; - Window(size_t & index_, const DAGSchema & output_schema_, std::vector func_descs_, std::vector partition_by_exprs_, std::vector order_by_exprs_, MockWindowFrame frame_) + Window(size_t & index_, const DAGSchema & output_schema_, std::vector func_descs_, std::vector partition_by_exprs_, std::vector order_by_exprs_, MockWindowFrame frame_, uint64_t fine_grained_shuffle_stream_count_ = 0) : Executor(index_, "window_" + std::to_string(index_), output_schema_) , func_descs(std::move(func_descs_)) , partition_by_exprs(std::move(partition_by_exprs_)) , order_by_exprs(order_by_exprs_) , frame(frame_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { } // Currently only use Window Executor in Unit Test which don't call columnPrume. @@ -311,11 +316,13 @@ struct Sort : Executor { std::vector by_exprs; bool is_partial_sort; + uint64_t fine_grained_shuffle_stream_count; - Sort(size_t & index_, const DAGSchema & output_schema_, std::vector by_exprs_, bool is_partial_sort_) + Sort(size_t & index_, const DAGSchema & output_schema_, std::vector by_exprs_, bool is_partial_sort_, uint64_t fine_grained_shuffle_stream_count_ = 0) : Executor(index_, "sort_" + std::to_string(index_), output_schema_) , by_exprs(by_exprs_) , is_partial_sort(is_partial_sort_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { } // Currently only use Sort Executor in Unit Test which don't call columnPrume. @@ -343,11 +350,11 @@ ExecutorPtr compileJoin(size_t & executor_index, ExecutorPtr left, ExecutorPtr r ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, tipb::ExchangeType exchange_type); -ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema); +ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema, uint64_t fine_grained_shuffle_stream_count = 0); -ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame); +ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); -ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort); +ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count = 0); void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & field, tipb::Expr * expr, Int32 collator_id); } // namespace DB diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index e9335d1e2bd..62a8b7537f1 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -290,8 +290,9 @@ BlockInputStreamPtr executeQuery(Context & context, RegionID region_id, const DA tipb_exchange_receiver.encoded_task_meta_size(), 10, /*req_id=*/"", - /*executor_id=*/""); - BlockInputStreamPtr ret = std::make_shared(exchange_receiver, /*req_id=*/"", /*executor_id=*/""); + /*executor_id=*/"", + /*fine_grained_shuffle_stream_count=*/0); + BlockInputStreamPtr ret = std::make_shared(exchange_receiver, /*req_id=*/"", /*executor_id=*/"", /*stream_id*/ 0); return ret; } else diff --git a/dbms/src/Flash/Coprocessor/CoprocessorReader.h b/dbms/src/Flash/Coprocessor/CoprocessorReader.h index 25c07cff49c..b48fdbcd6dc 100644 --- a/dbms/src/Flash/Coprocessor/CoprocessorReader.h +++ b/dbms/src/Flash/Coprocessor/CoprocessorReader.h @@ -139,7 +139,8 @@ class CoprocessorReader return detail; } - CoprocessorReaderResult nextResult(std::queue & block_queue, const Block & header) + // stream_id is only meaningful for ExchagneReceiver. + CoprocessorReaderResult nextResult(std::queue & block_queue, const Block & header, size_t /*stream_id*/) { auto && [result, has_next] = resp_iter.next(); if (!result.error.empty()) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index ec0544c6ee4..1cf7a0d6c87 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -30,6 +30,8 @@ extern const int DIVIDED_BY_ZERO; extern const int INVALID_TIME; } // namespace ErrorCodes +const String enableFineGrainedShuffleExtraInfo = "enable fine grained shuffle"; + bool strictSqlMode(UInt64 sql_mode) { return sql_mode & TiDBSQLMode::STRICT_ALL_TABLES || sql_mode & TiDBSQLMode::STRICT_TRANS_TABLES; @@ -75,6 +77,11 @@ std::unordered_map & DAGContext::getProfileStreamsMap return profile_streams_map; } +void DAGContext::updateFinalConcurrency(size_t cur_streams_size, size_t streams_upper_limit) +{ + final_concurrency = std::min(std::max(final_concurrency, cur_streams_size), streams_upper_limit); +} + void DAGContext::initExecutorIdToJoinIdMap() { // only mpp task has join executor diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index a50a4d4007b..7bfc67afcad 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -116,6 +116,13 @@ constexpr UInt64 NO_ENGINE_SUBSTITUTION = 1ul << 30ul; constexpr UInt64 ALLOW_INVALID_DATES = 1ul << 32ul; } // namespace TiDBSQLMode +inline bool enableFineGrainedShuffle(uint64_t stream_count) +{ + return stream_count > 0; +} + +extern const String enableFineGrainedShuffleExtraInfo; + /// A context used to track the information that needs to be passed around during DAG planning. class DAGContext { @@ -303,6 +310,8 @@ class DAGContext return sql_mode & f; } + void updateFinalConcurrency(size_t cur_streams_size, size_t streams_upper_limit); + bool isTest() const { return is_test; } void setColumnsForTest(std::unordered_map & columns_for_test_map_) { columns_for_test_map = columns_for_test_map_; } ColumnsWithTypeAndName columnsForTest(String executor_id); diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 55a2024a8bc..9fe388f8fe4 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -72,6 +72,7 @@ DAGDriver::DAGDriver( ::grpc::ServerWriter<::coprocessor::BatchResponse> * writer_, bool internal_) : context(context_) + , dag_response(nullptr) , writer(writer_) , internal(internal_) , log(&Poco::Logger::get("DAGDriver")) @@ -129,7 +130,7 @@ try auto streaming_writer = std::make_shared(writer); TiDB::TiDBCollators collators; - std::unique_ptr response_writer = std::make_unique>( + std::unique_ptr response_writer = std::make_unique>( streaming_writer, std::vector(), collators, @@ -137,7 +138,9 @@ try context.getSettingsRef().dag_records_per_chunk, context.getSettingsRef().batch_send_min_limit, true, - dag_context); + dag_context, + /*fine_grained_shuffle_stream_count=*/0, + /*fine_grained_shuffle_batch_size=*/0); dag_output_stream = std::make_shared(streams.in->getHeader(), std::move(response_writer)); copyData(*streams.in, *dag_output_stream); } diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index aa269469cdb..5fbd86e9762 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -1130,30 +1130,40 @@ NamesWithAliases DAGExpressionAnalyzer::appendFinalProjectForRootQueryBlock( const std::vector & output_offsets, const String & column_prefix, bool keep_session_timezone_info) +{ + auto & step = initAndGetLastStep(chain); + + NamesWithAliases final_project = buildFinalProjection(step.actions, schema, output_offsets, column_prefix, keep_session_timezone_info); + + for (const auto & name : final_project) + { + step.required_output.push_back(name.first); + } + return final_project; +} + +NamesWithAliases DAGExpressionAnalyzer::buildFinalProjection( + const ExpressionActionsPtr & actions, + const std::vector & schema, + const std::vector & output_offsets, + const String & column_prefix, + bool keep_session_timezone_info) { if (unlikely(output_offsets.empty())) - throw Exception("Root Query block without output_offsets", ErrorCodes::LOGICAL_ERROR); + throw Exception("DAGRequest without output_offsets", ErrorCodes::LOGICAL_ERROR); bool need_append_timezone_cast = !keep_session_timezone_info && !context.getTimezoneInfo().is_utc_timezone; auto [need_append_type_cast, need_append_type_cast_vec] = isCastRequiredForRootFinalProjection(schema, output_offsets); assert(need_append_type_cast_vec.size() == output_offsets.size()); - auto & step = initAndGetLastStep(chain); - if (need_append_timezone_cast || need_append_type_cast) { // after appendCastForRootFinalProjection, source_columns has been modified. - appendCastForRootFinalProjection(step.actions, schema, output_offsets, need_append_timezone_cast, need_append_type_cast_vec); + appendCastForRootFinalProjection(actions, schema, output_offsets, need_append_timezone_cast, need_append_type_cast_vec); } // generate project aliases from source_columns. - NamesWithAliases final_project = genRootFinalProjectAliases(column_prefix, output_offsets); - - for (const auto & name : final_project) - { - step.required_output.push_back(name.first); - } - return final_project; + return genRootFinalProjectAliases(column_prefix, output_offsets); } String DAGExpressionAnalyzer::alignReturnType( diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 046088ab2b2..63d35abe26d 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -102,6 +102,8 @@ class DAGExpressionAnalyzer : private boost::noncopyable ExpressionActionsChain & chain, const String & column_prefix) const; + NamesWithAliases genNonRootFinalProjectAliases(const String & column_prefix) const; + // Generate a project action for root DAGQueryBlock, // to keep the schema of Block and tidb-schema the same. NamesWithAliases appendFinalProjectForRootQueryBlock( @@ -111,6 +113,13 @@ class DAGExpressionAnalyzer : private boost::noncopyable const String & column_prefix, bool keep_session_timezone_info); + NamesWithAliases buildFinalProjection( + const ExpressionActionsPtr & actions, + const std::vector & schema, + const std::vector & output_offsets, + const String & column_prefix, + bool keep_session_timezone_info); + String getActions( const tipb::Expr & expr, const ExpressionActionsPtr & actions, @@ -153,17 +162,38 @@ class DAGExpressionAnalyzer : private boost::noncopyable const tipb::Window & window, size_t window_columns_start_index); -#ifndef DBMS_PUBLIC_GTEST -private: -#endif NamesAndTypes buildOrderColumns( const ExpressionActionsPtr & actions, const ::google::protobuf::RepeatedPtrField & order_by); + String buildFilterColumn( + const ExpressionActionsPtr & actions, + const std::vector & conditions); + + void buildAggFuncs( + const tipb::Aggregation & aggregation, + const ExpressionActionsPtr & actions, + AggregateDescriptions & aggregate_descriptions, + NamesAndTypes & aggregated_columns); + + void buildAggGroupBy( + const google::protobuf::RepeatedPtrField & group_by, + const ExpressionActionsPtr & actions, + AggregateDescriptions & aggregate_descriptions, + NamesAndTypes & aggregated_columns, + Names & aggregation_keys, + std::unordered_set & agg_key_set, + bool group_by_collation_sensitive, + TiDB::TiDBCollators & collators); + void appendCastAfterAgg( const ExpressionActionsPtr & actions, const tipb::Aggregation & agg); +#ifndef DBMS_PUBLIC_GTEST +private: +#endif + String buildTupleFunctionForGroupConcat( const tipb::Expr & expr, SortDescription & sort_desc, @@ -187,22 +217,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable NamesAndTypes & aggregated_columns, bool empty_input_as_null); - void buildAggFuncs( - const tipb::Aggregation & aggregation, - const ExpressionActionsPtr & actions, - AggregateDescriptions & aggregate_descriptions, - NamesAndTypes & aggregated_columns); - - void buildAggGroupBy( - const google::protobuf::RepeatedPtrField & group_by, - const ExpressionActionsPtr & actions, - AggregateDescriptions & aggregate_descriptions, - NamesAndTypes & aggregated_columns, - Names & aggregation_keys, - std::unordered_set & agg_key_set, - bool group_by_collation_sensitive, - TiDB::TiDBCollators & collators); - void fillArgumentDetail( const ExpressionActionsPtr & actions, const tipb::Expr & arg, @@ -275,12 +289,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const String & column_name); - String buildFilterColumn( - const ExpressionActionsPtr & actions, - const std::vector & conditions); - - NamesWithAliases genNonRootFinalProjectAliases(const String & column_prefix) const; - NamesWithAliases genRootFinalProjectAliases( const String & column_prefix, const std::vector & output_offsets) const; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index e322a830744..764bf07f533 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -268,7 +267,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & size_t join_build_concurrency = settings.join_concurrent_build ? std::min(max_streams, build_pipeline.streams.size()) : 1; /// build side streams - executeExpression(build_pipeline, build_side_prepare_actions, "append join key and join filters for build side"); + executeExpression(build_pipeline, build_side_prepare_actions, log, "append join key and join filters for build side"); // add a HashJoinBuildBlockInputStream to build a shared hash table auto get_concurrency_build_index = JoinInterpreterHelper::concurrencyBuildIndexGenerator(join_build_concurrency); build_pipeline.transform([&](auto & stream) { @@ -284,7 +283,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & join_ptr->init(right_query.source->getHeader(), join_build_concurrency); /// probe side streams - executeExpression(probe_pipeline, probe_side_prepare_actions, "append join key and join filters for probe side"); + executeExpression(probe_pipeline, probe_side_prepare_actions, log, "append join key and join filters for probe side"); NamesAndTypes source_columns; for (const auto & p : probe_pipeline.firstStream()->getHeader()) source_columns.emplace_back(p.name, p.type); @@ -347,14 +346,26 @@ void DAGQueryBlockInterpreter::executeWhere(DAGPipeline & pipeline, const Expres void DAGQueryBlockInterpreter::executeWindow( DAGPipeline & pipeline, - WindowDescription & window_description) + WindowDescription & window_description, + bool enable_fine_grained_shuffle) { - executeExpression(pipeline, window_description.before_window, "before window"); + executeExpression(pipeline, window_description.before_window, log, "before window"); - /// If there are several streams, we merge them into one - executeUnion(pipeline, max_streams, log, false, "merge into one for window input"); - assert(pipeline.streams.size() == 1); - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), window_description, log->identifier()); + if (enable_fine_grained_shuffle) + { + /// Window function can be multiple threaded when fine grained shuffle is enabled. + pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, window_description, log->identifier()); + stream->setExtraInfo(enableFineGrainedShuffleExtraInfo); + }); + } + else + { + /// If there are several streams, we merge them into one. + executeUnion(pipeline, max_streams, log, false, "merge into one for window input"); + assert(pipeline.streams.size() == 1); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), window_description, log->identifier()); + } } void DAGQueryBlockInterpreter::executeAggregation( @@ -365,10 +376,7 @@ void DAGQueryBlockInterpreter::executeAggregation( AggregateDescriptions & aggregate_descriptions, bool is_final_agg) { - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, expression_actions_ptr, log->identifier()); - stream->setExtraInfo("before aggregation"); - }); + executeExpression(pipeline, expression_actions_ptr, log, "before aggregation"); Block before_agg_header = pipeline.firstStream()->getHeader(); @@ -383,34 +391,39 @@ void DAGQueryBlockInterpreter::executeAggregation( is_final_agg); /// If there are several sources, then we perform parallel aggregation - if (pipeline.streams.size() > 1) + if (pipeline.streams.size() > 1 || pipeline.streams_with_non_joined_data.size() > 1) { const Settings & settings = context.getSettingsRef(); - BlockInputStreamPtr stream_with_non_joined_data = combinedNonJoinedDataStream(pipeline, max_streams, log); - pipeline.firstStream() = std::make_shared( + BlockInputStreamPtr stream = std::make_shared( pipeline.streams, - stream_with_non_joined_data, + pipeline.streams_with_non_joined_data, params, context.getFileProvider(), true, max_streams, settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads), log->identifier()); + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); + // should record for agg before restore concurrency. See #3804. recordProfileStreams(pipeline, query_block.aggregation_name); restorePipelineConcurrency(pipeline); } else { - BlockInputStreamPtr stream_with_non_joined_data = combinedNonJoinedDataStream(pipeline, max_streams, log); BlockInputStreams inputs; if (!pipeline.streams.empty()) inputs.push_back(pipeline.firstStream()); - else - pipeline.streams.resize(1); - if (stream_with_non_joined_data) - inputs.push_back(stream_with_non_joined_data); + + if (!pipeline.streams_with_non_joined_data.empty()) + inputs.push_back(pipeline.streams_with_non_joined_data.at(0)); + + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::make_shared( std::make_shared(inputs, log->identifier()), params, @@ -421,56 +434,15 @@ void DAGQueryBlockInterpreter::executeAggregation( } } -void DAGQueryBlockInterpreter::executeExpression(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, const String & extra_info) -{ - if (!expressionActionsPtr->getActions().empty()) - { - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, expressionActionsPtr, log->identifier()); - stream->setExtraInfo(extra_info); - }); - } -} - -void DAGQueryBlockInterpreter::executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc) +void DAGQueryBlockInterpreter::executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc, bool enable_fine_grained_shuffle) { - orderStreams(pipeline, sort_desc, 0); + orderStreams(pipeline, max_streams, sort_desc, 0, enable_fine_grained_shuffle, context, log); } void DAGQueryBlockInterpreter::executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns) { Int64 limit = query_block.limit_or_topn->topn().limit(); - orderStreams(pipeline, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit); -} - -void DAGQueryBlockInterpreter::orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit) -{ - const Settings & settings = context.getSettingsRef(); - - pipeline.transform([&](auto & stream) { - auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); - - /// Limits on sorting - IProfilingBlockInputStream::LocalLimits limits; - limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; - limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sorting_stream->setLimits(limits); - - stream = sorting_stream; - }); - - /// If there are several streams, we merge them into one - executeUnion(pipeline, max_streams, log, false, "for partial order"); - - /// Merge the sorted blocks. - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), - order_descr, - settings.max_block_size, - limit, - settings.max_bytes_before_external_sort, - context.getTemporaryPath(), - log->identifier()); + orderStreams(pipeline, max_streams, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit, false, context, log); } void DAGQueryBlockInterpreter::recordProfileStreams(DAGPipeline & pipeline, const String & key) @@ -486,12 +458,25 @@ void DAGQueryBlockInterpreter::handleExchangeReceiver(DAGPipeline & pipeline) throw Exception("Can not find exchange receiver for " + query_block.source_name, ErrorCodes::LOGICAL_ERROR); // todo choose a more reasonable stream number auto & exchange_receiver_io_input_streams = dagContext().getInBoundIOInputStreamsMap()[query_block.source_name]; - for (size_t i = 0; i < max_streams; ++i) + + const bool enable_fine_grained_shuffle = enableFineGrainedShuffle(exchange_receiver->getFineGrainedShuffleStreamCount()); + String extra_info = "squashing after exchange receiver"; + size_t stream_count = max_streams; + if (enable_fine_grained_shuffle) + { + extra_info += ", " + enableFineGrainedShuffleExtraInfo; + stream_count = std::min(max_streams, exchange_receiver->getFineGrainedShuffleStreamCount()); + } + + for (size_t i = 0; i < stream_count; ++i) { - BlockInputStreamPtr stream = std::make_shared(exchange_receiver, log->identifier(), query_block.source_name); + BlockInputStreamPtr stream = std::make_shared(exchange_receiver, + log->identifier(), + query_block.source_name, + /*stream_id=*/enable_fine_grained_shuffle ? i : 0); exchange_receiver_io_input_streams.push_back(stream); stream = std::make_shared(stream, 8192, 0, log->identifier()); - stream->setExtraInfo("squashing after exchange receiver"); + stream->setExtraInfo(extra_info); pipeline.streams.push_back(stream); } NamesAndTypes source_columns; @@ -548,15 +533,12 @@ void DAGQueryBlockInterpreter::handleProjection(DAGPipeline & pipeline, const ti output_columns.emplace_back(alias, col.type); project_cols.emplace_back(col.name, alias); } - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, chain.getLastActions(), log->identifier()); - stream->setExtraInfo("before projection"); - }); + executeExpression(pipeline, chain.getLastActions(), log, "before projection"); executeProject(pipeline, project_cols, "projection"); analyzer = std::make_unique(std::move(output_columns), context); } -void DAGQueryBlockInterpreter::handleWindow(DAGPipeline & pipeline, const tipb::Window & window) +void DAGQueryBlockInterpreter::handleWindow(DAGPipeline & pipeline, const tipb::Window & window, bool enable_fine_grained_shuffle) { NamesAndTypes input_columns; assert(input_streams_vec.size() == 1); @@ -565,13 +547,13 @@ void DAGQueryBlockInterpreter::handleWindow(DAGPipeline & pipeline, const tipb:: input_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(input_columns, context); WindowDescription window_description = dag_analyzer.buildWindowDescription(window); - executeWindow(pipeline, window_description); - executeExpression(pipeline, window_description.after_window, "cast after window"); + executeWindow(pipeline, window_description, enable_fine_grained_shuffle); + executeExpression(pipeline, window_description.after_window, log, "cast after window"); analyzer = std::make_unique(window_description.after_window_columns, context); } -void DAGQueryBlockInterpreter::handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort) +void DAGQueryBlockInterpreter::handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort, bool enable_fine_grained_shuffle) { NamesAndTypes input_columns; assert(input_streams_vec.size() == 1); @@ -580,7 +562,7 @@ void DAGQueryBlockInterpreter::handleWindowOrder(DAGPipeline & pipeline, const t input_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(input_columns, context); auto order_columns = dag_analyzer.buildWindowOrderColumns(window_sort); - executeWindowOrder(pipeline, getSortDescription(order_columns, window_sort.byitems())); + executeWindowOrder(pipeline, getSortDescription(order_columns, window_sort.byitems()), enable_fine_grained_shuffle); analyzer = std::make_unique(std::move(input_columns), context); } @@ -628,13 +610,13 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) } else if (query_block.source->tp() == tipb::ExecType::TypeWindow) { - handleWindow(pipeline, query_block.source->window()); + handleWindow(pipeline, query_block.source->window(), enableFineGrainedShuffle(query_block.source->fine_grained_shuffle_stream_count())); recordProfileStreams(pipeline, query_block.source_name); restorePipelineConcurrency(pipeline); } else if (query_block.source->tp() == tipb::ExecType::TypeSort) { - handleWindowOrder(pipeline, query_block.source->sort()); + handleWindowOrder(pipeline, query_block.source->sort(), enableFineGrainedShuffle(query_block.source->fine_grained_shuffle_stream_count())); recordProfileStreams(pipeline, query_block.source_name); } else @@ -678,7 +660,7 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) } if (res.before_order_and_select) { - executeExpression(pipeline, res.before_order_and_select, "before order and select"); + executeExpression(pipeline, res.before_order_and_select, log, "before order and select"); } if (!res.order_columns.empty()) @@ -714,10 +696,7 @@ void DAGQueryBlockInterpreter::executeProject(DAGPipeline & pipeline, NamesWithA if (project_cols.empty()) return; ExpressionActionsPtr project = generateProjectExpressionActions(pipeline.firstStream(), context, project_cols); - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, project, log->identifier()); - stream->setExtraInfo(extra_info); - }); + executeExpression(pipeline, project, log, extra_info); } void DAGQueryBlockInterpreter::executeLimit(DAGPipeline & pipeline) @@ -743,19 +722,47 @@ void DAGQueryBlockInterpreter::handleExchangeSender(DAGPipeline & pipeline) std::vector partition_col_ids = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender); TiDB::TiDBCollators partition_col_collators = ExchangeSenderInterpreterHelper::genPartitionColCollators(exchange_sender); int stream_id = 0; - pipeline.transform([&](auto & stream) { - // construct writer - std::unique_ptr response_writer = std::make_unique>( - context.getDAGContext()->tunnel_set, - partition_col_ids, - partition_col_collators, - exchange_sender.tp(), - context.getSettingsRef().dag_records_per_chunk, - context.getSettingsRef().batch_send_min_limit, - stream_id++ == 0, /// only one stream needs to sending execution summaries for the last response - dagContext()); - stream = std::make_shared(stream, std::move(response_writer), log->identifier()); - }); + const uint64_t stream_count = query_block.exchange_sender->fine_grained_shuffle_stream_count(); + const uint64_t batch_size = query_block.exchange_sender->fine_grained_shuffle_batch_size(); + + if (enableFineGrainedShuffle(stream_count)) + { + pipeline.transform([&](auto & stream) { + // construct writer + std::unique_ptr response_writer = std::make_unique>( + context.getDAGContext()->tunnel_set, + partition_col_ids, + partition_col_collators, + exchange_sender.tp(), + context.getSettingsRef().dag_records_per_chunk, + context.getSettingsRef().batch_send_min_limit, + stream_id++ == 0, /// only one stream needs to sending execution summaries for the last response + dagContext(), + stream_count, + batch_size); + stream = std::make_shared(stream, std::move(response_writer), log->identifier()); + stream->setExtraInfo(enableFineGrainedShuffleExtraInfo); + }); + RUNTIME_CHECK(exchange_sender.tp() == tipb::ExchangeType::Hash, Exception, "exchange_sender has to be hash partition when fine grained shuffle is enabled"); + RUNTIME_CHECK(stream_count <= 1024, Exception, "fine_grained_shuffle_stream_count should not be greater than 1024"); + } + else + { + pipeline.transform([&](auto & stream) { + std::unique_ptr response_writer = std::make_unique>( + context.getDAGContext()->tunnel_set, + partition_col_ids, + partition_col_collators, + exchange_sender.tp(), + context.getSettingsRef().dag_records_per_chunk, + context.getSettingsRef().batch_send_min_limit, + stream_id++ == 0, /// only one stream needs to sending execution summaries for the last response + dagContext(), + stream_count, + batch_size); + stream = std::make_shared(stream, std::move(response_writer), log->identifier()); + }); + } } void DAGQueryBlockInterpreter::handleMockExchangeSender(DAGPipeline & pipeline) @@ -783,4 +790,4 @@ BlockInputStreams DAGQueryBlockInterpreter::execute() return pipeline.streams; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index e68c4f91cee..c449b37e360 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -64,17 +64,16 @@ class DAGQueryBlockInterpreter void handleExchangeReceiver(DAGPipeline & pipeline); void handleMockExchangeReceiver(DAGPipeline & pipeline); void handleProjection(DAGPipeline & pipeline, const tipb::Projection & projection); - void handleWindow(DAGPipeline & pipeline, const tipb::Window & window); - void handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort); + void handleWindow(DAGPipeline & pipeline, const tipb::Window & window, bool enable_fine_grained_shuffle); + void handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort, bool enable_fine_grained_shuffle); void executeWhere(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column, const String & extra_info = ""); - void executeExpression(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, const String & extra_info = ""); - void executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc); - void orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit); + void executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc, bool enable_fine_grained_shuffle); void executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns); void executeLimit(DAGPipeline & pipeline); void executeWindow( DAGPipeline & pipeline, - WindowDescription & window_description); + WindowDescription & window_description, + bool enable_fine_grained_shuffle); void executeAggregation( DAGPipeline & pipeline, const ExpressionActionsPtr & expression_actions_ptr, diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index ad2de7217e0..390ce7b9948 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -487,7 +487,7 @@ void DAGStorageInterpreter::buildRemoteStreams(std::vector && rem auto coprocessor_reader = std::make_shared(schema, cluster, tasks, has_enforce_encode_type, 1); context.getDAGContext()->addCoprocessorReader(coprocessor_reader); - BlockInputStreamPtr input = std::make_shared(coprocessor_reader, log->identifier(), table_scan.getTableScanExecutorID()); + BlockInputStreamPtr input = std::make_shared(coprocessor_reader, log->identifier(), table_scan.getTableScanExecutorID(), /*stream_id=*/0); pipeline.streams.push_back(input); task_start = task_end; } diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index a4d491e5637..2003103a20a 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -513,7 +513,7 @@ const std::unordered_map scalar_func_map({ //{tipb::ScalarFuncSig::YearWeekWithMode, "cast"}, //{tipb::ScalarFuncSig::YearWeekWithoutMode, "cast"}, - //{tipb::ScalarFuncSig::GetFormat, "cast"}, + {tipb::ScalarFuncSig::GetFormat, "getFormat"}, {tipb::ScalarFuncSig::SysDateWithFsp, "sysDateWithFsp"}, {tipb::ScalarFuncSig::SysDateWithoutFsp, "sysDateWithoutFsp"}, //{tipb::ScalarFuncSig::CurrentDate, "cast"}, diff --git a/dbms/src/Flash/Coprocessor/DecodeDetail.h b/dbms/src/Flash/Coprocessor/DecodeDetail.h index 9bad0ca2b72..91851650d9e 100644 --- a/dbms/src/Flash/Coprocessor/DecodeDetail.h +++ b/dbms/src/Flash/Coprocessor/DecodeDetail.h @@ -21,8 +21,12 @@ namespace DB /// Detail of the packet that decoding in TiRemoteInputStream.RemoteReader.decodeChunks() struct DecodeDetail { + // For fine grained shuffle, each ExchangeReceiver/thread will decode its own blocks. + // So this is the row number of partial blocks of the original packet. + // This will be the row number of all blocks of the original packet if it's not fine grained shuffle. Int64 rows = 0; - // byte size of origin packet. + + // Total byte size of the origin packet, even for fine grained shuffle. Int64 packet_bytes = 0; }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp index be3475f714f..efb8a08f1d8 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -54,4 +54,15 @@ ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_ } return column_with_type_and_names; } -} // namespace DB \ No newline at end of file + +NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema) +{ + NamesAndTypes names_and_types; + for (const auto & col : dag_schema) + { + auto tp = getDataTypeByColumnInfoForComputingLayer(col.second); + names_and_types.emplace_back(col.first, tp); + } + return names_and_types; +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h index 617f69de925..96f202d800e 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -23,4 +24,5 @@ namespace DB { NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan); ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_types); -} // namespace DB \ No newline at end of file +NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema); +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index c9810454218..002a06d07b9 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -12,8 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include #include #include +#include #include #include @@ -39,32 +43,6 @@ void restoreConcurrency( } } -BlockInputStreamPtr combinedNonJoinedDataStream( - DAGPipeline & pipeline, - size_t max_threads, - const LoggerPtr & log, - bool ignore_block) -{ - BlockInputStreamPtr ret = nullptr; - if (pipeline.streams_with_non_joined_data.size() == 1) - ret = pipeline.streams_with_non_joined_data.at(0); - else if (pipeline.streams_with_non_joined_data.size() > 1) - { - if (ignore_block) - { - ret = std::make_shared(pipeline.streams_with_non_joined_data, nullptr, max_threads, log->identifier()); - ret->setExtraInfo("combine non joined(ignore block)"); - } - else - { - ret = std::make_shared(pipeline.streams_with_non_joined_data, nullptr, max_threads, log->identifier()); - ret->setExtraInfo("combine non joined"); - } - } - pipeline.streams_with_non_joined_data.clear(); - return ret; -} - void executeUnion( DAGPipeline & pipeline, size_t max_streams, @@ -72,21 +50,33 @@ void executeUnion( bool ignore_block, const String & extra_info) { - if (pipeline.streams.size() == 1 && pipeline.streams_with_non_joined_data.empty()) - return; - auto non_joined_data_stream = combinedNonJoinedDataStream(pipeline, max_streams, log, ignore_block); - if (!pipeline.streams.empty()) + switch (pipeline.streams.size() + pipeline.streams_with_non_joined_data.size()) + { + case 0: + break; + case 1: { + if (pipeline.streams.size() == 1) + break; + // streams_with_non_joined_data's size is 1. + pipeline.streams.push_back(pipeline.streams_with_non_joined_data.at(0)); + pipeline.streams_with_non_joined_data.clear(); + break; + } + default: + { + BlockInputStreamPtr stream; if (ignore_block) - pipeline.firstStream() = std::make_shared(pipeline.streams, non_joined_data_stream, max_streams, log->identifier()); + stream = std::make_shared(pipeline.streams, pipeline.streams_with_non_joined_data, max_streams, log->identifier()); else - pipeline.firstStream() = std::make_shared(pipeline.streams, non_joined_data_stream, max_streams, log->identifier()); - pipeline.firstStream()->setExtraInfo(extra_info); + stream = std::make_shared(pipeline.streams, pipeline.streams_with_non_joined_data, max_streams, log->identifier()); + stream->setExtraInfo(extra_info); + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); + break; } - else if (non_joined_data_stream != nullptr) - { - pipeline.streams.push_back(non_joined_data_stream); } } @@ -102,4 +92,77 @@ ExpressionActionsPtr generateProjectExpressionActions( project->add(ExpressionAction::project(project_cols)); return project; } + +void executeExpression( + DAGPipeline & pipeline, + const ExpressionActionsPtr & expr_actions, + const LoggerPtr & log, + const String & extra_info) +{ + if (expr_actions && !expr_actions->getActions().empty()) + { + pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, expr_actions, log->identifier()); + stream->setExtraInfo(extra_info); + }); + } +} + +void orderStreams( + DAGPipeline & pipeline, + size_t max_streams, + SortDescription order_descr, + Int64 limit, + bool enable_fine_grained_shuffle, + const Context & context, + const LoggerPtr & log) +{ + const Settings & settings = context.getSettingsRef(); + String extra_info; + if (enable_fine_grained_shuffle) + extra_info = enableFineGrainedShuffleExtraInfo; + + pipeline.transform([&](auto & stream) { + auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); + + /// Limits on sorting + IProfilingBlockInputStream::LocalLimits limits; + limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sorting_stream->setLimits(limits); + + stream = sorting_stream; + stream->setExtraInfo(extra_info); + }); + + if (enable_fine_grained_shuffle) + { + pipeline.transform([&](auto & stream) { + stream = std::make_shared( + stream, + order_descr, + settings.max_block_size, + limit, + settings.max_bytes_before_external_sort, + context.getTemporaryPath(), + log->identifier()); + stream->setExtraInfo(enableFineGrainedShuffleExtraInfo); + }); + } + else + { + /// If there are several streams, we merge them into one + executeUnion(pipeline, max_streams, log, false, "for partial order"); + + /// Merge the sorted blocks. + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), + order_descr, + settings.max_block_size, + limit, + settings.max_bytes_before_external_sort, + context.getTemporaryPath(), + log->identifier()); + } +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 5c4d4721d5e..bd64346718c 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include @@ -44,4 +45,19 @@ ExpressionActionsPtr generateProjectExpressionActions( const BlockInputStreamPtr & stream, const Context & context, const NamesWithAliases & project_cols); + +void executeExpression( + DAGPipeline & pipeline, + const ExpressionActionsPtr & expr_actions, + const LoggerPtr & log, + const String & extra_info = ""); + +void orderStreams( + DAGPipeline & pipeline, + size_t max_streams, + SortDescription order_descr, + Int64 limit, + bool enable_fine_grained_shuffle, + const Context & context, + const LoggerPtr & log); } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp index f915653fe96..a72dfcc16ef 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp +++ b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp @@ -23,6 +23,8 @@ #include #include +#include + namespace DB { namespace ErrorCodes @@ -37,8 +39,8 @@ inline void serializeToPacket(mpp::MPPDataPacket & packet, const tipb::SelectRes throw Exception(fmt::format("Fail to serialize response, response size: {}", response.ByteSizeLong())); } -template -StreamingDAGResponseWriter::StreamingDAGResponseWriter( +template +StreamingDAGResponseWriter::StreamingDAGResponseWriter( StreamWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, @@ -46,7 +48,9 @@ StreamingDAGResponseWriter::StreamingDAGResponseWriter( Int64 records_per_chunk_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last_, - DAGContext & dag_context_) + DAGContext & dag_context_, + uint64_t fine_grained_shuffle_stream_count_, + UInt64 fine_grained_shuffle_batch_size_) : DAGResponseWriter(records_per_chunk_, dag_context_) , batch_send_min_limit(batch_send_min_limit_) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) @@ -54,6 +58,8 @@ StreamingDAGResponseWriter::StreamingDAGResponseWriter( , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) + , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); @@ -71,17 +77,37 @@ StreamingDAGResponseWriter::StreamingDAGResponseWriter( } } -template -void StreamingDAGResponseWriter::finishWrite() +template +void StreamingDAGResponseWriter::finishWrite() { if (should_send_exec_summary_at_last) - batchWrite(); + { + if constexpr (enable_fine_grained_shuffle) + { + assert(exchange_type == tipb::ExchangeType::Hash); + batchWriteFineGrainedShuffle(); + } + else + { + batchWrite(); + } + } else - batchWrite(); + { + if constexpr (enable_fine_grained_shuffle) + { + assert(exchange_type == tipb::ExchangeType::Hash); + batchWriteFineGrainedShuffle(); + } + else + { + batchWrite(); + } + } } -template -void StreamingDAGResponseWriter::write(const Block & block) +template +void StreamingDAGResponseWriter::write(const Block & block) { if (block.columns() != dag_context.result_field_types.size()) throw TiFlashException("Output column size mismatch with field type size", Errors::Coprocessor::Internal); @@ -91,15 +117,23 @@ void StreamingDAGResponseWriter::write(const Block & block) { blocks.push_back(block); } - if (static_cast(rows_in_blocks) > (dag_context.encode_type == tipb::EncodeType::TypeCHBlock ? batch_send_min_limit : records_per_chunk - 1)) + + if constexpr (enable_fine_grained_shuffle) { - batchWrite(); + assert(exchange_type == tipb::ExchangeType::Hash); + if (static_cast(rows_in_blocks) >= fine_grained_shuffle_batch_size) + batchWriteFineGrainedShuffle(); + } + else + { + if (static_cast(rows_in_blocks) > (dag_context.encode_type == tipb::EncodeType::TypeCHBlock ? batch_send_min_limit : records_per_chunk - 1)) + batchWrite(); } } -template +template template -void StreamingDAGResponseWriter::encodeThenWriteBlocks( +void StreamingDAGResponseWriter::encodeThenWriteBlocks( const std::vector & input_blocks, tipb::SelectResponse & response) const { @@ -191,133 +225,238 @@ void StreamingDAGResponseWriter::encodeThenWriteBlocks( } } -/// hash exchanging data among only TiFlash nodes. -template + +template template -void StreamingDAGResponseWriter::partitionAndEncodeThenWriteBlocks( - std::vector & input_blocks, - tipb::SelectResponse & response) const +void StreamingDAGResponseWriter::batchWrite() { - std::vector packet(partition_num); - - std::vector responses_row_count(partition_num); + tipb::SelectResponse response; + if constexpr (send_exec_summary_at_last) + addExecuteSummaries(response, !dag_context.isMPPTask() || dag_context.isRootMPPTask()); + if (exchange_type == tipb::ExchangeType::Hash) + { + partitionAndEncodeThenWriteBlocks(blocks, response); + } + else + { + encodeThenWriteBlocks(blocks, response); + } + blocks.clear(); + rows_in_blocks = 0; +} +template +template +void StreamingDAGResponseWriter::handleExecSummary( + const std::vector & input_blocks, + std::vector & packet, + tipb::SelectResponse & response) const +{ if constexpr (send_exec_summary_at_last) { /// Sending the response to only one node, default the first one. serializeToPacket(packet[0], response); - } - if (input_blocks.empty()) - { - if constexpr (send_exec_summary_at_last) + // No need to send data when blocks are not empty, + // because exec_summary will be sent together with blocks. + if (input_blocks.empty()) { for (auto part_id = 0; part_id < partition_num; ++part_id) { writer->write(packet[part_id], part_id); } } - return; } +} - // partition tuples in blocks - // 1) compute partition id - // 2) partition each row - // 3) encode each chunk and send it - std::vector partition_key_containers(collators.size()); - for (auto & block : input_blocks) +template +template +void StreamingDAGResponseWriter::writePackets(const std::vector & responses_row_count, + std::vector & packets) const +{ + for (size_t part_id = 0; part_id < packets.size(); ++part_id) { - std::vector dest_blocks(partition_num); - std::vector dest_tbl_cols(partition_num); - - for (size_t i = 0; i < block.columns(); ++i) + if constexpr (send_exec_summary_at_last) { - if (ColumnPtr converted = block.getByPosition(i).column->convertToFullColumnIfConst()) - { - block.getByPosition(i).column = converted; - } + writer->write(packets[part_id], part_id); } - - for (auto i = 0; i < partition_num; ++i) + else { - dest_tbl_cols[i] = block.cloneEmptyColumns(); - dest_blocks[i] = block.cloneEmpty(); + if (responses_row_count[part_id] > 0) + writer->write(packets[part_id], part_id); } + } +} - size_t rows = block.rows(); - WeakHash32 hash(rows); - - // get hash values by all partition key columns - for (size_t i = 0; i < partition_col_ids.size(); i++) +inline void initInputBlocks(std::vector & input_blocks) +{ + for (auto & input_block : input_blocks) + { + for (size_t i = 0; i < input_block.columns(); ++i) { - block.getByPosition(partition_col_ids[i]).column->updateWeakHash32(hash, collators[i], partition_key_containers[i]); + if (ColumnPtr converted = input_block.getByPosition(i).column->convertToFullColumnIfConst()) + input_block.getByPosition(i).column = converted; } - const auto & hash_data = hash.getData(); + } +} - // partition each row - IColumn::Selector selector(rows); - for (size_t row = 0; row < rows; ++row) - { - /// Row from interval [(2^32 / partition_num) * i, (2^32 / partition_num) * (i + 1)) goes to bucket with number i. - selector[row] = hash_data[row]; /// [0, 2^32) - selector[row] *= partition_num; /// [0, partition_num * 2^32), selector stores 64 bit values. - selector[row] >>= 32u; /// [0, partition_num) - } +inline void initDestColumns(const Block & input_block, std::vector & dest_tbl_cols) +{ + for (auto & cols : dest_tbl_cols) + { + cols = input_block.cloneEmptyColumns(); + } +} - for (size_t col_id = 0; col_id < block.columns(); ++col_id) - { - // Scatter columns to different partitions - auto scattered_columns = block.getByPosition(col_id).column->scatter(partition_num, selector); - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - dest_tbl_cols[part_id][col_id] = std::move(scattered_columns[part_id]); - } - } - // serialize each partitioned block and write it to its destination - for (auto part_id = 0; part_id < partition_num; ++part_id) - { - dest_blocks[part_id].setColumns(std::move(dest_tbl_cols[part_id])); - responses_row_count[part_id] += dest_blocks[part_id].rows(); - chunk_codec_stream->encode(dest_blocks[part_id], 0, dest_blocks[part_id].rows()); - packet[part_id].add_chunks(chunk_codec_stream->getString()); - chunk_codec_stream->clear(); - } +void computeHash(const Block & input_block, + uint32_t bucket_num, + const TiDB::TiDBCollators & collators, + std::vector & partition_key_containers, + const std::vector & partition_col_ids, + std::vector> & result_columns) +{ + size_t rows = input_block.rows(); + WeakHash32 hash(rows); + + // get hash values by all partition key columns + for (size_t i = 0; i < partition_col_ids.size(); ++i) + { + input_block.getByPosition(partition_col_ids[i]).column->updateWeakHash32(hash, collators[i], partition_key_containers[i]); } - for (auto part_id = 0; part_id < partition_num; ++part_id) + const auto & hash_data = hash.getData(); + + // partition each row + IColumn::Selector selector(rows); + for (size_t row = 0; row < rows; ++row) { - if constexpr (send_exec_summary_at_last) + /// Row from interval [(2^32 / bucket_num) * i, (2^32 / bucket_num) * (i + 1)) goes to bucket with number i. + selector[row] = hash_data[row]; /// [0, 2^32) + selector[row] *= bucket_num; /// [0, bucket_num * 2^32), selector stores 64 bit values. + selector[row] >>= 32u; /// [0, bucket_num) + } + + for (size_t col_id = 0; col_id < input_block.columns(); ++col_id) + { + // Scatter columns to different partitions + std::vector part_columns = input_block.getByPosition(col_id).column->scatter(bucket_num, selector); + assert(part_columns.size() == bucket_num); + for (size_t bucket_idx = 0; bucket_idx < bucket_num; ++bucket_idx) { - writer->write(packet[part_id], part_id); + result_columns[bucket_idx][col_id] = std::move(part_columns[bucket_idx]); } - else + } +} + +/// Hash exchanging data among only TiFlash nodes. Only be called when enable_fine_grained_shuffle is false. +template +template +void StreamingDAGResponseWriter::partitionAndEncodeThenWriteBlocks( + std::vector & input_blocks, + tipb::SelectResponse & response) const +{ + static_assert(!enable_fine_grained_shuffle); + std::vector packet(partition_num); + std::vector responses_row_count(partition_num); + handleExecSummary(input_blocks, packet, response); + if (input_blocks.empty()) + return; + + initInputBlocks(input_blocks); + Block dest_block = input_blocks[0].cloneEmpty(); + std::vector partition_key_containers(collators.size()); + for (const auto & block : input_blocks) + { + std::vector dest_tbl_cols(partition_num); + initDestColumns(block, dest_tbl_cols); + + computeHash(block, partition_num, collators, partition_key_containers, partition_col_ids, dest_tbl_cols); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) { - if (responses_row_count[part_id] > 0) - writer->write(packet[part_id], part_id); + dest_block.setColumns(std::move(dest_tbl_cols[part_id])); + responses_row_count[part_id] += dest_block.rows(); + chunk_codec_stream->encode(dest_block, 0, dest_block.rows()); + packet[part_id].add_chunks(chunk_codec_stream->getString()); + chunk_codec_stream->clear(); } } + + writePackets(responses_row_count, packet); } -template +/// Hash exchanging data among only TiFlash nodes. Only be called when enable_fine_grained_shuffle is true. +template template -void StreamingDAGResponseWriter::batchWrite() +void StreamingDAGResponseWriter::batchWriteFineGrainedShuffle() { + static_assert(enable_fine_grained_shuffle); + assert(exchange_type == tipb::ExchangeType::Hash); + assert(fine_grained_shuffle_stream_count <= 1024); + tipb::SelectResponse response; if constexpr (send_exec_summary_at_last) addExecuteSummaries(response, !dag_context.isMPPTask() || dag_context.isRootMPPTask()); - if (exchange_type == tipb::ExchangeType::Hash) - { - partitionAndEncodeThenWriteBlocks(blocks, response); - } - else + + std::vector packet(partition_num); + std::vector responses_row_count(partition_num, 0); + + // fine_grained_shuffle_stream_count is in [0, 1024], and partition_num is uint16_t, so will not overflow. + uint32_t bucket_num = partition_num * fine_grained_shuffle_stream_count; + handleExecSummary(blocks, packet, response); + if (!blocks.empty()) { - encodeThenWriteBlocks(blocks, response); + std::vector final_dest_tbl_columns(bucket_num); + initInputBlocks(blocks); + initDestColumns(blocks[0], final_dest_tbl_columns); + + // Hash partition input_blocks into bucket_num. + for (const auto & block : blocks) + { + std::vector partition_key_containers(collators.size()); + std::vector dest_tbl_columns(bucket_num); + initDestColumns(block, dest_tbl_columns); + computeHash(block, bucket_num, collators, partition_key_containers, partition_col_ids, dest_tbl_columns); + for (size_t bucket_idx = 0; bucket_idx < bucket_num; ++bucket_idx) + { + for (size_t col_id = 0; col_id < block.columns(); ++col_id) + { + const MutableColumnPtr & src_col = dest_tbl_columns[bucket_idx][col_id]; + final_dest_tbl_columns[bucket_idx][col_id]->insertRangeFrom(*src_col, 0, src_col->size()); + } + } + } + + // For i-th stream_count buckets, send to i-th tiflash node. + for (size_t bucket_idx = 0; bucket_idx < bucket_num; bucket_idx += fine_grained_shuffle_stream_count) + { + size_t part_id = bucket_idx / fine_grained_shuffle_stream_count; // NOLINT(clang-analyzer-core.DivideZero) + size_t row_count_per_part = 0; + for (uint64_t stream_idx = 0; stream_idx < fine_grained_shuffle_stream_count; ++stream_idx) + { + Block dest_block = blocks[0].cloneEmpty(); + // For now we put all rows into one Block, may cause this Block too large. + dest_block.setColumns(std::move(final_dest_tbl_columns[bucket_idx + stream_idx])); + row_count_per_part += dest_block.rows(); + + chunk_codec_stream->encode(dest_block, 0, dest_block.rows()); + packet[part_id].add_chunks(chunk_codec_stream->getString()); + packet[part_id].add_stream_ids(stream_idx); + chunk_codec_stream->clear(); + } + responses_row_count[part_id] = row_count_per_part; + } } + + writePackets(responses_row_count, packet); + blocks.clear(); rows_in_blocks = 0; } -template class StreamingDAGResponseWriter; -template class StreamingDAGResponseWriter; +template class StreamingDAGResponseWriter; +template class StreamingDAGResponseWriter; +template class StreamingDAGResponseWriter; +template class StreamingDAGResponseWriter; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h index 9b5e3864c64..cd7559d1e79 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h +++ b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h @@ -33,7 +33,7 @@ namespace DB /// Serializes the stream of blocks and sends them to TiDB or TiFlash with different serialization paths. /// When sending data to TiDB, blocks with extra info are written into tipb::SelectResponse, then the whole tipb::SelectResponse is further serialized into mpp::MPPDataPacket.data. /// Differently when sending data to TiFlash, blocks with only tuples are directly serialized into mpp::MPPDataPacket.chunks, but for the last block, its extra info (like execution summaries) is written into tipb::SelectResponse, then further serialized into mpp::MPPDataPacket.data. -template +template class StreamingDAGResponseWriter : public DAGResponseWriter { public: @@ -45,18 +45,30 @@ class StreamingDAGResponseWriter : public DAGResponseWriter Int64 records_per_chunk_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last, - DAGContext & dag_context_); + DAGContext & dag_context_, + UInt64 fine_grained_shuffle_stream_count_, + UInt64 fine_grained_shuffle_batch_size); void write(const Block & block) override; void finishWrite() override; private: template void batchWrite(); + template + void batchWriteFineGrainedShuffle(); + template void encodeThenWriteBlocks(const std::vector & input_blocks, tipb::SelectResponse & response) const; template void partitionAndEncodeThenWriteBlocks(std::vector & input_blocks, tipb::SelectResponse & response) const; + template + void handleExecSummary(const std::vector & input_blocks, + std::vector & packet, + tipb::SelectResponse & response) const; + template + void writePackets(const std::vector & responses_row_count, std::vector & packets) const; + Int64 batch_send_min_limit; bool should_send_exec_summary_at_last; /// only one stream needs to sending execution summaries at last. tipb::ExchangeType exchange_type; @@ -67,6 +79,8 @@ class StreamingDAGResponseWriter : public DAGResponseWriter size_t rows_in_blocks; uint16_t partition_num; std::unique_ptr chunk_codec_stream; + UInt64 fine_grained_shuffle_stream_count; + UInt64 fine_grained_shuffle_batch_size; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_streaming_dag_writer.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_streaming_dag_writer.cpp new file mode 100644 index 00000000000..5d4186123b7 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/tests/gtest_streaming_dag_writer.cpp @@ -0,0 +1,184 @@ +// 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. + +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ +namespace tests +{ + +using BlockPtr = std::shared_ptr; +class TestStreamingDAGResponseWriter : public testing::Test +{ +protected: + void SetUp() override + { + dag_context_ptr = std::make_unique(1024); + dag_context_ptr->encode_type = tipb::EncodeType::TypeCHBlock; + dag_context_ptr->is_mpp_task = true; + dag_context_ptr->is_root_mpp_task = false; + dag_context_ptr->result_field_types = makeFields(); + context.setDAGContext(dag_context_ptr.get()); + } + +public: + TestStreamingDAGResponseWriter() + : context(TiFlashTestEnv::getContext()) + , part_col_ids{0} + , part_col_collators{ + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::BINARY)} + {} + + // Return 10 Int64 column. + static std::vector makeFields() + { + std::vector fields(10); + for (int i = 0; i < 10; ++i) + { + fields[i].set_tp(TiDB::TypeLongLong); + } + return fields; + } + + // Return a block with **rows** and 10 Int64 column. + static BlockPtr prepareBlock(const std::vector & rows) + { + BlockPtr block = std::make_shared(); + for (int i = 0; i < 10; ++i) + { + DataTypePtr int64_data_type = std::make_shared(); + DataTypePtr nullable_int64_data_type = std::make_shared(int64_data_type); + MutableColumnPtr int64_col = nullable_int64_data_type->createColumn(); + for (Int64 r : rows) + { + int64_col->insert(Field(r)); + } + block->insert(ColumnWithTypeAndName{std::move(int64_col), + nullable_int64_data_type, + String("col") + std::to_string(i)}); + } + return block; + } + + Context context; + std::vector part_col_ids; + TiDB::TiDBCollators part_col_collators; + + std::unique_ptr dag_context_ptr; +}; + +using MockStreamWriterChecker = std::function; + +struct MockStreamWriter +{ + MockStreamWriter(MockStreamWriterChecker checker_, + uint16_t part_num_) + : checker(checker_) + , part_num(part_num_) + {} + + void write(mpp::MPPDataPacket &) { FAIL() << "cannot reach here, because we only expect hash partition"; } + void write(mpp::MPPDataPacket & packet, uint16_t part_id) { checker(packet, part_id); } + void write(tipb::SelectResponse &, uint16_t) { FAIL() << "cannot reach here, only consider CH Block format"; } + void write(tipb::SelectResponse &) { FAIL() << "cannot reach here, only consider CH Block format"; } + uint16_t getPartitionNum() const { return part_num; } + +private: + MockStreamWriterChecker checker; + uint16_t part_num; +}; + +// Input block data is distributed uniform. +// partition_num: 4 +// fine_grained_shuffle_stream_count: 8 +TEST_F(TestStreamingDAGResponseWriter, testBatchWriteFineGrainedShuffle) +try +{ + const size_t block_rows = 1024; + const uint16_t part_num = 4; + const uint32_t fine_grained_shuffle_stream_count = 8; + const Int64 fine_grained_shuffle_batch_size = 4096; + + // Set these to 1, because when fine grained shuffle is enabled, + // batchWriteFineGrainedShuffle() only check fine_grained_shuffle_batch_size. + // records_per_chunk and batch_send_min_limit are useless. + const Int64 records_per_chunk = 1; + const Int64 batch_send_min_limit = 1; + const bool should_send_exec_summary_at_last = true; + + // 1. Build Block. + std::vector uniform_data_set; + for (size_t i = 0; i < block_rows; ++i) + { + uniform_data_set.push_back(i); + } + BlockPtr block = prepareBlock(uniform_data_set); + + // 2. Build MockStreamWriter. + std::unordered_map write_report; + auto checker = [&write_report](mpp::MPPDataPacket & packet, uint16_t part_id) { + auto res = write_report.insert({part_id, packet}); + // Should always insert succeed. + // Because block.rows(1024) < fine_grained_shuffle_batch_size(4096), + // batchWriteFineGrainedShuffle() only called once, so will only be one packet for each partition. + ASSERT_TRUE(res.second); + }; + auto mock_writer = std::make_shared(checker, part_num); + + // 3. Start to write. + auto dag_writer = std::make_shared, /*enable_fine_grained_shuffle=*/true>>( + mock_writer, + part_col_ids, + part_col_collators, + tipb::ExchangeType::Hash, + records_per_chunk, + batch_send_min_limit, + should_send_exec_summary_at_last, + *dag_context_ptr, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size); + dag_writer->write(*block); + dag_writer->finishWrite(); + + // 4. Start to check write_report. + std::vector decoded_blocks; + ASSERT_EQ(write_report.size(), part_num); + for (const auto & ele : write_report) + { + const mpp::MPPDataPacket & packet = ele.second; + ASSERT_EQ(packet.chunks_size(), packet.stream_ids_size()); + for (int i = 0; i < packet.chunks_size(); ++i) + { + decoded_blocks.push_back(CHBlockChunkCodec::decode(packet.chunks(i), *block)); + } + } + ASSERT_EQ(decoded_blocks.size(), fine_grained_shuffle_stream_count * part_num); + for (const auto & block : decoded_blocks) + { + ASSERT_EQ(block.rows(), block_rows / (fine_grained_shuffle_stream_count * part_num)); + } +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 966babb832f..ab8d83a1481 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -48,6 +49,106 @@ String getReceiverStateStr(const ExchangeReceiverState & s) } } +// If enable_fine_grained_shuffle: +// Seperate chunks according to packet.stream_ids[i], then push to msg_channels[stream_id]. +// If fine grained_shuffle is disabled: +// Push all chunks to msg_channels[0]. +// Return true if all push succeed, otherwise return false. +// NOTE: shared_ptr will be hold by all ExchangeReceiverBlockInputStream to make chunk pointer valid. +template +bool pushPacket(size_t source_index, + const String & req_info, + MPPDataPacketPtr & packet, + const std::vector & msg_channels, + LoggerPtr & log) +{ + bool push_succeed = true; + + const mpp::Error * error_ptr = nullptr; + if (packet->has_error()) + error_ptr = &packet->error(); + const String * resp_ptr = nullptr; + if (!packet->data().empty()) + resp_ptr = &packet->data(); + + if constexpr (enable_fine_grained_shuffle) + { + std::vector> chunks(msg_channels.size()); + if (!packet->chunks().empty()) + { + // Packet not empty. + if (unlikely(packet->stream_ids().empty())) + { + // Fine grained shuffle is enabled in receiver, but sender didn't. We cannot handle this, so return error. + // This can happen when there are old version nodes when upgrading. + LOG_FMT_ERROR(log, "MPPDataPacket.stream_ids empty, it means ExchangeSender is old version of binary " + "(source_index: {}) while fine grained shuffle of ExchangeReceiver is enabled. " + "Cannot handle this.", + source_index); + return false; + } + // packet.stream_ids[i] is corresponding to packet.chunks[i], + // indicating which stream_id this chunk belongs to. + assert(packet->chunks_size() == packet->stream_ids_size()); + + for (int i = 0; i < packet->stream_ids_size(); ++i) + { + UInt64 stream_id = packet->stream_ids(i) % msg_channels.size(); + chunks[stream_id].push_back(&packet->chunks(i)); + } + } + // Still need to send error_ptr or resp_ptr even if packet.chunks_size() is zero. + for (size_t i = 0; i < msg_channels.size() && push_succeed; ++i) + { + if (resp_ptr == nullptr && error_ptr == nullptr && chunks[i].empty()) + continue; + + std::shared_ptr recv_msg = std::make_shared( + source_index, + req_info, + packet, + error_ptr, + resp_ptr, + std::move(chunks[i])); + push_succeed = msg_channels[i]->push(std::move(recv_msg)); + if constexpr (is_sync) + fiu_do_on(FailPoints::random_receiver_sync_msg_push_failure_failpoint, push_succeed = false;); + else + fiu_do_on(FailPoints::random_receiver_async_msg_push_failure_failpoint, push_succeed = false;); + + // Only the first ExchangeReceiverInputStream need to handle resp. + resp_ptr = nullptr; + } + } + else + { + std::vector chunks(packet->chunks_size()); + for (int i = 0; i < packet->chunks_size(); ++i) + { + chunks[i] = &packet->chunks(i); + } + + if (!(resp_ptr == nullptr && error_ptr == nullptr && chunks.empty())) + { + std::shared_ptr recv_msg = std::make_shared( + source_index, + req_info, + packet, + error_ptr, + resp_ptr, + std::move(chunks)); + + push_succeed = msg_channels[0]->push(std::move(recv_msg)); + if constexpr (is_sync) + fiu_do_on(FailPoints::random_receiver_sync_msg_push_failure_failpoint, push_succeed = false;); + else + fiu_do_on(FailPoints::random_receiver_async_msg_push_failure_failpoint, push_succeed = false;); + } + } + LOG_FMT_DEBUG(log, "push recv_msg to msg_channels(size: {}) succeed:{}, enable_fine_grained_shuffle: {}", msg_channels.size(), push_succeed, enable_fine_grained_shuffle); + return push_succeed; +} + enum class AsyncRequestStage { NEED_INIT, @@ -64,25 +165,25 @@ using TimePoint = Clock::time_point; constexpr Int32 max_retry_times = 10; constexpr Int32 batch_packet_count = 16; -template +template class AsyncRequestHandler : public UnaryCallback { public: using Status = typename RPCContext::Status; using Request = typename RPCContext::Request; using AsyncReader = typename RPCContext::AsyncReader; - using Self = AsyncRequestHandler; + using Self = AsyncRequestHandler; AsyncRequestHandler( MPMCQueue * queue, - MPMCQueue> * msg_channel_, + std::vector * msg_channels_, const std::shared_ptr & context, const Request & req, const String & req_id) : rpc_context(context) , request(&req) , notify_queue(queue) - , msg_channel(msg_channel_) + , msg_channels(msg_channels_) , req_info(fmt::format("tunnel{}+{}", req.send_task_id, req.recv_task_id)) , log(Logger::get("ExchangeReceiver", req_id, req_info)) { @@ -260,13 +361,7 @@ class AsyncRequestHandler : public UnaryCallback for (size_t i = 0; i < read_packet_index; ++i) { auto & packet = packets[i]; - auto recv_msg = std::make_shared(); - recv_msg->packet = std::move(packet); - recv_msg->source_index = request->source_index; - recv_msg->req_info = req_info; - bool push_success = msg_channel->push(std::move(recv_msg)); - fiu_do_on(FailPoints::random_receiver_async_msg_push_failure_failpoint, push_success = false;); - if (!push_success) + if (!pushPacket(request->source_index, req_info, packet, *msg_channels, log)) return false; // can't reuse packet since it is sent to readers. packet = std::make_shared(); @@ -283,7 +378,7 @@ class AsyncRequestHandler : public UnaryCallback std::shared_ptr rpc_context; const Request * request; // won't be null MPMCQueue * notify_queue; // won't be null - MPMCQueue> * msg_channel; // won't be null + std::vector * msg_channels; // won't be null String req_info; bool meet_error = false; @@ -308,20 +403,32 @@ ExchangeReceiverBase::ExchangeReceiverBase( size_t source_num_, size_t max_streams_, const String & req_id, - const String & executor_id) + const String & executor_id, + uint64_t fine_grained_shuffle_stream_count_) : rpc_context(std::move(rpc_context_)) , source_num(source_num_) , max_streams(max_streams_) , max_buffer_size(std::max(batch_packet_count, std::max(source_num, max_streams_) * 2)) , thread_manager(newThreadManager()) - , msg_channel(max_buffer_size) , live_connections(source_num) , state(ExchangeReceiverState::NORMAL) , exc_log(Logger::get("ExchangeReceiver", req_id, executor_id)) , collected(false) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { try { + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count_)) + { + for (size_t i = 0; i < max_streams_; ++i) + { + msg_channels.push_back(std::make_unique>>(max_buffer_size)); + } + } + else + { + msg_channels.push_back(std::make_unique>>(max_buffer_size)); + } rpc_context->fillSchema(schema); setUpConnection(); } @@ -358,14 +465,14 @@ template void ExchangeReceiverBase::cancel() { setEndState(ExchangeReceiverState::CANCELED); - msg_channel.cancel(); + cancelAllMsgChannels(); } template void ExchangeReceiverBase::close() { setEndState(ExchangeReceiverState::CLOSED); - msg_channel.finish(); + finishAllMsgChannels(); } template @@ -380,7 +487,12 @@ void ExchangeReceiverBase::setUpConnection() async_requests.push_back(std::move(req)); else { - thread_manager->schedule(true, "Receiver", [this, req = std::move(req)] { readLoop(req); }); + thread_manager->schedule(true, "Receiver", [this, req = std::move(req)] { + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) + readLoop(req); + else + readLoop(req); + }); ++thread_count; } } @@ -388,15 +500,21 @@ void ExchangeReceiverBase::setUpConnection() // TODO: reduce this thread in the future. if (!async_requests.empty()) { - thread_manager->schedule(true, "RecvReactor", [this, async_requests = std::move(async_requests)] { reactor(async_requests); }); + thread_manager->schedule(true, "RecvReactor", [this, async_requests = std::move(async_requests)] { + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) + reactor(async_requests); + else + reactor(async_requests); + }); ++thread_count; } } template +template void ExchangeReceiverBase::reactor(const std::vector & async_requests) { - using AsyncHandler = AsyncRequestHandler; + using AsyncHandler = AsyncRequestHandler; GET_METRIC(tiflash_thread_count, type_threads_of_receiver_reactor).Increment(); SCOPE_EXIT({ @@ -412,7 +530,7 @@ void ExchangeReceiverBase::reactor(const std::vector & asyn std::vector> handlers; handlers.reserve(alive_async_connections); for (const auto & req : async_requests) - handlers.emplace_back(std::make_unique(&ready_requests, &msg_channel, rpc_context, req, exc_log->identifier())); + handlers.emplace_back(std::make_unique(&ready_requests, &msg_channels, rpc_context, req, exc_log->identifier())); while (alive_async_connections > 0) { @@ -457,6 +575,7 @@ void ExchangeReceiverBase::reactor(const std::vector & asyn } template +template void ExchangeReceiverBase::readLoop(const Request & req) { GET_METRIC(tiflash_thread_count, type_threads_of_receiver_read_loop).Increment(); @@ -481,20 +600,15 @@ void ExchangeReceiverBase::readLoop(const Request & req) for (;;) { LOG_FMT_TRACE(log, "begin next "); - auto recv_msg = std::make_shared(); - recv_msg->packet = std::make_shared(); - recv_msg->req_info = req_info; - recv_msg->source_index = req.source_index; - bool success = reader->read(recv_msg->packet); + MPPDataPacketPtr packet = std::make_shared(); + bool success = reader->read(packet); if (!success) break; has_data = true; - if (recv_msg->packet->has_error()) - throw Exception("Exchange receiver meet error : " + recv_msg->packet->error().msg()); + if (packet->has_error()) + throw Exception("Exchange receiver meet error : " + packet->error().msg()); - bool push_success = msg_channel.push(std::move(recv_msg)); - fiu_do_on(FailPoints::random_receiver_sync_msg_push_failure_failpoint, push_success = false;); - if (!push_success) + if (!pushPacket(req.source_index, req_info, packet, msg_channels, log)) { meet_error = true; auto local_state = getState(); @@ -564,15 +678,15 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( assert(recv_msg != nullptr); DecodeDetail detail; - int chunk_size = recv_msg->packet->chunks_size(); - if (chunk_size == 0) + if (recv_msg->chunks.empty()) return detail; + // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = recv_msg->packet->ByteSizeLong(); - /// ExchangeReceiverBase should receive chunks of TypeCHBlock - for (int i = 0; i < chunk_size; ++i) + + for (const String * chunk : recv_msg->chunks) { - Block block = CHBlockChunkCodec::decode(recv_msg->packet->chunks(i), header); + Block block = CHBlockChunkCodec::decode(*chunk, header); detail.rows += block.rows(); if (unlikely(block.rows() == 0)) continue; @@ -582,10 +696,15 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( } template -ExchangeReceiverResult ExchangeReceiverBase::nextResult(std::queue & block_queue, const Block & header) +ExchangeReceiverResult ExchangeReceiverBase::nextResult(std::queue & block_queue, const Block & header, size_t stream_id) { + if (unlikely(stream_id >= msg_channels.size())) + { + LOG_FMT_ERROR(exc_log, "stream_id out of range, stream_id: {}, total_stream_count: {}", stream_id, msg_channels.size()); + return {nullptr, 0, "", true, "stream_id out of range", false}; + } std::shared_ptr recv_msg; - if (!msg_channel.pop(recv_msg)) + if (!msg_channels[stream_id]->pop(recv_msg)) { std::unique_lock lock(mu); @@ -607,29 +726,32 @@ ExchangeReceiverResult ExchangeReceiverBase::nextResult(std::queuepacket != nullptr); + assert(recv_msg != nullptr); ExchangeReceiverResult result; - if (recv_msg->packet->has_error()) + if (recv_msg->error_ptr != nullptr) { - result = {nullptr, recv_msg->source_index, recv_msg->req_info, true, recv_msg->packet->error().msg(), false}; + result = {nullptr, recv_msg->source_index, recv_msg->req_info, true, recv_msg->error_ptr->msg(), false}; } else { - if (!recv_msg->packet->data().empty()) /// the data of the last packet is serialized from tipb::SelectResponse including execution summaries. + if (recv_msg->resp_ptr != nullptr) /// the data of the last packet is serialized from tipb::SelectResponse including execution summaries. { - auto resp_ptr = std::make_shared(); - if (!resp_ptr->ParseFromString(recv_msg->packet->data())) + auto select_resp = std::make_shared(); + if (!select_resp->ParseFromString(*(recv_msg->resp_ptr))) { result = {nullptr, recv_msg->source_index, recv_msg->req_info, true, "decode error", false}; } else { - result = {resp_ptr, recv_msg->source_index, recv_msg->req_info, false, "", false}; - /// If mocking TiFlash as TiDB, here should decode chunks from resp_ptr. - if (!resp_ptr->chunks().empty()) + result = {select_resp, recv_msg->source_index, recv_msg->req_info, false, "", false}; + /// If mocking TiFlash as TiDB, here should decode chunks from select_resp. + if (!select_resp->chunks().empty()) { - assert(recv_msg->packet->chunks().empty()); - result.decode_detail = CoprocessorReader::decodeChunks(resp_ptr, block_queue, header, schema); + assert(recv_msg->chunks.empty()); + // Fine grained shuffle should only be enabled when sending data to TiFlash node. + // So all data should be encoded into MPPDataPacket.chunks. + RUNTIME_CHECK(!enableFineGrainedShuffle(fine_grained_shuffle_stream_count), Exception, "Data should not be encoded into tipb::SelectResponse.chunks when fine grained shuffle is enabled"); + result.decode_detail = CoprocessorReader::decodeChunks(select_resp, block_queue, header, schema); } } } @@ -637,7 +759,7 @@ ExchangeReceiverResult ExchangeReceiverBase::nextResult(std::queuesource_index, recv_msg->req_info, false, "", false}; } - if (!result.meet_error && !recv_msg->packet->chunks().empty()) + if (!result.meet_error && !recv_msg->chunks.empty()) { assert(result.decode_detail.rows == 0); result.decode_detail = decodeChunks(recv_msg, block_queue, header); @@ -699,7 +821,21 @@ void ExchangeReceiverBase::connectionDone( throw Exception("live_connections should not be less than 0!"); if (meet_error || copy_live_conn == 0) - msg_channel.finish(); + finishAllMsgChannels(); +} + +template +void ExchangeReceiverBase::finishAllMsgChannels() +{ + for (auto & msg_channel : msg_channels) + msg_channel->finish(); +} + +template +void ExchangeReceiverBase::cancelAllMsgChannels() +{ + for (auto & msg_channel : msg_channels) + msg_channel->cancel(); } /// Explicit template instantiations - to avoid code bloat in headers. diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.h b/dbms/src/Flash/Mpp/ExchangeReceiver.h index 830dc6241a9..708f133f226 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.h +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.h @@ -35,9 +35,28 @@ namespace DB { struct ReceivedMessage { - std::shared_ptr packet; - size_t source_index = 0; + size_t source_index; String req_info; + // shared_ptr is copied to make sure error_ptr, resp_ptr and chunks are valid. + const std::shared_ptr packet; + const mpp::Error * error_ptr; + const String * resp_ptr; + std::vector chunks; + + // Constructor that move chunks. + ReceivedMessage(size_t source_index_, + const String & req_info_, + const std::shared_ptr & packet_, + const mpp::Error * error_ptr_, + const String * resp_ptr_, + std::vector && chunks_) + : source_index(source_index_) + , req_info(req_info_) + , packet(packet_) + , error_ptr(error_ptr_) + , resp_ptr(resp_ptr_) + , chunks(chunks_) + {} }; struct ExchangeReceiverResult @@ -78,6 +97,7 @@ enum class ExchangeReceiverState CLOSED, }; +using MsgChannelPtr = std::unique_ptr>>; template class ExchangeReceiverBase @@ -92,7 +112,8 @@ class ExchangeReceiverBase size_t source_num_, size_t max_streams_, const String & req_id, - const String & executor_id); + const String & executor_id, + uint64_t fine_grained_shuffle_stream_count); ~ExchangeReceiverBase(); @@ -104,9 +125,11 @@ class ExchangeReceiverBase ExchangeReceiverResult nextResult( std::queue & block_queue, - const Block & header); + const Block & header, + size_t stream_id); size_t getSourceNum() const { return source_num; } + uint64_t getFineGrainedShuffleStreamCount() const { return fine_grained_shuffle_stream_count; } int computeNewThreadCount() const { return thread_count; } @@ -128,7 +151,10 @@ class ExchangeReceiverBase using Request = typename RPCContext::Request; void setUpConnection(); + // Template argument enable_fine_grained_shuffle will be setup properly in setUpConnection(). + template void readLoop(const Request & req); + template void reactor(const std::vector & async_requests); bool setEndState(ExchangeReceiverState new_state); @@ -139,12 +165,14 @@ class ExchangeReceiverBase std::queue & block_queue, const Block & header); - void connectionDone( bool meet_error, const String & local_err_msg, const LoggerPtr & log); + void finishAllMsgChannels(); + void cancelAllMsgChannels(); + std::shared_ptr rpc_context; const tipb::ExchangeReceiver pb_exchange_receiver; @@ -156,7 +184,7 @@ class ExchangeReceiverBase std::shared_ptr thread_manager; DAGSchema schema; - MPMCQueue> msg_channel; + std::vector msg_channels; std::mutex mu; /// should lock `mu` when visit these members @@ -168,6 +196,7 @@ class ExchangeReceiverBase bool collected = false; int thread_count = 0; + uint64_t fine_grained_shuffle_stream_count; }; class ExchangeReceiver : public ExchangeReceiverBase diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index da8f3034abc..7ddc6af361f 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -172,7 +172,8 @@ void MPPTask::initExchangeReceivers() executor.exchange_receiver().encoded_task_meta_size(), context->getMaxStreams(), log->identifier(), - executor_id); + executor_id, + executor.fine_grained_shuffle_stream_count()); if (status != RUNNING) throw Exception("exchange receiver map can not be initialized, because the task is not in running state"); diff --git a/dbms/src/Flash/tests/WindowTestUtil.h b/dbms/src/Flash/tests/WindowTestUtil.h index 3f4cb7d595f..b7385380419 100644 --- a/dbms/src/Flash/tests/WindowTestUtil.h +++ b/dbms/src/Flash/tests/WindowTestUtil.h @@ -39,9 +39,9 @@ inline void mockExecuteProject(std::shared_ptr & mock_ mock_interpreter->executeProject(pipeline, final_project); } -inline void mockExecuteWindowOrder(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Sort & sort) +inline void mockExecuteWindowOrder(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Sort & sort, uint64_t fine_grained_shuffle_stream_count) { - mock_interpreter->handleWindowOrder(pipeline, sort); + mock_interpreter->handleWindowOrder(pipeline, sort, ::DB::enableFineGrainedShuffle(fine_grained_shuffle_stream_count)); mock_interpreter->input_streams_vec[0] = pipeline.streams; NamesWithAliases final_project; for (const auto & column : (*mock_interpreter->analyzer).source_columns) @@ -51,16 +51,9 @@ inline void mockExecuteWindowOrder(std::shared_ptr & m mockExecuteProject(mock_interpreter, pipeline, final_project); } -inline void mockExecuteWindowOrder(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const String & sort_json) +inline void mockExecuteWindow(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Window & window, uint64_t fine_grained_shuffle_stream_count) { - tipb::Sort sort; - ::google::protobuf::util::JsonStringToMessage(sort_json, &sort); - mockExecuteWindowOrder(mock_interpreter, pipeline, sort); -} - -inline void mockExecuteWindow(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Window & window) -{ - mock_interpreter->handleWindow(pipeline, window); + mock_interpreter->handleWindow(pipeline, window, ::DB::enableFineGrainedShuffle(fine_grained_shuffle_stream_count)); mock_interpreter->input_streams_vec[0] = pipeline.streams; NamesWithAliases final_project; for (const auto & column : (*mock_interpreter->analyzer).source_columns) @@ -70,12 +63,5 @@ inline void mockExecuteWindow(std::shared_ptr & mock_i mockExecuteProject(mock_interpreter, pipeline, final_project); } -inline void mockExecuteWindow(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, std::string window_json_str) -{ - tipb::Window window; - google::protobuf::util::JsonStringToMessage(window_json_str, &window); - mockExecuteWindow(mock_interpreter, pipeline, window); -} - } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/bench_exchange.cpp b/dbms/src/Flash/tests/bench_exchange.cpp index fbb53bfd4a4..d6e3f3e825e 100644 --- a/dbms/src/Flash/tests/bench_exchange.cpp +++ b/dbms/src/Flash/tests/bench_exchange.cpp @@ -47,29 +47,46 @@ MockFixedRowsBlockInputStream::MockFixedRowsBlockInputStream(size_t total_rows_, , blocks(blocks_) {} -Block makeBlock(int row_num) +Block makeBlock(int row_num, bool skew) { - std::mt19937 mt(rd()); - std::uniform_int_distribution int64_dist; - std::uniform_int_distribution len_dist(10, 20); - std::uniform_int_distribution char_dist; - InferredDataVector> int64_vec; InferredDataVector> int64_vec2; - for (int i = 0; i < row_num; ++i) + InferredDataVector> string_vec; + + if (skew) { - int64_vec.emplace_back(int64_dist(mt)); - int64_vec2.emplace_back(int64_dist(mt)); + for (int i = 0; i < row_num; ++i) + { + int64_vec.emplace_back(100); + int64_vec2.emplace_back(100); + } + + for (int i = 0; i < row_num; ++i) + { + string_vec.push_back("abcdefg"); + } } - - InferredDataVector> string_vec; - for (int i = 0; i < row_num; ++i) + else { - int len = len_dist(mt); - String s; - for (int j = 0; j < len; ++j) - s.push_back(char_dist(mt)); - string_vec.push_back(std::move(s)); + std::mt19937 mt(rd()); + std::uniform_int_distribution int64_dist; + std::uniform_int_distribution len_dist(10, 20); + std::uniform_int_distribution char_dist; + + for (int i = 0; i < row_num; ++i) + { + int64_vec.emplace_back(int64_dist(mt)); + int64_vec2.emplace_back(int64_dist(mt)); + } + + for (int i = 0; i < row_num; ++i) + { + int len = len_dist(mt); + String s; + for (int j = 0; j < len; ++j) + s.push_back(char_dist(mt)); + string_vec.push_back(std::move(s)); + } } auto int64_data_type = makeDataType>(); @@ -82,11 +99,11 @@ Block makeBlock(int row_num) return Block({int64_column, string_column, int64_column2}); } -std::vector makeBlocks(int block_num, int row_num) +std::vector makeBlocks(int block_num, int row_num, bool skew) { std::vector blocks; for (int i = 0; i < block_num; ++i) - blocks.push_back(makeBlock(row_num)); + blocks.push_back(makeBlock(row_num, skew)); return blocks; } @@ -139,32 +156,10 @@ void printException(const Exception & e) << e.getStackTrace().toString() << std::endl; } -void sendPacket(const std::vector & packets, const PacketQueuePtr & queue, StopFlag & stop_flag) -{ - std::mt19937 mt(rd()); - std::uniform_int_distribution dist(0, packets.size() - 1); - - while (!stop_flag.load()) - { - int i = dist(mt); - queue->tryPush(packets[i], std::chrono::milliseconds(10)); - } - queue->finish(); -} - -void receivePacket(const PacketQueuePtr & queue) -{ - while (true) - { - PacketPtr packet; - if (!queue->pop(packet)) - break; - } -} - -ReceiverHelper::ReceiverHelper(int concurrency_, int source_num_) +ReceiverHelper::ReceiverHelper(int concurrency_, int source_num_, uint32_t fine_grained_shuffle_stream_count_) : concurrency(concurrency_) , source_num(source_num_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { pb_exchange_receiver.set_tp(tipb::Hash); for (int i = 0; i < source_num; ++i) @@ -198,16 +193,21 @@ MockExchangeReceiverPtr ReceiverHelper::buildReceiver() source_num, concurrency, "mock_req_id", - "mock_exchange_receiver_id"); + "mock_exchange_receiver_id", + fine_grained_shuffle_stream_count); } std::vector ReceiverHelper::buildExchangeReceiverStream() { auto receiver = buildReceiver(); std::vector streams(concurrency); + // NOTE: check if need fine_grained_shuffle_stream_count for (int i = 0; i < concurrency; ++i) { - streams[i] = std::make_shared(receiver, "mock_req_id", "mock_executor_id" + std::to_string(i)); + streams[i] = std::make_shared(receiver, + "mock_req_id", + "mock_executor_id" + std::to_string(i), + /*stream_id=*/enableFineGrainedShuffle(fine_grained_shuffle_stream_count) ? i : 0); } return streams; } @@ -215,7 +215,7 @@ std::vector ReceiverHelper::buildExchangeReceiverStream() BlockInputStreamPtr ReceiverHelper::buildUnionStream() { auto streams = buildExchangeReceiverStream(); - return std::make_shared>(streams, nullptr, concurrency, /*req_id=*/""); + return std::make_shared>(streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } void ReceiverHelper::finish() @@ -230,10 +230,14 @@ void ReceiverHelper::finish() SenderHelper::SenderHelper( int source_num_, int concurrency_, + uint32_t fine_grained_shuffle_stream_count_, + int64_t fine_grained_shuffle_batch_size_, const std::vector & queues_, const std::vector & fields) : source_num(source_num_) , concurrency(concurrency_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) + , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) , queues(queues_) { mpp::TaskMeta task_meta; @@ -277,20 +281,41 @@ BlockInputStreamPtr SenderHelper::buildUnionStream( for (int i = 0; i < concurrency; ++i) { BlockInputStreamPtr stream = std::make_shared(blocks, stop_flag); - std::unique_ptr response_writer( - new StreamingDAGResponseWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - tipb::Hash, - -1, - -1, - true, - *dag_context)); - send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) + { + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } + else + { + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } } - return std::make_shared>(send_streams, nullptr, concurrency, /*req_id=*/""); + return std::make_shared>(send_streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } BlockInputStreamPtr SenderHelper::buildUnionStream(size_t total_rows, const std::vector & blocks) @@ -299,20 +324,41 @@ BlockInputStreamPtr SenderHelper::buildUnionStream(size_t total_rows, const std: for (int i = 0; i < concurrency; ++i) { BlockInputStreamPtr stream = std::make_shared(total_rows / concurrency, blocks); - std::unique_ptr response_writer( - new StreamingDAGResponseWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - tipb::Hash, - -1, - -1, - true, - *dag_context)); - send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) + { + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } + else + { + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } } - return std::make_shared>(send_streams, nullptr, concurrency, /*req_id=*/""); + return std::make_shared>(send_streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } void SenderHelper::finish() @@ -327,13 +373,12 @@ void SenderHelper::finish() void ExchangeBench::SetUp(const benchmark::State &) { - Poco::Logger::root().setLevel("error"); - DynamicThreadPool::global_instance = std::make_unique( /*fixed_thread_num=*/300, std::chrono::milliseconds(100000)); - input_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024); + uniform_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024); + skew_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024, /*skew=*/true); try { @@ -348,7 +393,8 @@ void ExchangeBench::SetUp(const benchmark::State &) void ExchangeBench::TearDown(const benchmark::State &) { - input_blocks.clear(); + uniform_blocks.clear(); + skew_blocks.clear(); // NOTE: Must reset here, otherwise DynamicThreadPool::fixedWork() may core because metrics already destroyed. DynamicThreadPool::global_instance.reset(); } @@ -383,25 +429,38 @@ try const int concurrency = state.range(0); const int source_num = state.range(1); const int total_rows = state.range(2); + const int fine_grained_shuffle_stream_count = state.range(3); + const int fine_grained_shuffle_batch_size = state.range(4); Context context = TiFlashTestEnv::getContext(); for (auto _ : state) { - std::shared_ptr receiver_helper = std::make_shared(concurrency, source_num); + std::shared_ptr receiver_helper = std::make_shared(concurrency, source_num, fine_grained_shuffle_stream_count); BlockInputStreamPtr receiver_stream = receiver_helper->buildUnionStream(); std::shared_ptr sender_helper = std::make_shared(source_num, concurrency, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size, receiver_helper->queues, receiver_helper->fields); - BlockInputStreamPtr sender_stream = sender_helper->buildUnionStream(total_rows, input_blocks); + BlockInputStreamPtr sender_stream = sender_helper->buildUnionStream(total_rows, uniform_blocks); runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); } } CATCH BENCHMARK_REGISTER_F(ExchangeBench, basic_send_receive) - ->Args({8, 1, 1024 * 1000}); + ->Args({8, 1, 1024 * 1000, 0, 4096}) + ->Args({8, 1, 1024 * 1000, 4, 4096}) + ->Args({8, 1, 1024 * 1000, 8, 4096}) + ->Args({8, 1, 1024 * 1000, 16, 4096}) + ->Args({8, 1, 1024 * 1000, 32, 4096}) + ->Args({8, 1, 1024 * 1000, 8, 1}) + ->Args({8, 1, 1024 * 1000, 8, 1000}) + ->Args({8, 1, 1024 * 1000, 8, 10000}) + ->Args({8, 1, 1024 * 1000, 8, 100000}); + } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/bench_exchange.h b/dbms/src/Flash/tests/bench_exchange.h index 6b09e319613..d8300d45740 100644 --- a/dbms/src/Flash/tests/bench_exchange.h +++ b/dbms/src/Flash/tests/bench_exchange.h @@ -69,7 +69,9 @@ struct MockReceiverContext : queue(queue_) {} - void initialize() const {} + void initialize() const + { + } bool read(PacketPtr & packet [[maybe_unused]]) const { @@ -105,7 +107,8 @@ struct MockReceiverContext const std::vector & field_types_) : queues(queues_) , field_types(field_types_) - {} + { + } void fillSchema(DAGSchema & schema) const { @@ -220,8 +223,8 @@ struct MockFixedRowsBlockInputStream : public IProfilingBlockInputStream } }; -Block makeBlock(int row_num); -std::vector makeBlocks(int block_num, int row_num); +Block makeBlock(int row_num, bool skew = false); +std::vector makeBlocks(int block_num, int row_num, bool skew = false); mpp::MPPDataPacket makePacket(ChunkCodecStream & codec, int row_num); std::vector makePackets(ChunkCodecStream & codec, int packet_num, int row_num); std::vector makePacketQueues(int source_num, int queue_size); @@ -234,17 +237,17 @@ struct ReceiverHelper { const int concurrency; const int source_num; + const uint32_t fine_grained_shuffle_stream_count; tipb::ExchangeReceiver pb_exchange_receiver; std::vector fields; mpp::TaskMeta task_meta; std::vector queues; std::shared_ptr join_ptr; - explicit ReceiverHelper(int concurrency_, int source_num_); + explicit ReceiverHelper(int concurrency_, int source_num_, uint32_t fine_grained_shuffle_stream_count_); MockExchangeReceiverPtr buildReceiver(); std::vector buildExchangeReceiverStream(); BlockInputStreamPtr buildUnionStream(); - BlockInputStreamPtr buildUnionStreamWithHashJoinBuildStream(); void finish(); }; @@ -252,6 +255,8 @@ struct SenderHelper { const int source_num; const int concurrency; + const uint32_t fine_grained_shuffle_stream_count; + const int64_t fine_grained_shuffle_batch_size; std::vector queues; std::vector mock_writers; @@ -262,6 +267,8 @@ struct SenderHelper SenderHelper( int source_num_, int concurrency_, + uint32_t fine_grained_shuffle_stream_count_, + int64_t fine_grained_shuffle_batch_size_, const std::vector & queues_, const std::vector & fields); @@ -283,7 +290,8 @@ class ExchangeBench : public benchmark::Fixture std::shared_ptr & sender_helper, BlockInputStreamPtr sender_stream); - std::vector input_blocks; + std::vector uniform_blocks; + std::vector skew_blocks; }; diff --git a/dbms/src/Flash/tests/bench_window.cpp b/dbms/src/Flash/tests/bench_window.cpp index dfdb358c46c..75dc53b065b 100644 --- a/dbms/src/Flash/tests/bench_window.cpp +++ b/dbms/src/Flash/tests/bench_window.cpp @@ -24,9 +24,13 @@ class WindowFunctionBench : public ExchangeBench public: void SetUp(const benchmark::State & state) override { - // build tipb::Window and tipb::Sort. + // Using DAGRequestBuilder to build tipb::Window and tipb::Sort. // select row_number() over w1 from t1 window w1 as (partition by c1, c2, c3 order by c1, c2, c3); ExchangeBench::SetUp(state); + } + + static void setupPB(uint64_t fine_grained_shuffle_stream_count, tipb::Window & window, tipb::Sort & sort) + { MockColumnInfoVec columns{ {"c1", TiDB::TP::TypeLongLong}, {"c2", TiDB::TP::TypeString}, @@ -36,11 +40,12 @@ class WindowFunctionBench : public ExchangeBench DAGRequestBuilder builder(executor_index); builder .mockTable("test", "t1", columns) - .sort({{"c1", false}, {"c2", false}, {"c3", false}}, true) + .sort({{"c1", false}, {"c2", false}, {"c3", false}}, true, fine_grained_shuffle_stream_count) .window(RowNumber(), {{"c1", false}, {"c2", false}, {"c3", false}}, {{"c1", false}, {"c2", false}, {"c3", false}}, - buildDefaultRowsFrame()); + buildDefaultRowsFrame(), + fine_grained_shuffle_stream_count); tipb::DAGRequest req; MPPInfo mpp_info(0, -1, -1, {}, std::unordered_map>{}); builder.getRoot()->toTiPBExecutor(req.mutable_root_executor(), /*collator_id=*/0, mpp_info, TiFlashTestEnv::getContext()); @@ -50,13 +55,17 @@ class WindowFunctionBench : public ExchangeBench sort = window.child().sort(); } - void prepareWindowStream(Context & context, int concurrency, int source_num, int total_rows, const std::vector & blocks, BlockInputStreamPtr & sender_stream, BlockInputStreamPtr & receiver_stream, std::shared_ptr & sender_helper, std::shared_ptr & receiver_helper) const + static void prepareWindowStream(Context & context, int concurrency, int source_num, int total_rows, uint32_t fine_grained_shuffle_stream_count, uint64_t fine_grained_shuffle_batch_size, const std::vector & blocks, BlockInputStreamPtr & sender_stream, BlockInputStreamPtr & receiver_stream, std::shared_ptr & sender_helper, std::shared_ptr & receiver_helper, bool build_window = true) { + tipb::Window window; + tipb::Sort sort; + setupPB(fine_grained_shuffle_stream_count, window, sort); + DAGPipeline pipeline; - receiver_helper = std::make_shared(concurrency, source_num); + receiver_helper = std::make_shared(concurrency, source_num, fine_grained_shuffle_stream_count); pipeline.streams = receiver_helper->buildExchangeReceiverStream(); - sender_helper = std::make_shared(source_num, concurrency, receiver_helper->queues, receiver_helper->fields); + sender_helper = std::make_shared(source_num, concurrency, fine_grained_shuffle_stream_count, fine_grained_shuffle_batch_size, receiver_helper->queues, receiver_helper->fields); sender_stream = sender_helper->buildUnionStream(total_rows, blocks); context.setDAGContext(sender_helper->dag_context.get()); @@ -66,16 +75,16 @@ class WindowFunctionBench : public ExchangeBench NameAndTypePair("c3", makeNullable(std::make_shared()))}; auto mock_interpreter = mockInterpreter(context, source_columns, concurrency); mock_interpreter->input_streams_vec.push_back(pipeline.streams); - mockExecuteWindowOrder(mock_interpreter, pipeline, sort); - mockExecuteWindow(mock_interpreter, pipeline, window); + mockExecuteWindowOrder(mock_interpreter, pipeline, sort, fine_grained_shuffle_stream_count); + if (build_window) + { + mockExecuteWindow(mock_interpreter, pipeline, window, fine_grained_shuffle_stream_count); + } pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, 8192, 0, "mock_executor_id_squashing"); }); - receiver_stream = std::make_shared>(pipeline.streams, nullptr, concurrency, /*req_id=*/""); + receiver_stream = std::make_shared>(pipeline.streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } - - tipb::Window window; - tipb::Sort sort; }; BENCHMARK_DEFINE_F(WindowFunctionBench, basic_row_number) @@ -85,8 +94,15 @@ try const int concurrency = state.range(0); const int source_num = state.range(1); const int total_rows = state.range(2); + const int fine_grained_shuffle_stream_count = state.range(3); + const int fine_grained_shuffle_batch_size = state.range(4); + const bool skew = state.range(5); Context context = TiFlashTestEnv::getContext(); + std::vector * blocks = &uniform_blocks; + if (skew) + blocks = &skew_blocks; + for (auto _ : state) { std::shared_ptr sender_helper; @@ -94,14 +110,58 @@ try BlockInputStreamPtr sender_stream; BlockInputStreamPtr receiver_stream; - prepareWindowStream(context, concurrency, source_num, total_rows, input_blocks, sender_stream, receiver_stream, sender_helper, receiver_helper); + prepareWindowStream(context, concurrency, source_num, total_rows, fine_grained_shuffle_stream_count, fine_grained_shuffle_batch_size, *blocks, sender_stream, receiver_stream, sender_helper, receiver_helper); runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); } } CATCH BENCHMARK_REGISTER_F(WindowFunctionBench, basic_row_number) - ->Args({8, 1, 1024 * 1000}); + ->Args({8, 1, 1024 * 1000, 0, 4096, false}) // Test fine_grained_shuffle_stream_count. + ->Args({8, 1, 1024 * 1000, 4, 4096, false}) + ->Args({8, 1, 1024 * 1000, 8, 4096, false}) + ->Args({8, 1, 1024 * 1000, 16, 4096, false}) + ->Args({8, 1, 1024 * 1000, 32, 4096, false}) + ->Args({8, 1, 1024 * 1000, 8, 1, false}) // Test fine_grained_shuffle_batch_size. + ->Args({8, 1, 1024 * 1000, 8, 1000, false}) + ->Args({8, 1, 1024 * 1000, 8, 10000, false}) + ->Args({8, 1, 1024 * 1000, 8, 100000, false}) + ->Args({8, 1, 1024 * 1000, 0, 4096, true}) // Test skew dataset. + ->Args({8, 1, 1024 * 1000, 4, 4096, true}) + ->Args({8, 1, 1024 * 1000, 8, 4096, true}) + ->Args({8, 1, 1024 * 1000, 16, 4096, true}); + +BENCHMARK_DEFINE_F(WindowFunctionBench, partial_sort_skew_dataset) +(benchmark::State & state) +try +{ + const int concurrency = state.range(0); + const int source_num = state.range(1); + const int total_rows = state.range(2); + const int fine_grained_shuffle_stream_count = state.range(3); + const int fine_grained_shuffle_batch_size = state.range(4); + Context context = TiFlashTestEnv::getContext(); + std::vector * blocks = &skew_blocks; + + for (auto _ : state) + { + std::shared_ptr sender_helper; + std::shared_ptr receiver_helper; + BlockInputStreamPtr sender_stream; + BlockInputStreamPtr receiver_stream; + + // Only build partial sort. + prepareWindowStream(context, concurrency, source_num, total_rows, fine_grained_shuffle_stream_count, fine_grained_shuffle_batch_size, *blocks, sender_stream, receiver_stream, sender_helper, receiver_helper, /*build_window=*/false); + + runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); + } +} +CATCH +BENCHMARK_REGISTER_F(WindowFunctionBench, partial_sort_skew_dataset) + ->Args({1, 1, 1024 * 10000, 0, 4096}) // Test how much multiple-thread improves performance for partial sort. + ->Args({2, 1, 1024 * 10000, 0, 4096}) + ->Args({4, 1, 1024 * 10000, 0, 4096}) + ->Args({8, 1, 1024 * 10000, 0, 4096}); } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index 2d4d4014780..75a0857465e 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -31,8 +31,8 @@ class InterpreterExecuteTest : public DB::tests::ExecutorTest 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.addExchangeRelationSchema("sender_1", {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); - context.addExchangeRelationSchema("sender_l", {{"l_a", TiDB::TP::TypeString}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); - context.addExchangeRelationSchema("sender_r", {{"r_a", TiDB::TP::TypeString}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_l", {{"l_a", TiDB::TP::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_r", {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); } }; @@ -103,15 +103,12 @@ try Union: Expression x 10: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: - Expression: - Expression: - Expression: - MockTableScan)"; + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -125,16 +122,14 @@ Union: Union: Expression x 10: Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -150,22 +145,18 @@ Union: Union: Expression x 10: Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -188,68 +179,22 @@ Union: Limit x 10, limit = 10 Expression: Expression: - Expression: - Expression: - Expression: - Filter: - Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - // 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"); - - request = table1.join( - table2.join( - table3.join(table4, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) - .build(context); - { - String expected = R"( -CreatingSets - Union: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - MockTableScan - Union x 2: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - Expression: - HashJoinProbe: Expression: - MockTableScan - Union: - Expression x 10: - Expression: - HashJoinProbe: - Expression: - MockTableScan)"; + Expression: + Filter: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -263,15 +208,12 @@ CreatingSets Union: Expression x 10: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -287,35 +229,230 @@ Union: MockExchangeSender x 10 Expression: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(InterpreterExecuteTest, 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} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + 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: + Expression: + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + 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: + Expression: + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Union: + Expression x 10: Expression: - Expression: + SharedQuery: Expression: - MockExchangeReceiver)"; + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } +} +CATCH - // 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"); - - request = receiver1.join( - receiver2.join( - receiver3.join(receiver4, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) +TEST_F(InterpreterExecuteTest, 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 10: + Expression: + Window: , function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting: , limit = 0 + PartialSorting: : limit = 0 + Expression: + 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: + SharedQuery x 10: + Expression: + 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} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + 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(InterpreterExecuteTest, 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, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union x 2: + HashJoinBuildBlockInputStream 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, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .build(context); + String expected = R"( CreatingSets Union: @@ -340,24 +477,25 @@ CreatingSets ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } - // join + receiver + sender - // TODO: Find a way to write the request easier. - DAGRequestBuilder receiver5 = context.receive("sender_l"); - DAGRequestBuilder receiver6 = context.receive("sender_r"); - DAGRequestBuilder receiver7 = context.receive("sender_l"); - DAGRequestBuilder receiver8 = context.receive("sender_r"); - request = receiver5.join( - receiver6.join( - receiver7.join(receiver8, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) - .exchangeSender(tipb::PassThrough) - .build(context); { + // 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, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .exchangeSender(tipb::PassThrough) + .build(context); + String expected = R"( CreatingSets Union: @@ -385,85 +523,111 @@ CreatingSets } CATCH -TEST_F(InterpreterExecuteTest, Window) +TEST_F(InterpreterExecuteTest, JoinThenAgg) try { - auto request = context - .scan("test_db", "test_table") - .sort({{"s1", true}, {"s2", false}}, true) - .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) - .build(context); { + // Left Join. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + {col("join_c")}, + ASTTableJoin::Kind::Left) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); String expected = R"( -Union: - Expression x 10: - SharedQuery: - Expression: - Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} - Expression: - MergeSorting, limit = 0 - Union: - PartialSorting x 10: limit = 0 - Expression: - MockTableScan)"; +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + HashJoinProbe: + Expression: + 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); { + // Right Join + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + {col("join_c")}, + ASTTableJoin::Kind::Right) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); String expected = R"( -Union: - Expression x 10: - Expression: - Expression: +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Right + Expression: Expression: - SharedQuery: - Expression: - Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MockTableScan + Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + HashJoinProbe: + Expression: Expression: - MergeSorting, limit = 0 - Union: - PartialSorting x 10: limit = 0 - Expression: - MockTableScan)"; + MockTableScan + Expression x 10: + NonJoined: )"; 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); { + // Right join + receiver + sender + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2, + {col("join_c")}, + ASTTableJoin::Kind::Right) + .aggregation({Sum(col("r_a"))}, {col("join_c")}) + .exchangeSender(tipb::PassThrough) + .limit(10) + .build(context); String expected = R"( -Union: - Expression x 10: - Expression: - Expression: +CreatingSets + Union: + HashJoinBuildBlockInputStream x 20: , join_kind = Right + Expression: Expression: - SharedQuery: - Expression: - Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} - Union: - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 0 - Union: - PartialSorting x 10: limit = 0 - Expression: - MockTableScan)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + MockExchangeReceiver + Union: + MockExchangeSender x 20 + SharedQuery: + Limit, limit = 10 + Union: + Limit x 20, limit = 10 + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 20, final: true + Expression x 20: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver + Expression x 20: + NonJoined: )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 20); } } CATCH } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Functions/FunctionsConversion.cpp b/dbms/src/Functions/FunctionsConversion.cpp index 118574ed33d..0446f76bd51 100644 --- a/dbms/src/Functions/FunctionsConversion.cpp +++ b/dbms/src/Functions/FunctionsConversion.cpp @@ -240,6 +240,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction>(); factory.registerFunction>(); factory.registerFunction>(); diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index ddf64a70ca1..e8333ceeeea 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1751,6 +1751,120 @@ class FunctionDateFormat : public IFunction } }; +class FunctionGetFormat : public IFunction +{ +private: + static String get_format(const StringRef & time_type, const StringRef & location) + { + if (time_type == "DATE") + { + if (location == "USA") + return "%m.%d.%Y"; + else if (location == "JIS") + return "%Y-%m-%d"; + else if (location == "ISO") + return "%Y-%m-%d"; + else if (location == "EUR") + return "%d.%m.%Y"; + else if (location == "INTERNAL") + return "%Y%m%d"; + } + else if (time_type == "DATETIME" || time_type == "TIMESTAMP") + { + if (location == "USA") + return "%Y-%m-%d %H.%i.%s"; + else if (location == "JIS") + return "%Y-%m-%d %H:%i:%s"; + else if (location == "ISO") + return "%Y-%m-%d %H:%i:%s"; + else if (location == "EUR") + return "%Y-%m-%d %H.%i.%s"; + else if (location == "INTERNAL") + return "%Y%m%d%H%i%s"; + } + else if (time_type == "TIME") + { + if (location == "USA") + return "%h:%i:%s %p"; + else if (location == "JIS") + return "%H:%i:%s"; + else if (location == "ISO") + return "%H:%i:%s"; + else if (location == "EUR") + return "%H.%i.%s"; + else if (location == "INTERNAL") + return "%H%i%s"; + } + return ""; + } + +public: + static constexpr auto name = "getFormat"; + static FunctionPtr create(const Context &) { return std::make_shared(); }; + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!arguments[0].type->isString()) + throw Exception("First argument for function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN); + if (!arguments[1].type->isString()) + throw Exception("Second argument for function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + /** + * @brief The first argument is designed as a MySQL reserved word. You would encounter a syntax error when wrap it around with quote in SQL. + * For example, select GET_FORMAT("DATE", "USA") will fail. Removing the quote can solve the problem. + * Thus the first argument should always be a ColumnConst. See details in the link below: + * https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_get-format + * + * @return ColumnNumbers + */ + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) const override + { + const auto * location_col = checkAndGetColumn(block.getByPosition(arguments[1]).column.get()); + assert(location_col); + size_t size = location_col->size(); + const auto & time_type_col = block.getByPosition(arguments[0]).column; + auto col_to = ColumnString::create(); + + if (time_type_col->isColumnConst()) + { + const auto & time_type_col_const = checkAndGetColumnConst(time_type_col.get()); + const auto & time_type = time_type_col_const->getValue(); + + ColumnString::Chars_t & data_to = col_to->getChars(); + ColumnString::Offsets & offsets_to = col_to->getOffsets(); + auto max_length = 18; + data_to.resize(size * max_length); + offsets_to.resize(size); + WriteBufferFromVector write_buffer(data_to); + for (size_t i = 0; i < size; ++i) + { + const auto & location = location_col->getDataAt(i); + const auto & result = get_format(StringRef(time_type), location); + write_buffer.write(result.c_str(), result.size()); + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + } + data_to.resize(write_buffer.count()); + block.getByPosition(result).column = std::move(col_to); + } + else + { + throw Exception("First argument for function " + getName() + " must be String constant", ErrorCodes::ILLEGAL_COLUMN); + } + } +}; + struct NameStrToDateDate { static constexpr auto name = "strToDateDate"; diff --git a/dbms/src/Functions/FunctionsString.cpp b/dbms/src/Functions/FunctionsString.cpp index ad75c69a090..76022b983ad 100644 --- a/dbms/src/Functions/FunctionsString.cpp +++ b/dbms/src/Functions/FunctionsString.cpp @@ -4573,7 +4573,9 @@ class FormatImpl : public IFunction using NumberFieldType = typename NumberType::FieldType; using NumberColVec = std::conditional_t, ColumnDecimal, ColumnVector>; const auto * number_raw = block.getByPosition(arguments[0]).column.get(); + TiDBDecimalRoundInfo info{number_type, number_type}; + info.output_prec = info.output_prec < 65 ? info.output_prec + 1 : 65; return getPrecisionType(precision_base_type, [&](const auto & precision_type, bool) { using PrecisionType = std::decay_t; @@ -4723,10 +4725,11 @@ class FormatImpl : public IFunction static void format( T number, size_t max_num_decimals, - const TiDBDecimalRoundInfo & info, + TiDBDecimalRoundInfo & info, ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets) { + info.output_scale = std::min(max_num_decimals, static_cast(info.input_scale)); auto round_number = round(number, max_num_decimals, info); std::string round_number_str = number2Str(round_number, info); std::string buffer = Format::apply(round_number_str, max_num_decimals); diff --git a/dbms/src/Functions/tests/gtest_get_format.cpp b/dbms/src/Functions/tests/gtest_get_format.cpp new file mode 100644 index 00000000000..61a8d80e7b4 --- /dev/null +++ b/dbms/src/Functions/tests/gtest_get_format.cpp @@ -0,0 +1,153 @@ +// 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. + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wsign-compare" +#include + +#pragma GCC diagnostic pop + +namespace DB::tests +{ +class GetFormatTest : public DB::tests::FunctionTest +{ +public: + static constexpr auto funcName = "getFormat"; +}; + +TEST_F(GetFormatTest, testBoundary) +try +{ + // const(non-null), vector + // time_type is a const with non null value + // location is a vector containing null + ASSERT_COLUMN_EQ( + createColumn>({"%m.%d.%Y", {}}), + executeFunction( + funcName, + createConstColumn>(2, "DATE"), + createColumn>({"USA", {}}))); + + // const(null), vector + // time_type is a const with null value + // location is a vector containing null + ASSERT_COLUMN_EQ( + createConstColumn>(2, {}), + executeFunction( + funcName, + createConstColumn>(2, {}), + createColumn>({"USA", {}}))); + + // const(non-null), const(non-null) + // time_type is a const with non null value + // location is a const with non null value + ASSERT_COLUMN_EQ( + createConstColumn(2, "%m.%d.%Y"), + executeFunction( + funcName, + createConstColumn>(2, "DATE"), + createConstColumn>(2, "USA"))); + + // const(non-null), const(null) + // time_type is a const with non null value + // location is a const with null value + ASSERT_COLUMN_EQ( + createConstColumn>(2, {}), + executeFunction( + funcName, + createConstColumn>(2, "DATE"), + createConstColumn>(2, {}))); + + // The time_type is a system pre_defined macro, thus assume time_type column is const + // Throw an exception is time_type is not ColumnConst + ASSERT_THROW( + executeFunction( + funcName, + createColumn>({"DATE", "TIME"}), + createColumn>({"USA", {}})), + DB::Exception); +} +CATCH + +TEST_F(GetFormatTest, testMoreCases) +try +{ + // time_type: DATE + // all locations + ASSERT_COLUMN_EQ( + createColumn>({"%m.%d.%Y", "%Y-%m-%d", "%Y-%m-%d", "%d.%m.%Y", "%Y%m%d"}), + executeFunction( + funcName, + createConstColumn>(5, "DATE"), + createColumn>({"USA", "JIS", "ISO", "EUR", "INTERNAL"}))); + + // time_type: DATETIME + // all locations + ASSERT_COLUMN_EQ( + createColumn>({"%Y-%m-%d %H.%i.%s", "%Y-%m-%d %H:%i:%s", "%Y-%m-%d %H:%i:%s", "%Y-%m-%d %H.%i.%s", "%Y%m%d%H%i%s"}), + executeFunction( + funcName, + createConstColumn>(5, "DATETIME"), + createColumn>({"USA", "JIS", "ISO", "EUR", "INTERNAL"}))); + + // time_type: TIMESTAMP + // all locations + ASSERT_COLUMN_EQ( + createColumn>({"%Y-%m-%d %H.%i.%s", "%Y-%m-%d %H:%i:%s", "%Y-%m-%d %H:%i:%s", "%Y-%m-%d %H.%i.%s", "%Y%m%d%H%i%s"}), + executeFunction( + funcName, + createConstColumn>(5, "TIMESTAMP"), + createColumn>({"USA", "JIS", "ISO", "EUR", "INTERNAL"}))); + + // time_type: TIME + // all locations + ASSERT_COLUMN_EQ( + createColumn>({"%h:%i:%s %p", "%H:%i:%s", "%H:%i:%s", "%H.%i.%s", "%H%i%s"}), + executeFunction( + funcName, + createConstColumn>(5, "TIME"), + createColumn>({"USA", "JIS", "ISO", "EUR", "INTERNAL"}))); + + // the location is not in ("USA", "JIS", "ISO", "EUR", "INTERNAL") + ASSERT_COLUMN_EQ( + createColumn>({"", ""}), + executeFunction( + funcName, + createConstColumn>(2, "TIME"), + createColumn>({"CAN", ""}))); + + // the time_type is not in ("DATE", "DATETIME", "TIMESTAMP", "TIME") + ASSERT_COLUMN_EQ( + createColumn>({"", ""}), + executeFunction( + funcName, + createConstColumn>(2, "TIMEINUTC"), + createColumn>({"USA", "ISO"}))); +} +CATCH + +} // namespace DB::tests diff --git a/dbms/src/Functions/tests/gtest_strings_format.cpp b/dbms/src/Functions/tests/gtest_strings_format.cpp index 2d571a9bb1b..8f3b899316e 100644 --- a/dbms/src/Functions/tests/gtest_strings_format.cpp +++ b/dbms/src/Functions/tests/gtest_strings_format.cpp @@ -34,7 +34,7 @@ class StringFormat : public DB::tests::FunctionTest using FieldType = DecimalField; using NullableDecimal = Nullable; ASSERT_COLUMN_EQ( - createColumn>({"0.0000", "-0.0120", "0.0120", "12,332.1000", "12,332", "12,332", "12,332.300000000000000000000000000000", "-12,332.30000", "-1,000.0", "-333.33", {}}), + createColumn>({"0.0000", "-0.0120", "0.0120", "12,332.1000", "12,332", "12,332", "12,332.300000000000000000000000000000", "-12,332.30000", "-1,000.0", "-333.33", {}, "99,999.9999000000", "100,000.000", "100,000"}), executeFunction( func_name, createColumn( @@ -49,8 +49,11 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-123323000), 4), FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4), - FieldType(static_cast(0), 0)}), - createColumn>({4, 4, 4, 4, 0, -1, 31, 5, 1, 2, {}}))); + FieldType(static_cast(0), 0), + FieldType(static_cast(999999999), 4), + FieldType(static_cast(999999999), 4), + FieldType(static_cast(999999999), 4)}), + createColumn>({4, 4, 4, 4, 0, -1, 31, 5, 1, 2, {}, 10, 3, -5}))); ASSERT_COLUMN_EQ( createColumn>({"12,332.100", "-12,332.300", "-1,000.000", "-333.333"}), executeFunction( @@ -62,8 +65,6 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4)}), createConstColumn>(4, 3))); - /// known issue https://github.com/pingcap/tiflash/issues/4891 - /* ASSERT_COLUMN_EQ( createColumn>({"-999.9999", "-1,000", "-1,000", "-999.999900000000000000000000000000", "-999.99990", "-1,000.0", "-1,000.00"}), executeFunction( @@ -74,7 +75,7 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4)), createColumn>({4, 0, -1, 31, 5, 1, 2}))); ASSERT_COLUMN_EQ( - createConstColumn>(1, "-1,000.000"), + createConstColumn(1, "-1,000.000"), executeFunction( func_name, createConstColumn( @@ -82,7 +83,6 @@ class StringFormat : public DB::tests::FunctionTest 1, FieldType(static_cast(-9999999), 4)), createConstColumn>(1, 3))); - */ ASSERT_COLUMN_EQ( createColumn>({"12,332.1000", "12,332", "12,332.300000000000000000000000000000", "-12,332.30000", "-1,000.0", "-333.33", {}}), executeFunction( @@ -108,8 +108,6 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4)}), createConstColumn>(4, 3))); - /// known issue https://github.com/pingcap/tiflash/issues/4891 - /* ASSERT_COLUMN_EQ( createColumn>({"-999.9999", "-1,000", "-999.999900000000000000000000000000", "-999.99990", "-1,000.0", "-1,000.00"}), executeFunction( @@ -120,7 +118,7 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4)), createColumn>({4, 0, 31, 5, 1, 2}))); ASSERT_COLUMN_EQ( - createConstColumn>(1, "-1,000.000"), + createConstColumn(1, "-1,000.000"), executeFunction( func_name, createConstColumn( @@ -128,7 +126,6 @@ class StringFormat : public DB::tests::FunctionTest 1, FieldType(static_cast(-9999999), 4)), createConstColumn>(1, 3))); - */ } template diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 44699a324f4..7cd0cb5ad53 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -68,9 +68,9 @@ #include #include -#include #include #include +#include namespace ProfileEvents @@ -78,8 +78,6 @@ namespace ProfileEvents extern const Event ContextLock; } -#include - namespace CurrentMetrics { extern const Metric GlobalStorageRunMode; @@ -1440,20 +1438,32 @@ void Context::dropCaches() const shared->mark_cache->reset(); } -BackgroundProcessingPool & Context::getBackgroundPool() +BackgroundProcessingPool & Context::initializeBackgroundPool(UInt16 pool_size) { auto lock = getLock(); if (!shared->background_pool) - shared->background_pool = std::make_shared(settings.background_pool_size); + shared->background_pool = std::make_shared(pool_size); return *shared->background_pool; } -BackgroundProcessingPool & Context::getBlockableBackgroundPool() +BackgroundProcessingPool & Context::getBackgroundPool() +{ + auto lock = getLock(); + return *shared->background_pool; +} + +BackgroundProcessingPool & Context::initializeBlockableBackgroundPool(UInt16 pool_size) { - // TODO: choose a better thread pool size and maybe a better name for the pool auto lock = getLock(); if (!shared->blockable_background_pool) - shared->blockable_background_pool = std::make_shared(settings.background_pool_size); + shared->blockable_background_pool = std::make_shared(pool_size); + return *shared->blockable_background_pool; +} + +BackgroundProcessingPool & Context::getBlockableBackgroundPool() +{ + // TODO: maybe a better name for the pool + auto lock = getLock(); return *shared->blockable_background_pool; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index b6e759e364b..7663b40f612 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -379,7 +379,9 @@ class Context void setUseL0Opt(bool use_l0_opt); bool useL0Opt() const; + BackgroundProcessingPool & initializeBackgroundPool(UInt16 pool_size); BackgroundProcessingPool & getBackgroundPool(); + BackgroundProcessingPool & initializeBlockableBackgroundPool(UInt16 pool_size); BackgroundProcessingPool & getBlockableBackgroundPool(); void createTMTContext(const TiFlashRaftConfig & raft_config, pingcap::ClusterConfig && cluster_config); @@ -505,7 +507,7 @@ class DDLGuard class SessionCleaner { public: - SessionCleaner(Context & context_) + explicit SessionCleaner(Context & context_) : context{context_} {} ~SessionCleaner(); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index fe8f04427a0..3514f915626 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -512,13 +512,13 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt { const auto & join = static_cast(*query.join()->table_join); if (join.kind == ASTTableJoin::Kind::Full || join.kind == ASTTableJoin::Kind::Right) - pipeline.stream_with_non_joined_data = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin( + pipeline.streams_with_non_joined_data.push_back(expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin( pipeline.firstStream()->getHeader(), 0, 1, - settings.max_block_size); + settings.max_block_size)); - for (auto & stream : pipeline.streams) /// Applies to all sources except stream_with_non_joined_data. + for (auto & stream : pipeline.streams) /// Applies to all sources except streams_with_non_joined_data. stream = std::make_shared(stream, expressions.before_join, /*req_id=*/""); } @@ -603,7 +603,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (need_second_distinct_pass || query.limit_length || query.limit_by_expression_list - || pipeline.stream_with_non_joined_data) + || !pipeline.streams_with_non_joined_data.empty()) { need_merge_streams = true; } @@ -987,11 +987,11 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre Aggregator::Params params(header, keys, aggregates, overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath()); /// If there are several sources, then we perform parallel aggregation - if (pipeline.streams.size() > 1) + if (pipeline.streams.size() > 1 || pipeline.streams_with_non_joined_data.size() > 1) { - pipeline.firstStream() = std::make_shared( + auto stream = std::make_shared( pipeline.streams, - pipeline.stream_with_non_joined_data, + pipeline.streams_with_non_joined_data, params, file_provider, final, @@ -1001,19 +1001,21 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre : static_cast(settings.max_threads), /*req_id=*/""); - pipeline.stream_with_non_joined_data = nullptr; pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); } else { BlockInputStreams inputs; if (!pipeline.streams.empty()) inputs.push_back(pipeline.firstStream()); - else - pipeline.streams.resize(1); - if (pipeline.stream_with_non_joined_data) - inputs.push_back(pipeline.stream_with_non_joined_data); + if (!pipeline.streams_with_non_joined_data.empty()) + inputs.push_back(pipeline.streams_with_non_joined_data.at(0)); + + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); pipeline.firstStream() = std::make_shared( std::make_shared(inputs, /*req_id=*/""), @@ -1021,8 +1023,6 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre file_provider, final, /*req_id=*/""); - - pipeline.stream_with_non_joined_data = nullptr; } } @@ -1244,21 +1244,33 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or void InterpreterSelectQuery::executeUnion(Pipeline & pipeline) { - /// If there are still several streams, then we combine them into one - if (pipeline.hasMoreThanOneStream()) + switch (pipeline.streams.size() + pipeline.streams_with_non_joined_data.size()) { - pipeline.firstStream() = std::make_shared>( + case 0: + break; + case 1: + { + if (pipeline.streams.size() == 1) + break; + // streams_with_non_joined_data's size is 1. + pipeline.streams.push_back(pipeline.streams_with_non_joined_data.at(0)); + pipeline.streams_with_non_joined_data.clear(); + break; + } + default: + { + BlockInputStreamPtr stream = std::make_shared>( pipeline.streams, - pipeline.stream_with_non_joined_data, + pipeline.streams_with_non_joined_data, max_streams, /*req_id=*/""); - pipeline.stream_with_non_joined_data = nullptr; + ; + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); + break; } - else if (pipeline.stream_with_non_joined_data) - { - pipeline.streams.push_back(pipeline.stream_with_non_joined_data); - pipeline.stream_with_non_joined_data = nullptr; } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 474ace7ee84..d1bcec2a3dd 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -95,7 +95,7 @@ class InterpreterSelectQuery : public IInterpreter * It has a special meaning, since reading from it should be done after reading from the main streams. * It is appended to the main streams in UnionBlockInputStream or ParallelAggregatingBlockInputStream. */ - BlockInputStreamPtr stream_with_non_joined_data; + BlockInputStreams streams_with_non_joined_data; BlockInputStreamPtr & firstStream() { return streams.at(0); } @@ -105,13 +105,13 @@ class InterpreterSelectQuery : public IInterpreter for (auto & stream : streams) transform(stream); - if (stream_with_non_joined_data) - transform(stream_with_non_joined_data); + for (auto & stream : streams_with_non_joined_data) + transform(stream); } bool hasMoreThanOneStream() const { - return streams.size() + (stream_with_non_joined_data ? 1 : 0) > 1; + return streams.size() + streams_with_non_joined_data.size() > 1; } }; diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 5e73b1e5f3e..076c290cc9d 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -224,7 +224,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute() } else { - result_stream = std::make_shared>(nested_streams, nullptr, settings.max_threads, /*req_id=*/""); + result_stream = std::make_shared>(nested_streams, BlockInputStreams{}, settings.max_threads, /*req_id=*/""); nested_streams.clear(); } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 9361e0525d2..add761c581d 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -209,7 +209,7 @@ struct Settings * Basically, limits are checked for each block (not every row). That is, the limits can be slightly violated. \ * Almost all limits apply only to SELECTs. \ * Almost all limits apply to each stream individually. \ - */ \ + */ \ \ M(SettingUInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it " \ "is only checked on a remote server.") \ @@ -272,7 +272,7 @@ struct Settings M(SettingUInt64, dt_segment_delta_small_column_file_size, 8388608, "Determine whether a column file in delta is small or not. 8MB by default.") \ M(SettingUInt64, dt_segment_stable_pack_rows, DEFAULT_MERGE_BLOCK_SIZE, "Expected stable pack rows in DeltaTree Engine.") \ M(SettingFloat, dt_segment_wait_duration_factor, 1, "The factor of wait duration in a write stall.") \ - M(SettingUInt64, dt_bg_gc_check_interval, 60, "Background gc thread check interval, the unit is second.") \ + M(SettingUInt64, dt_bg_gc_check_interval, 60, "Background gc thread check interval, the unit is second.") \ M(SettingInt64, dt_bg_gc_max_segments_to_check_every_round, 100, "Max segments to check in every gc round, value less than or equal to 0 means gc no segments.") \ M(SettingFloat, dt_bg_gc_ratio_threhold_to_trigger_gc, 1.2, "Trigger segment's gc when the ratio of invalid version exceed this threhold. Values smaller than or equal to 1.0 means gc all " \ "segments") \ @@ -355,15 +355,15 @@ struct Settings M(SettingUInt64, elastic_threadpool_init_cap, 400, "The size of elastic thread pool.") \ M(SettingUInt64, elastic_threadpool_shrink_period_ms, 300000, "The shrink period(ms) of elastic thread pool.") \ M(SettingBool, enable_local_tunnel, true, "Enable local data transfer between local MPP tasks.") \ - M(SettingBool, enable_async_grpc_client, true, "Enable async grpc in MPP.") \ - M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means using hardware_concurrency.")\ + M(SettingBool, enable_async_grpc_client, true, "Enable async grpc in MPP.") \ + M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means using hardware_concurrency.") \ M(SettingBool, enable_async_server, true, "Enable async rpc server.") \ M(SettingUInt64, async_pollers_per_cq, 200, "grpc async pollers per cqs") \ M(SettingUInt64, async_cqs, 1, "grpc async cqs") \ M(SettingUInt64, preallocated_request_count_per_poller, 20, "grpc preallocated_request_count_per_poller") \ \ M(SettingUInt64, manual_compact_pool_size, 1, "The number of worker threads to handle manual compact requests.") \ - M(SettingUInt64, manual_compact_max_concurrency, 10, "Max concurrent tasks. It should be larger than pool size.") \ + M(SettingUInt64, manual_compact_max_concurrency, 10, "Max concurrent tasks. It should be larger than pool size.") \ 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.") // clang-format on diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index a398aa9c74d..aabca11cf9c 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -54,10 +54,15 @@ #include #include #include +#include +#include +#include #include #include #include +#include #include +#include #include #include #include @@ -82,12 +87,6 @@ #include #include -#include "HTTPHandlerFactory.h" -#include "MetricsPrometheus.h" -#include "MetricsTransmitter.h" -#include "StatusFile.h" -#include "TCPHandlerFactory.h" - #if Poco_NetSSL_FOUND #include #include @@ -1144,6 +1143,12 @@ int Server::main(const std::vector & /*args*/) global_context->getPathCapacity(), global_context->getFileProvider()); + /// Initialize the background & blockable background thread pool. + Settings & settings = global_context->getSettingsRef(); + LOG_FMT_INFO(log, "Background & Blockable Background pool size: {}", settings.background_pool_size); + auto & bg_pool = global_context->initializeBackgroundPool(settings.background_pool_size); + auto & blockable_bg_pool = global_context->initializeBlockableBackgroundPool(settings.background_pool_size); + global_context->initializePageStorageMode(global_context->getPathPool(), STORAGE_FORMAT_CURRENT.page); global_context->initializeGlobalStoragePoolIfNeed(global_context->getPathPool()); LOG_FMT_INFO(log, "Global PageStorage run mode is {}", static_cast(global_context->getPageStorageRunMode())); @@ -1260,13 +1265,6 @@ int Server::main(const std::vector & /*args*/) /// Load global settings from default_profile and system_profile. /// It internally depends on UserConfig::parseSettings. global_context->setDefaultProfiles(config()); - Settings & settings = global_context->getSettingsRef(); - - /// Initialize the background thread pool. - /// It internally depends on settings.background_pool_size, - /// so must be called after settings has been load. - auto & bg_pool = global_context->getBackgroundPool(); - auto & blockable_bg_pool = global_context->getBlockableBackgroundPool(); /// Initialize RateLimiter. global_context->initializeRateLimiter(config(), bg_pool, blockable_bg_pool); diff --git a/dbms/src/Storages/BackgroundProcessingPool.h b/dbms/src/Storages/BackgroundProcessingPool.h index 1ba6c4efcf8..49a01b3a397 100644 --- a/dbms/src/Storages/BackgroundProcessingPool.h +++ b/dbms/src/Storages/BackgroundProcessingPool.h @@ -81,7 +81,7 @@ class BackgroundProcessingPool using TaskHandle = std::shared_ptr; - BackgroundProcessingPool(int size_); + explicit BackgroundProcessingPool(int size_); size_t getNumberOfThreads() const { return size; } @@ -96,7 +96,7 @@ class BackgroundProcessingPool /// 2. thread B also get the same task /// 3. thread A finish the execution of the task quickly, release the task and try to update the next schedule time of the task /// 4. thread B find the task is not occupied and execute the task again almost immediately - TaskHandle addTask(const Task & task, const bool multi = true, const size_t interval_ms = 0); + TaskHandle addTask(const Task & task, bool multi = true, size_t interval_ms = 0); void removeTask(const TaskHandle & task); ~BackgroundProcessingPool(); diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index 9fbf3c9691f..1c8b0242bfa 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -120,6 +121,7 @@ ::testing::AssertionResult blockEqual( { const auto & expected_col = expected.getByPosition(i); const auto & actual_col = actual.getByPosition(i); + auto cmp_res = columnEqual(expected_col, actual_col); if (!cmp_res) return cmp_res; @@ -375,9 +377,63 @@ ColumnWithTypeAndName toNullableDatetimeVec(String name, const std::vector>(data_type, vec), data_type, name, 0}; } +String getColumnsContent(const ColumnsWithTypeAndName & cols) +{ + if (cols.size() <= 0) + return ""; + return getColumnsContent(cols, 0, cols[0].column->size() - 1); +} + +String getColumnsContent(const ColumnsWithTypeAndName & cols, size_t begin, size_t end) +{ + const size_t col_num = cols.size(); + if (col_num <= 0) + return ""; + + const size_t col_size = cols[0].column->size(); + assert(begin <= end); + assert(col_size > end); + assert(col_size > begin); + + bool is_same = true; + + for (size_t i = 1; i < col_num; ++i) + { + if (cols[i].column->size() != col_size) + is_same = false; + } + + assert(is_same); /// Ensure the sizes of columns in cols are the same + + std::vector> col_content; + FmtBuffer fmt_buf; + for (size_t i = 0; i < col_num; ++i) + { + /// Push the column name + fmt_buf.append(fmt::format("{}: (", cols[i].name)); + for (size_t j = begin; j <= end; ++j) + col_content.push_back(std::make_pair(j, (*cols[i].column)[j].toString())); + + /// Add content + fmt_buf.joinStr( + col_content.begin(), + col_content.end(), + [](const auto & content, FmtBuffer & fmt_buf) { + fmt_buf.append(fmt::format("{}: {}", content.first, content.second)); + }, + ", "); + + fmt_buf.append(")\n"); + col_content.clear(); + } + + return fmt_buf.toString(); +} + ColumnsWithTypeAndName createColumns(const ColumnsWithTypeAndName & cols) { return cols; } + } // namespace tests } // namespace DB diff --git a/dbms/src/TestUtils/FunctionTestUtils.h b/dbms/src/TestUtils/FunctionTestUtils.h index ad01e2e8441..8680d1886b1 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.h +++ b/dbms/src/TestUtils/FunctionTestUtils.h @@ -514,13 +514,17 @@ ColumnWithTypeAndName createConstColumn( return createConstColumn(data_type_args, size, InferredFieldType(std::nullopt), name); } +String getColumnsContent(const ColumnsWithTypeAndName & cols); + +/// We can designate the range of columns printed with begin and end. range: [begin, end] +String getColumnsContent(const ColumnsWithTypeAndName & cols, size_t begin, size_t end); + // This wrapper function only serves to construct columns input for function-like macros, // since preprocessor recognizes `{col1, col2, col3}` as three arguments instead of one. // E.g. preprocessor does not allow us to write `ASSERT_COLUMNS_EQ_R({col1, col2, col3}, actual_cols)`, // but with this func we can write `ASSERT_COLUMNS_EQ_R(createColumns{col1, col2, col3}, actual_cols)` instead. ColumnsWithTypeAndName createColumns(const ColumnsWithTypeAndName & cols); - ::testing::AssertionResult dataTypeEqual( const DataTypePtr & expected, const DataTypePtr & actual); diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index cbd42b57550..f44298cbafd 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -39,6 +39,11 @@ void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path, PageStorageR KeyManagerPtr key_manager = std::make_shared(false); global_context->initializeFileProvider(key_manager, false); + // initialize background & blockable background thread pool + Settings & settings = global_context->getSettingsRef(); + global_context->initializeBackgroundPool(settings.background_pool_size); + global_context->initializeBlockableBackgroundPool(settings.background_pool_size); + // Theses global variables should be initialized by the following order // 1. capacity // 2. path pool diff --git a/dbms/src/TestUtils/bench_dbms_main.cpp b/dbms/src/TestUtils/bench_dbms_main.cpp index 48bd02a71f7..092c45c35e2 100644 --- a/dbms/src/TestUtils/bench_dbms_main.cpp +++ b/dbms/src/TestUtils/bench_dbms_main.cpp @@ -20,6 +20,8 @@ int main(int argc, char * argv[]) { benchmark::Initialize(&argc, argv); DB::tests::TiFlashTestEnv::setupLogger(); + // Each time TiFlashTestEnv::getContext() is called, some log will print, it's annoying. + Poco::Logger::root().setLevel("error"); DB::tests::TiFlashTestEnv::initializeGlobalContext(); if (::benchmark::ReportUnrecognizedArguments(argc, argv)) return 1; diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 49ae9787ea4..30d05786c9a 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -119,12 +119,12 @@ DAGRequestBuilder & DAGRequestBuilder::mockTable(const MockTableName & name, con return mockTable(name.first, name.second, columns); } -DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfoVec & columns) +DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count) { - return buildExchangeReceiver(columns); + return buildExchangeReceiver(columns, fine_grained_shuffle_stream_count); } -DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInfoVec & columns) +DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count) { DAGSchema schema; for (const auto & column : columns) @@ -135,7 +135,7 @@ DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInf schema.push_back({column.first, info}); } - root = compileExchangeReceiver(getExecutorIndex(), schema); + root = compileExchangeReceiver(getExecutorIndex(), schema, fine_grained_shuffle_stream_count); return *this; } @@ -266,45 +266,45 @@ DAGRequestBuilder & DAGRequestBuilder::buildAggregation(ASTPtr agg_funcs, ASTPtr return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) { assert(root); auto window_func_list = std::make_shared(); window_func_list->children.push_back(window_func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec({partition_by}), buildOrderByItemVec({order_by}), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec({partition_by}), buildOrderByItemVec({order_by}), frame, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) { assert(root); auto window_func_list = std::make_shared(); window_func_list->children.push_back(window_func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) { assert(root); auto window_func_list = std::make_shared(); for (const auto & func : window_funcs) window_func_list->children.push_back(func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItem order_by, bool is_partial_sort) +DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItem order_by, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count) { assert(root); - root = compileSort(root, getExecutorIndex(), buildOrderByItemVec({order_by}), is_partial_sort); + root = compileSort(root, getExecutorIndex(), buildOrderByItemVec({order_by}), is_partial_sort, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItemVec order_by_vec, bool is_partial_sort) +DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItemVec order_by_vec, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count) { assert(root); - root = compileSort(root, getExecutorIndex(), buildOrderByItemVec(order_by_vec), is_partial_sort); + root = compileSort(root, getExecutorIndex(), buildOrderByItemVec(order_by_vec), is_partial_sort, fine_grained_shuffle_stream_count); return *this; } @@ -368,9 +368,9 @@ DAGRequestBuilder MockDAGRequestContext::scan(String db_name, String table_name) return builder; } -DAGRequestBuilder MockDAGRequestContext::receive(String exchange_name) +DAGRequestBuilder MockDAGRequestContext::receive(String exchange_name, uint64_t fine_grained_shuffle_stream_count) { - auto builder = DAGRequestBuilder(index).exchangeReceiver(exchange_schemas[exchange_name]); + auto builder = DAGRequestBuilder(index).exchangeReceiver(exchange_schemas[exchange_name], fine_grained_shuffle_stream_count); receiver_source_task_ids_map[builder.getRoot()->name] = {}; // If don't have related columns, user must pass input columns as argument of executeStreams in order to run Executors Tests. // If user don't want to test executors, it will be safe to run Interpreter Tests. @@ -380,5 +380,4 @@ DAGRequestBuilder MockDAGRequestContext::receive(String exchange_name) } return builder; } - } // namespace DB::tests diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 971061e93e7..8b5a6d300ff 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -66,7 +66,7 @@ class DAGRequestBuilder DAGRequestBuilder & mockTable(const String & db, const String & table, const MockColumnInfoVec & columns); DAGRequestBuilder & mockTable(const MockTableName & name, const MockColumnInfoVec & columns); - DAGRequestBuilder & exchangeReceiver(const MockColumnInfoVec & columns); + DAGRequestBuilder & exchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count = 0); DAGRequestBuilder & filter(ASTPtr filter_expr); @@ -83,7 +83,7 @@ class DAGRequestBuilder DAGRequestBuilder & exchangeSender(tipb::ExchangeType exchange_type); - // Currentlt only support inner join, left join and right join. + // Currently only support inner join, left join and right join. // TODO support more types of join. DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAstVec exprs); DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAstVec exprs, ASTTableJoin::Kind kind); @@ -93,16 +93,16 @@ class DAGRequestBuilder DAGRequestBuilder & aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs); // window - DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame); - DAGRequestBuilder & window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame); - DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame); - DAGRequestBuilder & sort(MockOrderByItem order_by, bool is_partial_sort); - DAGRequestBuilder & sort(MockOrderByItemVec order_by_vec, bool is_partial_sort); + DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & sort(MockOrderByItem order_by, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & sort(MockOrderByItemVec order_by_vec, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count = 0); private: void initDAGRequest(tipb::DAGRequest & dag_request); DAGRequestBuilder & buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs); - DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfoVec & columns); + DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count = 0); ExecutorPtr root; DAGProperties properties; @@ -139,7 +139,7 @@ class MockDAGRequestContext std::unordered_map & executorIdColumnsMap() { return executor_id_columns_map; } DAGRequestBuilder scan(String db_name, String table_name); - DAGRequestBuilder receive(String exchange_name); + DAGRequestBuilder receive(String exchange_name, uint64_t fine_grained_shuffle_stream_count = 0); private: size_t index; @@ -166,6 +166,7 @@ MockWindowFrame buildDefaultRowsFrame(); #define col(name) buildColumn((name)) #define lit(field) buildLiteral((field)) +#define concat(expr1, expr2) makeASTFunction("concat", (expr1), (expr2)) #define eq(expr1, expr2) makeASTFunction("equals", (expr1), (expr2)) #define Not_eq(expr1, expr2) makeASTFunction("notEquals", (expr1), (expr2)) #define lt(expr1, expr2) makeASTFunction("less", (expr1), (expr2)) @@ -174,6 +175,7 @@ MockWindowFrame buildDefaultRowsFrame(); #define Or(expr1, expr2) makeASTFunction("or", (expr1), (expr2)) #define NOT(expr) makeASTFunction("not", (expr)) #define Max(expr) makeASTFunction("max", (expr)) +#define Sum(expr) makeASTFunction("sum", (expr)) /// Window functions #define RowNumber() makeASTFunction("RowNumber") #define Rank() makeASTFunction("Rank") diff --git a/dbms/src/TestUtils/tests/gtest_print_columns.cpp b/dbms/src/TestUtils/tests/gtest_print_columns.cpp new file mode 100644 index 00000000000..50631fc4f4a --- /dev/null +++ b/dbms/src/TestUtils/tests/gtest_print_columns.cpp @@ -0,0 +1,57 @@ +// 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. + +#include +#include + +namespace DB +{ +namespace tests +{ + +class PrintColumnsTest : public DB::tests::ExecutorTest +{ +public: + using ColStringType = std::optional::FieldType>; + using ColInt32Type = std::optional::FieldType>; + using ColumnWithString = std::vector; + using ColumnWithInt32 = std::vector; + + void initializeContext() override + { + test_cols.push_back(toNullableVec("col1", ColumnWithInt32{36, 34, 32, 27, {}, {}})); + test_cols.push_back(toNullableVec("col2", ColumnWithString{"female", "male", "male", "female", "male", "female"})); + col_len = test_cols[0].column->size(); + } + + ColumnsWithTypeAndName test_cols; + size_t col_len; + const String result1{"col1: (0: Int64_36, 1: Int64_34, 2: Int64_32, 3: Int64_27, 4: NULL, 5: NULL)\ncol2: (0: 'female', 1: 'male', 2: 'male', 3: 'female', 4: 'male', 5: 'female')\n"}; + const String result2{"col1: (0: Int64_36, 1: Int64_34, 2: Int64_32, 3: Int64_27, 4: NULL, 5: NULL)\ncol2: (0: 'female', 1: 'male', 2: 'male', 3: 'female', 4: 'male', 5: 'female')\n"}; + const String result3{"col1: (0: Int64_36)\ncol2: (0: 'female')\n"}; + const String result4{"col1: (1: Int64_34, 2: Int64_32, 3: Int64_27, 4: NULL)\ncol2: (1: 'male', 2: 'male', 3: 'female', 4: 'male')\n"}; +}; + +TEST_F(PrintColumnsTest, SimpleTest) +try +{ + EXPECT_EQ(getColumnsContent(test_cols), result1); + EXPECT_EQ(getColumnsContent(test_cols, 0, col_len - 1), result2); + EXPECT_EQ(getColumnsContent(test_cols, 0, 0), result3); + EXPECT_EQ(getColumnsContent(test_cols, 1, col_len - 2), result4); +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/release-centos7-llvm/Makefile b/release-centos7-llvm/Makefile index 1b15df7ddc3..9c1bba42a53 100644 --- a/release-centos7-llvm/Makefile +++ b/release-centos7-llvm/Makefile @@ -23,6 +23,10 @@ image_tiflash_llvm_base_aarch64: build_tiflash_release_amd64: docker run --rm -v $(realpath ..):/build/tics hub.pingcap.net/tiflash/tiflash-llvm-base:amd64 /build/tics/release-centos7-llvm/scripts/build-release.sh +# Add build_tiflash_debug_amd64 target to enable FailPoints on x86. Since outputs are the same as release version, no new package targets added. +build_tiflash_debug_amd64: + docker run --rm -v $(realpath ..):/build/tics hub.pingcap.net/tiflash/tiflash-llvm-base:amd64 /build/tics/release-centos7-llvm/scripts/build-debug.sh + build_tiflash_ci_amd64: docker run --rm -v $(realpath ..):/build/tics hub.pingcap.net/tiflash/tiflash-llvm-base:amd64 /build/tics/release-centos7-llvm/scripts/build-tiflash-ci.sh diff --git a/release-centos7-llvm/scripts/build-debug.sh b/release-centos7-llvm/scripts/build-debug.sh new file mode 100755 index 00000000000..59dc9b86a54 --- /dev/null +++ b/release-centos7-llvm/scripts/build-debug.sh @@ -0,0 +1,23 @@ +#!/bin/bash +# 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. + + +CMAKE_PREFIX_PATH=$1 + +set -ueox pipefail + +SCRIPTPATH="$( cd "$(dirname "$0")" ; pwd -P )" + +${SCRIPTPATH}/build-tiflash-release.sh "DEBUG" "${CMAKE_PREFIX_PATH}" diff --git a/release-centos7-llvm/scripts/build-tiflash-release.sh b/release-centos7-llvm/scripts/build-tiflash-release.sh index 42993b51afe..01ca00e8706 100755 --- a/release-centos7-llvm/scripts/build-tiflash-release.sh +++ b/release-centos7-llvm/scripts/build-tiflash-release.sh @@ -47,7 +47,13 @@ ENABLE_PCH=${ENABLE_PCH:-ON} INSTALL_DIR="${SRCPATH}/release-centos7-llvm/tiflash" rm -rf ${INSTALL_DIR} && mkdir -p ${INSTALL_DIR} -BUILD_DIR="${SRCPATH}/release-centos7-llvm/build-release" +if [ $CMAKE_BUILD_TYPE == "RELWITHDEBINFO" ]; then + BUILD_DIR="$SRCPATH/release-centos7-llvm/build-release" + ENABLE_FAILPOINTS="OFF" +else + BUILD_DIR="$SRCPATH/release-centos7-llvm/build-debug" + ENABLE_FAILPOINTS="ON" +fi rm -rf ${BUILD_DIR} && mkdir -p ${BUILD_DIR} && cd ${BUILD_DIR} cmake -S "${SRCPATH}" \ @@ -55,6 +61,7 @@ cmake -S "${SRCPATH}" \ -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE} \ -DENABLE_TESTING=OFF \ -DENABLE_TESTS=OFF \ + -DENABLE_FAILPOINTS=${ENABLE_FAILPOINTS} \ -Wno-dev \ -DUSE_CCACHE=OFF \ -DRUN_HAVE_STD_REGEX=0 \ diff --git a/tests/fullstack-test/expr/format.test b/tests/fullstack-test/expr/format.test index 8cea75d6914..719e30c974d 100644 --- a/tests/fullstack-test/expr/format.test +++ b/tests/fullstack-test/expr/format.test @@ -44,3 +44,52 @@ int_val 1,234.000 mysql> drop table if exists test.t + +mysql> create table test.t(id int, value decimal(65,4)) +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values(1,9999999999999999999999999999999999999999999999999999999999999.9999) + +func> wait_table test t + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,-3) as result from test.t +result +10,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,0) as result from test.t +result +10,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,3) as result from test.t +result +10,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000.000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,10) as result from test.t +result +9,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999.9999000000 + + +mysql> drop table if exists test.t + +mysql> create table test.t(id int, value decimal(7,4)) +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values(1,999.9999) + +func> wait_table test t + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,-2) as result from test.t +result +1,000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,0) as result from test.t +result +1,000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,2) as result from test.t +result +1,000.00 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,10) as result from test.t +result +999.9999000000 + +mysql> drop table if exists test.t diff --git a/tests/fullstack-test/expr/get_format.test b/tests/fullstack-test/expr/get_format.test new file mode 100644 index 00000000000..5409302c10a --- /dev/null +++ b/tests/fullstack-test/expr/get_format.test @@ -0,0 +1,60 @@ +# 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. + +mysql> drop table if exists test.t; +mysql> create table test.t(location varchar(10)); +mysql> insert into test.t values('USA'), ('JIS'), ('ISO'), ('EUR'), ('INTERNAL'); +mysql> alter table test.t set tiflash replica 1; +func> wait_table test t +mysql> set @@tidb_enforce_mpp=1; set @@tidb_isolation_read_engines='tiflash'; select GET_FORMAT(DATE, location) from test.t; ++----------------------------+ +| GET_FORMAT(DATE, location) | ++----------------------------+ +| %m.%d.%Y | +| %Y-%m-%d | +| %Y-%m-%d | +| %d.%m.%Y | +| %Y%m%d | ++----------------------------+ +mysql> set @@tidb_enforce_mpp=1; set @@tidb_isolation_read_engines='tiflash'; select GET_FORMAT(DATETIME, location) from test.t; ++--------------------------------+ +| GET_FORMAT(DATETIME, location) | ++--------------------------------+ +| %Y-%m-%d %H.%i.%s | +| %Y-%m-%d %H:%i:%s | +| %Y-%m-%d %H:%i:%s | +| %Y-%m-%d %H.%i.%s | +| %Y%m%d%H%i%s | ++--------------------------------+ +mysql> set @@tidb_enforce_mpp=1; set @@tidb_isolation_read_engines='tiflash'; select GET_FORMAT(TIMESTAMP, location) from test.t; ++---------------------------------+ +| GET_FORMAT(TIMESTAMP, location) | ++---------------------------------+ +| %Y-%m-%d %H.%i.%s | +| %Y-%m-%d %H:%i:%s | +| %Y-%m-%d %H:%i:%s | +| %Y-%m-%d %H.%i.%s | +| %Y%m%d%H%i%s | ++---------------------------------+ +mysql> set @@tidb_enforce_mpp=1; set @@tidb_isolation_read_engines='tiflash'; select GET_FORMAT(TIME, location) from test.t; ++----------------------------+ +| GET_FORMAT(TIME, location) | ++----------------------------+ +| %h:%i:%s %p | +| %H:%i:%s | +| %H:%i:%s | +| %H.%i.%s | +| %H%i%s | ++----------------------------+ +mysql> drop table if exists test.t; diff --git a/tests/fullstack-test/mpp/window.test b/tests/fullstack-test/mpp/window.test new file mode 100644 index 00000000000..698d39ef2ea --- /dev/null +++ b/tests/fullstack-test/mpp/window.test @@ -0,0 +1,32 @@ +# 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. + +mysql> drop table if exists test.t1; +mysql> create table test.t1(c1 int, c2 int); +mysql> insert into test.t1 values(1, 1),(2, 2),(3, 3),(1, 1),(2, 2),(3, 3),(4, 4); +mysql> alter table test.t1 set tiflash replica 1; +func> wait_table test t1 +mysql> use test; set @@tidb_isolation_read_engines='tiflash'; select c1, c2, row_number() over w2, row_number() over w1 from test.t1 window w1 as(partition by c1), w2 as (partition by c1, c2) order by 1, 2, 3, 4; ++------+------+----------------------+----------------------+ +| c1 | c2 | row_number() over w2 | row_number() over w1 | ++------+------+----------------------+----------------------+ +| 1 | 1 | 1 | 1 | +| 1 | 1 | 2 | 2 | +| 2 | 2 | 1 | 1 | +| 2 | 2 | 2 | 2 | +| 3 | 3 | 1 | 1 | +| 3 | 3 | 2 | 2 | +| 4 | 4 | 1 | 1 | ++------+------+----------------------+----------------------+ +mysql> drop table if exists test.t1;