From f3832a18f8fc5c3a07007355bf10a256ae59db71 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 2 Dec 2022 17:22:01 +0800 Subject: [PATCH 01/42] Revert "Use `num_streams` as parallelism when read thread is enabled." (#6408) close pingcap/tiflash#6409 --- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 44 +++++++++---------- 1 file changed, 20 insertions(+), 24 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 79d3ccb1d63..809878b7a30 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -902,11 +902,10 @@ BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, auto after_segment_read = [&](const DMContextPtr & dm_context_, const SegmentPtr & segment_) { this->checkSegmentUpdate(dm_context_, segment_, ThreadType::Read); }; + size_t final_num_stream = std::min(num_streams, tasks.size()); String req_info; if (db_context.getDAGContext() != nullptr && db_context.getDAGContext()->isMPPTask()) req_info = db_context.getDAGContext()->getMPPTaskId().toString(); - // We can use num_streams as parallelism when read thread is enabled. - size_t final_num_stream = enable_read_thread ? num_streams : std::min(num_streams, tasks.size()); auto read_task_pool = std::make_shared( physical_table_id, dm_context, @@ -921,23 +920,21 @@ BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, enable_read_thread); BlockInputStreams res; - if (enable_read_thread) + for (size_t i = 0; i < final_num_stream; ++i) { - for (size_t i = 0; i < final_num_stream; ++i) + BlockInputStreamPtr stream; + if (enable_read_thread) { - res.emplace_back(std::make_shared( + stream = std::make_shared( read_task_pool, columns_to_read, extra_table_id_index, physical_table_id, - req_info)); + req_info); } - } - else - { - for (size_t i = 0; i < final_num_stream; ++i) + else { - res.emplace_back(std::make_shared( + stream = std::make_shared( dm_context, read_task_pool, after_segment_read, @@ -948,8 +945,9 @@ BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, /* read_mode */ ReadMode::Raw, extra_table_id_index, physical_table_id, - req_info)); + req_info); } + res.push_back(stream); } return res; } @@ -992,8 +990,7 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, }; GET_METRIC(tiflash_storage_read_tasks_count).Increment(tasks.size()); - // We can use num_streams as parallelism when read thread is enabled. - size_t final_num_stream = enable_read_thread ? std::max(1, num_streams) : std::max(1, std::min(num_streams, tasks.size())); + size_t final_num_stream = std::max(1, std::min(num_streams, tasks.size())); auto read_task_pool = std::make_shared( physical_table_id, dm_context, @@ -1008,23 +1005,21 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, enable_read_thread); BlockInputStreams res; - if (enable_read_thread) + for (size_t i = 0; i < final_num_stream; ++i) { - for (size_t i = 0; i < final_num_stream; ++i) + BlockInputStreamPtr stream; + if (enable_read_thread) { - res.emplace_back(std::make_shared( + stream = std::make_shared( read_task_pool, columns_to_read, extra_table_id_index, physical_table_id, - log_tracing_id)); + log_tracing_id); } - } - else - { - for (size_t i = 0; i < final_num_stream; ++i) + else { - res.emplace_back(std::make_shared( + stream = std::make_shared( dm_context, read_task_pool, after_segment_read, @@ -1035,8 +1030,9 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, /* read_mode = */ is_fast_scan ? ReadMode::Fast : ReadMode::Normal, extra_table_id_index, physical_table_id, - log_tracing_id)); + log_tracing_id); } + res.push_back(stream); } LOG_DEBUG(tracing_logger, "Read create stream done"); From e255112ddff31a8719e996122abe4e05d5939b01 Mon Sep 17 00:00:00 2001 From: Meng Xin Date: Mon, 5 Dec 2022 12:06:02 +0800 Subject: [PATCH 02/42] make join as top level operator (#6356) close pingcap/tiflash#6351 --- .../HashJoinProbeBlockInputStream.cpp | 31 ++++++++++++------- .../HashJoinProbeBlockInputStream.h | 6 ++-- .../Coprocessor/DAGExpressionAnalyzer.cpp | 10 ------ .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 5 --- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 5 +-- .../Coprocessor/JoinInterpreterHelper.cpp | 18 ----------- .../Flash/Coprocessor/JoinInterpreterHelper.h | 7 ----- dbms/src/Flash/Planner/plans/PhysicalJoin.cpp | 6 +--- dbms/src/Flash/Planner/plans/PhysicalJoin.h | 4 --- 9 files changed, 25 insertions(+), 67 deletions(-) diff --git a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp index 26b62519b6c..a5a293f3e43 100644 --- a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp +++ b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp @@ -19,18 +19,14 @@ namespace DB { HashJoinProbeBlockInputStream::HashJoinProbeBlockInputStream( const BlockInputStreamPtr & input, - const ExpressionActionsPtr & join_probe_actions_, + const JoinPtr & join_, const String & req_id) : log(Logger::get(req_id)) - , join_probe_actions(join_probe_actions_) + , join(join_) { children.push_back(input); - if (!join_probe_actions || join_probe_actions->getActions().size() != 1 - || join_probe_actions->getActions().back().type != ExpressionAction::Type::JOIN) - { - throw Exception("isn't valid join probe actions", ErrorCodes::LOGICAL_ERROR); - } + RUNTIME_CHECK_MSG(join != nullptr, "join ptr should not be null."); } Block HashJoinProbeBlockInputStream::getTotals() @@ -38,16 +34,29 @@ Block HashJoinProbeBlockInputStream::getTotals() if (auto * child = dynamic_cast(&*children.back())) { totals = child->getTotals(); - join_probe_actions->executeOnTotals(totals); + if (!totals) + { + if (join->hasTotals()) + { + for (const auto & name_and_type : child->getHeader().getColumnsWithTypeAndName()) + { + auto column = name_and_type.type->createColumn(); + column->insertDefault(); + totals.insert(ColumnWithTypeAndName(std::move(column), name_and_type.type, name_and_type.name)); + } + } + else + return totals; /// There's nothing to JOIN. + } + join->joinTotals(totals); } - return totals; } Block HashJoinProbeBlockInputStream::getHeader() const { Block res = children.back()->getHeader(); - join_probe_actions->execute(res); + join->joinBlock(res); return res; } @@ -57,7 +66,7 @@ Block HashJoinProbeBlockInputStream::readImpl() if (!res) return res; - join_probe_actions->execute(res); + join->joinBlock(res); // TODO split block if block.size() > settings.max_block_size // https://github.com/pingcap/tiflash/issues/3436 diff --git a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h index 1d45768ff34..df68d041994 100644 --- a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h +++ b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h @@ -15,6 +15,7 @@ #pragma once #include +#include namespace DB { @@ -31,13 +32,12 @@ class ExpressionActions; class HashJoinProbeBlockInputStream : public IProfilingBlockInputStream { private: - using ExpressionActionsPtr = std::shared_ptr; static constexpr auto name = "HashJoinProbe"; public: HashJoinProbeBlockInputStream( const BlockInputStreamPtr & input, - const ExpressionActionsPtr & join_probe_actions_, + const JoinPtr & join_, const String & req_id); String getName() const override { return name; } @@ -49,7 +49,7 @@ class HashJoinProbeBlockInputStream : public IProfilingBlockInputStream private: const LoggerPtr log; - ExpressionActionsPtr join_probe_actions; + JoinPtr join; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 98eb74ec53b..48774f0c457 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -910,16 +910,6 @@ String DAGExpressionAnalyzer::appendDurationCast( return applyFunction(func_name, {dur_expr, fsp_expr}, actions, nullptr); } -void DAGExpressionAnalyzer::appendJoin( - ExpressionActionsChain & chain, - SubqueryForSet & join_query, - const NamesAndTypesList & columns_added_by_join) const -{ - initChain(chain, getCurrentInputColumns()); - ExpressionActionsPtr actions = chain.getLastActions(); - actions->add(ExpressionAction::ordinaryJoin(join_query.join, columns_added_by_join)); -} - std::pair DAGExpressionAnalyzer::buildJoinKey( const ExpressionActionsPtr & actions, const google::protobuf::RepeatedPtrField & keys, diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 11155e88d88..f0d64b2232e 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -102,11 +102,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable ExpressionActionsChain::Step & initAndGetLastStep(ExpressionActionsChain & chain) const; - void appendJoin( - ExpressionActionsChain & chain, - SubqueryForSet & join_query, - const NamesAndTypesList & columns_added_by_join) const; - // Generate a project action for non-root DAGQueryBlock, // to keep the schema of Block and tidb-schema the same, and // guarantee that left/right block of join don't have duplicated column names. diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index a7d2aae92b7..7af8a63e978 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -212,7 +212,6 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & const Block & right_input_header = input_streams_vec[1].back()->getHeader(); String match_helper_name = tiflash_join.genMatchHelperName(left_input_header, right_input_header); - NamesAndTypesList columns_added_by_join = tiflash_join.genColumnsAddedByJoin(build_pipeline.firstStream()->getHeader(), match_helper_name); NamesAndTypes join_output_columns = tiflash_join.genJoinOutputColumns(left_input_header, right_input_header, match_helper_name); /// add necessary transformation if the join key is an expression @@ -302,8 +301,6 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & for (const auto & p : probe_pipeline.firstStream()->getHeader()) source_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(std::move(source_columns), context); - ExpressionActionsChain chain; - dag_analyzer.appendJoin(chain, right_query, columns_added_by_join); pipeline.streams = probe_pipeline.streams; /// add join input stream if (is_tiflash_right_join) @@ -323,7 +320,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & } for (auto & stream : pipeline.streams) { - stream = std::make_shared(stream, chain.getLastActions(), log->identifier()); + stream = std::make_shared(stream, join_ptr, log->identifier()); stream->setExtraInfo(fmt::format("join probe, join_executor_id = {}", query_block.source_name)); } diff --git a/dbms/src/Flash/Coprocessor/JoinInterpreterHelper.cpp b/dbms/src/Flash/Coprocessor/JoinInterpreterHelper.cpp index 63cee5859ff..275042fddb0 100644 --- a/dbms/src/Flash/Coprocessor/JoinInterpreterHelper.cpp +++ b/dbms/src/Flash/Coprocessor/JoinInterpreterHelper.cpp @@ -300,24 +300,6 @@ NamesAndTypes TiFlashJoin::genColumnsForOtherJoinFilter( return columns_for_other_join_filter; } -/// all the columns from build side streams should be added after join, even for the join key. -NamesAndTypesList TiFlashJoin::genColumnsAddedByJoin( - const Block & build_side_header, - const String & match_helper_name) const -{ - NamesAndTypesList columns_added_by_join; - bool make_nullable = isTiFlashLeftJoin(); - for (auto const & p : build_side_header) - { - columns_added_by_join.emplace_back(p.name, make_nullable ? makeNullable(p.type) : p.type); - } - if (!match_helper_name.empty()) - { - columns_added_by_join.emplace_back(match_helper_name, Join::match_helper_type); - } - return columns_added_by_join; -} - NamesAndTypes TiFlashJoin::genJoinOutputColumns( const Block & left_input_header, const Block & right_input_header, diff --git a/dbms/src/Flash/Coprocessor/JoinInterpreterHelper.h b/dbms/src/Flash/Coprocessor/JoinInterpreterHelper.h index 8f669db9c94..b9b107a82b0 100644 --- a/dbms/src/Flash/Coprocessor/JoinInterpreterHelper.h +++ b/dbms/src/Flash/Coprocessor/JoinInterpreterHelper.h @@ -86,13 +86,6 @@ struct TiFlashJoin /// return "" for everything else. String genMatchHelperName(const Block & header1, const Block & header2) const; - /// columns_added_by_join - /// = join_output_columns - probe_side_columns - /// = build_side_columns + match_helper_name - NamesAndTypesList genColumnsAddedByJoin( - const Block & build_side_header, - const String & match_helper_name) const; - /// The columns output by join will be: /// {columns of left_input, columns of right_input, match_helper_name} NamesAndTypes genJoinOutputColumns( diff --git a/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp b/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp index e8be141eff1..91d4e489c65 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp @@ -106,7 +106,6 @@ PhysicalPlanNodePtr PhysicalJoin::build( const Block & build_side_header = build_plan->getSampleBlock(); String match_helper_name = tiflash_join.genMatchHelperName(left_input_header, right_input_header); - NamesAndTypesList columns_added_by_join = tiflash_join.genColumnsAddedByJoin(build_side_header, match_helper_name); NamesAndTypes join_output_schema = tiflash_join.genJoinOutputColumns(left_input_header, right_input_header, match_helper_name); auto & dag_context = *context.getDAGContext(); @@ -171,7 +170,6 @@ PhysicalPlanNodePtr PhysicalJoin::build( probe_plan, build_plan, join_ptr, - columns_added_by_join, probe_side_prepare_actions, build_side_prepare_actions, is_tiflash_right_join, @@ -191,8 +189,6 @@ void PhysicalJoin::probeSideTransform(DAGPipeline & probe_pipeline, Context & co /// probe side streams assert(probe_pipeline.streams_with_non_joined_data.empty()); executeExpression(probe_pipeline, probe_side_prepare_actions, log, "append join key and join filters for probe side"); - auto join_probe_actions = PhysicalPlanHelper::newActions(probe_pipeline.firstStream()->getHeader(), context); - join_probe_actions->add(ExpressionAction::ordinaryJoin(join_ptr, columns_added_by_join)); /// add join input stream if (has_non_joined) { @@ -210,7 +206,7 @@ void PhysicalJoin::probeSideTransform(DAGPipeline & probe_pipeline, Context & co String join_probe_extra_info = fmt::format("join probe, join_executor_id = {}", execId()); for (auto & stream : probe_pipeline.streams) { - stream = std::make_shared(stream, join_probe_actions, log->identifier()); + stream = std::make_shared(stream, join_ptr, log->identifier()); stream->setExtraInfo(join_probe_extra_info); } } diff --git a/dbms/src/Flash/Planner/plans/PhysicalJoin.h b/dbms/src/Flash/Planner/plans/PhysicalJoin.h index 8af849b6492..f5225cf87ab 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalJoin.h +++ b/dbms/src/Flash/Planner/plans/PhysicalJoin.h @@ -40,7 +40,6 @@ class PhysicalJoin : public PhysicalBinary const PhysicalPlanNodePtr & probe_, const PhysicalPlanNodePtr & build_, const JoinPtr & join_ptr_, - const NamesAndTypesList & columns_added_by_join_, const ExpressionActionsPtr & probe_side_prepare_actions_, const ExpressionActionsPtr & build_side_prepare_actions_, bool has_non_joined_, @@ -48,7 +47,6 @@ class PhysicalJoin : public PhysicalBinary const FineGrainedShuffle & fine_grained_shuffle_) : PhysicalBinary(executor_id_, PlanType::Join, schema_, req_id, probe_, build_) , join_ptr(join_ptr_) - , columns_added_by_join(columns_added_by_join_) , probe_side_prepare_actions(probe_side_prepare_actions_) , build_side_prepare_actions(build_side_prepare_actions_) , has_non_joined(has_non_joined_) @@ -76,8 +74,6 @@ class PhysicalJoin : public PhysicalBinary private: JoinPtr join_ptr; - NamesAndTypesList columns_added_by_join; - ExpressionActionsPtr probe_side_prepare_actions; ExpressionActionsPtr build_side_prepare_actions; From 2892ca889fddf3150dc0b928bb5ff67d97af9613 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Mon, 5 Dec 2022 17:00:04 +0800 Subject: [PATCH 03/42] update proxy of master to raftstore-proxy (#6423) ref pingcap/tiflash#4982 --- contrib/tiflash-proxy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index ea48821d77b..314bab7a5a8 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit ea48821d77b57a276ce3a1363de8875c07d96756 +Subproject commit 314bab7a5a8736c21d1a2cdf5ef66082a8dfcbd9 From e118b17882e087ef00c00bea4df2861dd3582751 Mon Sep 17 00:00:00 2001 From: Wenxuan Date: Mon, 5 Dec 2022 17:40:04 +0800 Subject: [PATCH 04/42] storage: Allow segment to be merged close to base size (#6416) close pingcap/tiflash#6414 --- dbms/src/Storages/DeltaMerge/DMContext.h | 6 ------ dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp | 6 +++--- .../Storages/DeltaMerge/tests/gtest_dm_store_background.cpp | 6 +++--- 3 files changed, 6 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DMContext.h b/dbms/src/Storages/DeltaMerge/DMContext.h index ca8757bbb73..371fc9ab5d0 100644 --- a/dbms/src/Storages/DeltaMerge/DMContext.h +++ b/dbms/src/Storages/DeltaMerge/DMContext.h @@ -75,10 +75,6 @@ struct DMContext : private boost::noncopyable const size_t delta_small_column_file_bytes; // The expected stable pack rows. const size_t stable_pack_rows; - // The rows of segment to be regarded as small. Small segments will be merged. - const size_t small_segment_rows; - // The bytes of segment to be regarded as small. Small segments will be merged. - const size_t small_segment_bytes; // The number of points to check for calculating region split. const size_t region_split_check_points = 128; @@ -121,8 +117,6 @@ struct DMContext : private boost::noncopyable , delta_small_column_file_rows(settings.dt_segment_delta_small_column_file_rows) , delta_small_column_file_bytes(settings.dt_segment_delta_small_column_file_size) , stable_pack_rows(settings.dt_segment_stable_pack_rows) - , small_segment_rows(settings.dt_segment_limit_rows / 3) - , small_segment_bytes(settings.dt_segment_limit_size / 3) , enable_logical_split(settings.dt_enable_logical_split) , read_delta_only(settings.dt_read_delta_only) , read_stable_only(settings.dt_read_stable_only) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp index 6cd39eef220..b875b57e08c 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp @@ -160,8 +160,8 @@ std::vector DeltaMergeStore::getMergeableSegments(const DMContextPtr // Note: it is possible that there is a very small segment close to a very large segment. // In this case, the small segment will not get merged. It is possible that we can allow // segment merging for this case in future. - auto max_total_rows = context->small_segment_rows; - auto max_total_bytes = context->small_segment_bytes; + auto max_total_rows = context->segment_limit_rows; + auto max_total_bytes = context->segment_limit_bytes; std::vector results; { @@ -517,7 +517,7 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMerge(const DMContextPtr & dm_context, c { auto segment_rows = segment->getEstimatedRows(); auto segment_bytes = segment->getEstimatedBytes(); - if (segment_rows >= dm_context->small_segment_rows || segment_bytes >= dm_context->small_segment_bytes) + if (segment_rows >= dm_context->segment_limit_rows || segment_bytes >= dm_context->segment_limit_bytes) { LOG_TRACE( log, diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_store_background.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_store_background.cpp index e35882b96e5..89a43f810e1 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_store_background.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_store_background.cpp @@ -96,13 +96,13 @@ try ASSERT_EQ(gc_n, 0); // In this case, merge two segments will exceed small_segment_rows, so no merge will happen - db_context->getGlobalContext().getSettingsRef().dt_segment_limit_rows = 55 * 3; + db_context->getGlobalContext().getSettingsRef().dt_segment_limit_rows = 55; gc_n = store->onSyncGc(100, gc_options); ASSERT_EQ(std::vector({0, 50, 100, 150, 200}), getSegmentBreakpoints()); ASSERT_EQ(gc_n, 0); // In this case, we will only merge two segments and then stop. - db_context->getGlobalContext().getSettingsRef().dt_segment_limit_rows = 105 * 3; + db_context->getGlobalContext().getSettingsRef().dt_segment_limit_rows = 105; gc_n = store->onSyncGc(100, gc_options); ASSERT_EQ(std::vector({0, 100, 200}), getSegmentBreakpoints()); ASSERT_EQ(gc_n, 2); @@ -127,7 +127,7 @@ try ensureSegmentBreakpoints({0, 50, 100, 150, 200}); // In this case, we will only merge two segments and then stop. - db_context->getGlobalContext().getSettingsRef().dt_segment_limit_rows = 105 * 3; + db_context->getGlobalContext().getSettingsRef().dt_segment_limit_rows = 105; auto gc_n = store->onSyncGc(1, gc_options); ASSERT_EQ(std::vector({0, 100, 150, 200}), getSegmentBreakpoints()); ASSERT_EQ(gc_n, 1); From 110bda27aebee21aaa86700fabc153a901405e82 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Mon, 5 Dec 2022 20:26:03 +0800 Subject: [PATCH 05/42] *: Refine LimitInputStream (#6268) ref pingcap/tiflash#5900 --- .../src/DataStreams/LimitBlockInputStream.cpp | 60 ++---- dbms/src/DataStreams/LimitBlockInputStream.h | 10 +- dbms/src/DataStreams/LimitTransformAction.cpp | 67 +++++++ dbms/src/DataStreams/LimitTransformAction.h | 36 ++++ dbms/src/DataStreams/tests/CMakeLists.txt | 15 -- .../DataStreams/tests/expression_stream.cpp | 90 --------- dbms/src/DataStreams/tests/filter_stream.cpp | 100 ---------- .../tests/filter_stream_hitlog.cpp | 162 ---------------- dbms/src/DataStreams/tests/sorting_stream.cpp | 173 ------------------ dbms/src/DataStreams/tests/union_stream2.cpp | 72 -------- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 4 +- .../src/Flash/Planner/plans/PhysicalLimit.cpp | 4 +- .../Interpreters/InterpreterSelectQuery.cpp | 21 +-- dbms/src/Interpreters/tests/CMakeLists.txt | 3 - dbms/src/Interpreters/tests/expression.cpp | 152 --------------- .../Storages/System/StorageSystemNumbers.cpp | 2 +- dbms/src/Storages/tests/CMakeLists.txt | 6 - dbms/src/Storages/tests/storage_log.cpp | 122 ------------ dbms/src/Storages/tests/system_numbers.cpp | 59 ------ 19 files changed, 124 insertions(+), 1034 deletions(-) create mode 100644 dbms/src/DataStreams/LimitTransformAction.cpp create mode 100644 dbms/src/DataStreams/LimitTransformAction.h delete mode 100644 dbms/src/DataStreams/tests/expression_stream.cpp delete mode 100644 dbms/src/DataStreams/tests/filter_stream.cpp delete mode 100644 dbms/src/DataStreams/tests/filter_stream_hitlog.cpp delete mode 100644 dbms/src/DataStreams/tests/sorting_stream.cpp delete mode 100644 dbms/src/DataStreams/tests/union_stream2.cpp delete mode 100644 dbms/src/Interpreters/tests/expression.cpp delete mode 100644 dbms/src/Storages/tests/storage_log.cpp delete mode 100644 dbms/src/Storages/tests/system_numbers.cpp diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index f910b8a9e82..ae9bfecffd0 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -22,13 +22,10 @@ namespace DB LimitBlockInputStream::LimitBlockInputStream( const BlockInputStreamPtr & input, size_t limit_, - size_t offset_, - const String & req_id, - bool always_read_till_end_) - : limit(limit_) - , offset(offset_) - , always_read_till_end(always_read_till_end_) - , log(Logger::get(req_id)) + const String & req_id) + : log(Logger::get(req_id)) + , limit_transform_action(input->getHeader(), limit_) + { children.push_back(input); } @@ -36,55 +33,20 @@ LimitBlockInputStream::LimitBlockInputStream( Block LimitBlockInputStream::readImpl() { - Block res; - size_t rows = 0; - - /// pos - how many lines were read, including the last read block + Block res = children.back()->read(); - if (pos >= offset + limit) + if (limit_transform_action.transform(res)) { - if (!always_read_till_end) - return res; - else - { - while (children.back()->read()) - ; - return res; - } + return res; } - - do + else { - res = children.back()->read(); - if (!res) - return res; - rows = res.rows(); - pos += rows; - } while (pos <= offset); - - /// give away the whole block - if (pos >= offset + rows && pos <= offset + limit) - return res; - - /// give away a piece of the block - size_t start = std::max( - static_cast(0), - static_cast(offset) - static_cast(pos) + static_cast(rows)); - - size_t length = std::min( - static_cast(limit), - std::min( - static_cast(pos) - static_cast(offset), - static_cast(limit) + static_cast(offset) - static_cast(pos) + static_cast(rows))); - - for (size_t i = 0; i < res.columns(); ++i) - res.safeGetByPosition(i).column = res.safeGetByPosition(i).column->cut(start, length); - - return res; + return {}; + } } void LimitBlockInputStream::appendInfo(FmtBuffer & buffer) const { - buffer.fmtAppend(", limit = {}", limit); + buffer.fmtAppend(", limit = {}", limit_transform_action.getLimit()); } } // namespace DB diff --git a/dbms/src/DataStreams/LimitBlockInputStream.h b/dbms/src/DataStreams/LimitBlockInputStream.h index e6a7013210b..d90cb19ca4d 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.h +++ b/dbms/src/DataStreams/LimitBlockInputStream.h @@ -15,6 +15,7 @@ #pragma once #include +#include namespace DB { @@ -33,9 +34,7 @@ class LimitBlockInputStream : public IProfilingBlockInputStream LimitBlockInputStream( const BlockInputStreamPtr & input, size_t limit_, - size_t offset_, - const String & req_id, - bool always_read_till_end_ = false); + const String & req_id); String getName() const override { return NAME; } @@ -46,11 +45,8 @@ class LimitBlockInputStream : public IProfilingBlockInputStream void appendInfo(FmtBuffer & buffer) const override; private: - size_t limit; - size_t offset; - size_t pos = 0; - bool always_read_till_end; LoggerPtr log; + LimitTransformAction limit_transform_action; }; } // namespace DB diff --git a/dbms/src/DataStreams/LimitTransformAction.cpp b/dbms/src/DataStreams/LimitTransformAction.cpp new file mode 100644 index 00000000000..7542d49e859 --- /dev/null +++ b/dbms/src/DataStreams/LimitTransformAction.cpp @@ -0,0 +1,67 @@ +// 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 +{ +LimitTransformAction::LimitTransformAction( + const Block & header_, + size_t limit_) + : header(header_) + , limit(limit_) +{ +} + +Block LimitTransformAction::getHeader() const +{ + return header; +} + +size_t LimitTransformAction::getLimit() const +{ + return limit; +} + +bool LimitTransformAction::transform(Block & block) +{ + if (unlikely(!block)) + return true; + + /// pos - how many lines were read, including the last read block + if (pos >= limit) + { + return false; + } + + auto rows = block.rows(); + pos += rows; + if (pos >= rows && pos <= limit) + { + // give away the whole block + return true; + } + else + { + // pos > limit + // give away a piece of the block + assert(rows + limit > pos); + size_t length = rows + limit - pos; + for (size_t i = 0; i < block.columns(); ++i) + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->cut(0, length); + return true; + } +} +} // namespace DB diff --git a/dbms/src/DataStreams/LimitTransformAction.h b/dbms/src/DataStreams/LimitTransformAction.h new file mode 100644 index 00000000000..36defe52602 --- /dev/null +++ b/dbms/src/DataStreams/LimitTransformAction.h @@ -0,0 +1,36 @@ +// 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. +#pragma once + +#include + +namespace DB +{ +struct LimitTransformAction +{ +public: + LimitTransformAction( + const Block & header_, + size_t limit_); + + bool transform(Block & block); + Block getHeader() const; + size_t getLimit() const; + +private: + Block header; + size_t limit; + size_t pos = 0; +}; +} // namespace DB diff --git a/dbms/src/DataStreams/tests/CMakeLists.txt b/dbms/src/DataStreams/tests/CMakeLists.txt index 9e710e9bb7f..cacf667d688 100644 --- a/dbms/src/DataStreams/tests/CMakeLists.txt +++ b/dbms/src/DataStreams/tests/CMakeLists.txt @@ -22,20 +22,5 @@ target_link_libraries (tab_separated_streams dbms clickhouse_functions) add_executable (block_row_transforms block_row_transforms.cpp ${SRCS}) target_link_libraries (block_row_transforms dbms clickhouse_functions) -add_executable (expression_stream expression_stream.cpp ${SRCS}) -target_link_libraries (expression_stream dbms clickhouse_storages_system clickhouse_functions) - add_executable (native_streams native_streams.cpp ${SRCS}) target_link_libraries (native_streams dbms clickhouse_functions) - -add_executable (filter_stream filter_stream.cpp ${SRCS}) -target_link_libraries (filter_stream dbms clickhouse_storages_system clickhouse_functions) - -add_executable (filter_stream_hitlog filter_stream_hitlog.cpp ${SRCS}) -target_link_libraries (filter_stream_hitlog dbms clickhouse_functions) - -add_executable (sorting_stream sorting_stream.cpp ${SRCS}) -target_link_libraries (sorting_stream dbms clickhouse_functions) - -add_executable (union_stream2 union_stream2.cpp ${SRCS}) -target_link_libraries (union_stream2 dbms clickhouse_functions) diff --git a/dbms/src/DataStreams/tests/expression_stream.cpp b/dbms/src/DataStreams/tests/expression_stream.cpp deleted file mode 100644 index 4e81a88ad83..00000000000 --- a/dbms/src/DataStreams/tests/expression_stream.cpp +++ /dev/null @@ -1,90 +0,0 @@ -// 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 -#include -#include -#include -#include - -#include -#include - - -int main(int argc, char ** argv) -try -{ - using namespace DB; - - size_t n = argc == 2 ? parse(argv[1]) : 10ULL; - - std::string input = "SELECT number, number / 3, number * number"; - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); - - Context context = Context::createGlobal(); - - ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared())}); - ExpressionActionsChain chain; - analyzer.appendSelect(chain, false); - analyzer.appendProjectResult(chain); - chain.finalize(); - ExpressionActionsPtr expression = chain.getLastActions(); - - StoragePtr table = StorageSystemNumbers::create("numbers", false); - - Names column_names; - column_names.push_back("number"); - - QueryProcessingStage::Enum stage; - - BlockInputStreamPtr in; - in = table->read(column_names, {}, context, stage, 8192, 1)[0]; - in = std::make_shared(in, expression, ""); - in = std::make_shared(in, 10, std::max(static_cast(0), static_cast(n) - 10)); - - WriteBufferFromOStream out1(std::cout); - RowOutputStreamPtr out2 = std::make_shared(out1, expression->getSampleBlock()); - BlockOutputStreamFromRowOutputStream out(out2, expression->getSampleBlock()); - - { - Stopwatch stopwatch; - stopwatch.start(); - - copyData(*in, out); - - stopwatch.stop(); - std::cout << std::fixed << std::setprecision(2) - << "Elapsed " << stopwatch.elapsedSeconds() << " sec." - << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." - << std::endl; - } - - return 0; -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - throw; -} diff --git a/dbms/src/DataStreams/tests/filter_stream.cpp b/dbms/src/DataStreams/tests/filter_stream.cpp deleted file mode 100644 index f25d9dc7902..00000000000 --- a/dbms/src/DataStreams/tests/filter_stream.cpp +++ /dev/null @@ -1,100 +0,0 @@ -// 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 -#include - -#include - -#include -#include -#include - -#include -#include -#include - - -int main(int argc, char ** argv) -try -{ - using namespace DB; - - size_t n = argc == 2 ? parse(argv[1]) : 10ULL; - - std::string input = "SELECT number, number % 3 == 1"; - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); - - formatAST(*ast, std::cerr); - std::cerr << std::endl; - - Context context = Context::createGlobal(); - - ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared())}); - ExpressionActionsChain chain; - analyzer.appendSelect(chain, false); - analyzer.appendProjectResult(chain); - chain.finalize(); - ExpressionActionsPtr expression = chain.getLastActions(); - - StoragePtr table = StorageSystemNumbers::create("numbers", false); - - Names column_names; - column_names.push_back("number"); - - QueryProcessingStage::Enum stage; - - BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0]; - in = std::make_shared(in, expression, "equals(modulo(number, 3), 1)"); - in = std::make_shared(in, 10, std::max(static_cast(0), static_cast(n) - 10)); - - WriteBufferFromOStream ob(std::cout); - RowOutputStreamPtr out_ = std::make_shared(ob, expression->getSampleBlock()); - BlockOutputStreamFromRowOutputStream out(out_, expression->getSampleBlock()); - - - { - Stopwatch stopwatch; - stopwatch.start(); - - copyData(*in, out); - - stopwatch.stop(); - std::cout << std::fixed << std::setprecision(2) - << "Elapsed " << stopwatch.elapsedSeconds() << " sec." - << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." - << std::endl; - } - - return 0; -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - throw; -} diff --git a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp b/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp deleted file mode 100644 index 422b8c15afd..00000000000 --- a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp +++ /dev/null @@ -1,162 +0,0 @@ -// 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 -#include - -#include -#include -#include -#include -#include - -#include -#include -#include - -#include -#include -#include - - -int main(int, char **) -{ - using namespace DB; - - try - { - NamesAndTypesList names_and_types_list - { - {"WatchID", std::make_shared()}, - {"JavaEnable", std::make_shared()}, - {"Title", std::make_shared()}, - {"EventTime", std::make_shared()}, - {"CounterID", std::make_shared()}, - {"ClientIP", std::make_shared()}, - {"RegionID", std::make_shared()}, - {"UniqID", std::make_shared()}, - {"CounterClass", std::make_shared()}, - {"OS", std::make_shared()}, - {"UserAgent", std::make_shared()}, - {"URL", std::make_shared()}, - {"Referer", std::make_shared()}, - {"ResolutionWidth", std::make_shared()}, - {"ResolutionHeight", std::make_shared()}, - {"ResolutionDepth", std::make_shared()}, - {"FlashMajor", std::make_shared()}, - {"FlashMinor", std::make_shared()}, - {"FlashMinor2", std::make_shared()}, - {"NetMajor", std::make_shared()}, - {"NetMinor", std::make_shared()}, - {"UserAgentMajor", std::make_shared()}, - {"UserAgentMinor", std::make_shared(2)}, - {"CookieEnable", std::make_shared()}, - {"JavascriptEnable", std::make_shared()}, - {"IsMobile", std::make_shared()}, - {"MobilePhone", std::make_shared()}, - {"MobilePhoneModel", std::make_shared()}, - {"Params", std::make_shared()}, - {"IPNetworkID", std::make_shared()}, - {"TraficSourceID", std::make_shared()}, - {"SearchEngineID", std::make_shared()}, - {"SearchPhrase", std::make_shared()}, - {"AdvEngineID", std::make_shared()}, - {"IsArtifical", std::make_shared()}, - {"WindowClientWidth", std::make_shared()}, - {"WindowClientHeight", std::make_shared()}, - {"ClientTimeZone", std::make_shared()}, - {"ClientEventTime", std::make_shared()}, - {"SilverlightVersion1", std::make_shared()}, - {"SilverlightVersion2", std::make_shared()}, - {"SilverlightVersion3", std::make_shared()}, - {"SilverlightVersion4", std::make_shared()}, - {"PageCharset", std::make_shared()}, - {"CodeVersion", std::make_shared()}, - {"IsLink", std::make_shared()}, - {"IsDownload", std::make_shared()}, - {"IsNotBounce", std::make_shared()}, - {"FUniqID", std::make_shared()}, - {"OriginalURL", std::make_shared()}, - {"HID", std::make_shared()}, - {"IsOldCounter", std::make_shared()}, - {"IsEvent", std::make_shared()}, - {"IsParameter", std::make_shared()}, - {"DontCountHits", std::make_shared()}, - {"WithHash", std::make_shared()}, - }; - - Context context = Context::createGlobal(); - - std::string input = "SELECT UniqID, URL, CounterID, IsLink WHERE URL = 'http://mail.yandex.ru/neo2/#inbox'"; - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); - - formatAST(*ast, std::cerr); - std::cerr << std::endl; - - /// create an object of an existing hit log table - - StoragePtr table = StorageLog::create( - "./", "HitLog", ColumnsDescription{names_and_types_list}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE); - table->startup(); - - /// read from it, apply the expression, filter, and write in tsv form to the console - - ExpressionAnalyzer analyzer(ast, context, nullptr, names_and_types_list); - ExpressionActionsChain chain; - analyzer.appendSelect(chain, false); - analyzer.appendWhere(chain, false); - chain.finalize(); - ExpressionActionsPtr expression = chain.getLastActions(); - - Names column_names - { - "UniqID", - "URL", - "CounterID", - "IsLink", - }; - - QueryProcessingStage::Enum stage; - - BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0]; - in = std::make_shared(in, expression, "equals(URL, 'http://mail.yandex.ru/neo2/#inbox')"); - //in = std::make_shared(in, 10, 0); - - WriteBufferFromOStream ob(std::cout); - RowOutputStreamPtr out_ = std::make_shared(ob, expression->getSampleBlock()); - BlockOutputStreamFromRowOutputStream out(out_, in->getHeader()); - - copyData(*in, out); - } - catch (const Exception & e) - { - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; - } - - return 0; -} diff --git a/dbms/src/DataStreams/tests/sorting_stream.cpp b/dbms/src/DataStreams/tests/sorting_stream.cpp deleted file mode 100644 index 24f2517cacf..00000000000 --- a/dbms/src/DataStreams/tests/sorting_stream.cpp +++ /dev/null @@ -1,173 +0,0 @@ -// 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 -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -using namespace DB; - - -int main(int argc, char ** argv) -try -{ - NamesAndTypesList names_and_types_list{ - {"WatchID", std::make_shared()}, - {"JavaEnable", std::make_shared()}, - {"Title", std::make_shared()}, - {"EventTime", std::make_shared()}, - {"CounterID", std::make_shared()}, - {"ClientIP", std::make_shared()}, - {"RegionID", std::make_shared()}, - {"UniqID", std::make_shared()}, - {"CounterClass", std::make_shared()}, - {"OS", std::make_shared()}, - {"UserAgent", std::make_shared()}, - {"URL", std::make_shared()}, - {"Referer", std::make_shared()}, - {"ResolutionWidth", std::make_shared()}, - {"ResolutionHeight", std::make_shared()}, - {"ResolutionDepth", std::make_shared()}, - {"FlashMajor", std::make_shared()}, - {"FlashMinor", std::make_shared()}, - {"FlashMinor2", std::make_shared()}, - {"NetMajor", std::make_shared()}, - {"NetMinor", std::make_shared()}, - {"UserAgentMajor", std::make_shared()}, - {"UserAgentMinor", std::make_shared(2)}, - {"CookieEnable", std::make_shared()}, - {"JavascriptEnable", std::make_shared()}, - {"IsMobile", std::make_shared()}, - {"MobilePhone", std::make_shared()}, - {"MobilePhoneModel", std::make_shared()}, - {"Params", std::make_shared()}, - {"IPNetworkID", std::make_shared()}, - {"TraficSourceID", std::make_shared()}, - {"SearchEngineID", std::make_shared()}, - {"SearchPhrase", std::make_shared()}, - {"AdvEngineID", std::make_shared()}, - {"IsArtifical", std::make_shared()}, - {"WindowClientWidth", std::make_shared()}, - {"WindowClientHeight", std::make_shared()}, - {"ClientTimeZone", std::make_shared()}, - {"ClientEventTime", std::make_shared()}, - {"SilverlightVersion1", std::make_shared()}, - {"SilverlightVersion2", std::make_shared()}, - {"SilverlightVersion3", std::make_shared()}, - {"SilverlightVersion4", std::make_shared()}, - {"PageCharset", std::make_shared()}, - {"CodeVersion", std::make_shared()}, - {"IsLink", std::make_shared()}, - {"IsDownload", std::make_shared()}, - {"IsNotBounce", std::make_shared()}, - {"FUniqID", std::make_shared()}, - {"OriginalURL", std::make_shared()}, - {"HID", std::make_shared()}, - {"IsOldCounter", std::make_shared()}, - {"IsEvent", std::make_shared()}, - {"IsParameter", std::make_shared()}, - {"DontCountHits", std::make_shared()}, - {"WithHash", std::make_shared()}, - }; - - using NamesAndTypesMap = std::map; - NamesAndTypesMap names_and_types_map; - - for (const auto & name_type : names_and_types_list) - names_and_types_map.emplace(name_type.name, name_type.type); - - std::string input = "SELECT UniqID, URL, CounterID, IsLink"; - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); - - formatAST(*ast, std::cerr); - std::cerr << std::endl; - - /// create an object of an existing hit log table - - StoragePtr table = StorageLog::create( - "./", - "HitLog", - ColumnsDescription{names_and_types_list}, - DEFAULT_MAX_COMPRESS_BLOCK_SIZE); - table->startup(); - - /// read from it, sort it, and write it in tsv form to the console - - Names column_names{ - "UniqID", - "URL", - "CounterID", - "IsLink", - }; - - DataTypes result_types = DataTypes{ - names_and_types_map["UniqID"], - names_and_types_map["URL"], - names_and_types_map["CounterID"], - names_and_types_map["IsLink"], - }; - - Block sample; - for (const auto & type : result_types) - { - ColumnWithTypeAndName col; - col.type = type; - sample.insert(std::move(col)); - } - - SortDescription sort_columns; - sort_columns.push_back(SortColumnDescription(1, -1, -1)); - sort_columns.push_back(SortColumnDescription(2, 1, 1)); - sort_columns.push_back(SortColumnDescription(0, 1, 1)); - sort_columns.push_back(SortColumnDescription(3, 1, 1)); - - QueryProcessingStage::Enum stage; - - BlockInputStreamPtr in = table->read(column_names, {}, Context::createGlobal(), stage, argc == 2 ? atoi(argv[1]) : 65536, 1)[0]; - in = std::make_shared(in, sort_columns, ""); - in = std::make_shared(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, ""); - //in = std::make_shared(in, 10, 0); - - WriteBufferFromOStream ob(std::cout); - RowOutputStreamPtr out_ = std::make_shared(ob, sample); - BlockOutputStreamFromRowOutputStream out(out_, sample); - - copyData(*in, out); - - return 0; -} -catch (const Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - throw; -} diff --git a/dbms/src/DataStreams/tests/union_stream2.cpp b/dbms/src/DataStreams/tests/union_stream2.cpp deleted file mode 100644 index fb3f7238414..00000000000 --- a/dbms/src/DataStreams/tests/union_stream2.cpp +++ /dev/null @@ -1,72 +0,0 @@ -// 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 -#include - -#include -#include - - -using namespace DB; - -int main(int, char **) -try -{ - Context context = Context::createGlobal(); - Settings settings = context.getSettings(); - - context.setPath("./"); - - loadMetadata(context); - - Names column_names; - column_names.push_back("WatchID"); - - StoragePtr table = context.getTable("default", "hits6"); - - QueryProcessingStage::Enum stage; - BlockInputStreams streams = table->read(column_names, {}, context, stage, settings.max_block_size, settings.max_threads); - - for (size_t i = 0, size = streams.size(); i < size; ++i) - streams[i] = std::make_shared(streams[i]); - - BlockInputStreamPtr stream = std::make_shared>(streams, BlockInputStreams{}, settings.max_threads, /*req_id=*/""); - stream = std::make_shared(stream, 10, 0, ""); - - WriteBufferFromFileDescriptor wb(STDERR_FILENO); - Block sample = table->getSampleBlock(); - BlockOutputStreamPtr out = context.getOutputFormat("TabSeparated", wb, sample); - - copyData(*stream, *out); - - return 0; -} -catch (const Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl - << std::endl - << "Stack trace:" << std::endl - << e.getStackTrace().toString(); - return 1; -} diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 7af8a63e978..a88fc1503b3 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -740,11 +740,11 @@ void DAGQueryBlockInterpreter::executeLimit(DAGPipeline & pipeline) limit = query_block.limit_or_topn->limit().limit(); else limit = query_block.limit_or_topn->topn().limit(); - pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, 0, log->identifier(), false); }); + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, log->identifier()); }); if (pipeline.hasMoreThanOneStream()) { executeUnion(pipeline, max_streams, log, false, "for partial limit"); - pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, 0, log->identifier(), false); }); + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, log->identifier()); }); } } diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp index 2722e7e1869..374a1007af5 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp @@ -42,11 +42,11 @@ void PhysicalLimit::transformImpl(DAGPipeline & pipeline, Context & context, siz { child->transform(pipeline, context, max_streams); - pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, 0, log->identifier(), false); }); + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, log->identifier()); }); if (pipeline.hasMoreThanOneStream()) { executeUnion(pipeline, max_streams, log, false, "for partial limit"); - pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, 0, log->identifier(), false); }); + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, log->identifier()); }); } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 795526ff4af..fdb00ad7b06 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1284,7 +1284,7 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) if (query.limit_length) { pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, limit_length + limit_offset, 0, /*req_id=*/"", false); + stream = std::make_shared(stream, limit_length + limit_offset, /*req_id=*/""); }); } } @@ -1341,25 +1341,8 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) /// If there is LIMIT if (query.limit_length) { - /** Rare case: - * if there is no WITH TOTALS and there is a subquery in FROM, and there is WITH TOTALS on one of the levels, - * then when using LIMIT, you should read the data to the end, rather than cancel the query earlier, - * because if you cancel the query, we will not get `totals` data from the remote server. - * - * Another case: - * if there is WITH TOTALS and there is no ORDER BY, then read the data to the end, - * otherwise TOTALS is counted according to incomplete data. - */ - bool always_read_till_end = false; - - if (query.group_by_with_totals && !query.order_expression_list) - always_read_till_end = true; - - if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query)) - always_read_till_end = true; - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, limit_length, limit_offset, /*req_id=*/"", always_read_till_end); + stream = std::make_shared(stream, limit_length, /*req_id=*/""); }); } } diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt index 1ae710cee6d..f310837b3f0 100644 --- a/dbms/src/Interpreters/tests/CMakeLists.txt +++ b/dbms/src/Interpreters/tests/CMakeLists.txt @@ -12,9 +12,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -add_executable (expression expression.cpp) -target_link_libraries (expression dbms) - add_executable (create_query create_query.cpp) target_link_libraries (create_query dbms) diff --git a/dbms/src/Interpreters/tests/expression.cpp b/dbms/src/Interpreters/tests/expression.cpp deleted file mode 100644 index 83fa7b37130..00000000000 --- a/dbms/src/Interpreters/tests/expression.cpp +++ /dev/null @@ -1,152 +0,0 @@ -// 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 -#include - -#include -#include -#include -#include -#include - -#include -#include -#include - - -int main(int argc, char ** argv) -{ - using namespace DB; - - try - { - std::string input = "SELECT x, s1, s2, " - "/*" - "2 + x * 2, x * 2, x % 3 == 1, " - "s1 == 'abc', s1 == s2, s1 != 'abc', s1 != s2, " - "s1 < 'abc', s1 < s2, s1 > 'abc', s1 > s2, " - "s1 <= 'abc', s1 <= s2, s1 >= 'abc', s1 >= s2, " - "*/" - "s1 < s2 AND x % 3 < x % 5"; - - ParserSelectQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); - - formatAST(*ast, std::cerr); - std::cerr << std::endl; - - Context context = Context::createGlobal(); - NamesAndTypesList columns - { - {"x", std::make_shared()}, - {"s1", std::make_shared()}, - {"s2", std::make_shared()} - }; - - ExpressionAnalyzer analyzer(ast, context, {}, columns); - ExpressionActionsChain chain; - analyzer.appendSelect(chain, false); - analyzer.appendProjectResult(chain); - chain.finalize(); - ExpressionActionsPtr expression = chain.getLastActions(); - - size_t n = argc == 2 ? atoi(argv[1]) : 10; - - Block block; - - { - ColumnWithTypeAndName column; - column.name = "x"; - column.type = std::make_shared(); - auto col = ColumnInt16::create(); - auto & vec_x = col->getData(); - - vec_x.resize(n); - for (size_t i = 0; i < n; ++i) - vec_x[i] = i % 9; - - column.column = std::move(col); - block.insert(column); - } - - const char * strings[] = {"abc", "def", "abcd", "defg", "ac"}; - - { - ColumnWithTypeAndName column; - column.name = "s1"; - column.type = std::make_shared(); - auto col = ColumnString::create(); - - for (size_t i = 0; i < n; ++i) - col->insert(std::string(strings[i % 5])); - - column.column = std::move(col); - block.insert(column); - } - - { - ColumnWithTypeAndName column; - column.name = "s2"; - column.type = std::make_shared(); - auto col = ColumnString::create(); - - for (size_t i = 0; i < n; ++i) - col->insert(std::string(strings[i % 3])); - - column.column = std::move(col); - block.insert(column); - } - - { - Stopwatch stopwatch; - stopwatch.start(); - - expression->execute(block); - - stopwatch.stop(); - std::cout << std::fixed << std::setprecision(2) - << "Elapsed " << stopwatch.elapsedSeconds() << " sec." - << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." - << std::endl; - } - - auto is = std::make_shared(block); - LimitBlockInputStream lis(is, 20, std::max(0, static_cast(n) - 20)); - WriteBufferFromOStream out_buf(std::cout); - RowOutputStreamPtr os_ = std::make_shared(out_buf, block); - BlockOutputStreamFromRowOutputStream os(os_, is->getHeader()); - - copyData(lis, os); - } - catch (const Exception & e) - { - std::cerr << e.displayText() << std::endl; - } - - return 0; -} diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index d2ea70e53ae..68130897963 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -96,7 +96,7 @@ BlockInputStreams StorageSystemNumbers::read( res[i] = std::make_shared(max_block_size, i * max_block_size, num_streams * max_block_size); if (limit) /// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly. - res[i] = std::make_shared(res[i], limit * (i + 1) / num_streams - limit * i / num_streams, 0, /*req_id=*/""); + res[i] = std::make_shared(res[i], limit * (i + 1) / num_streams - limit * i / num_streams, /*req_id=*/""); } return res; diff --git a/dbms/src/Storages/tests/CMakeLists.txt b/dbms/src/Storages/tests/CMakeLists.txt index 4c7470985fa..6216f092c25 100644 --- a/dbms/src/Storages/tests/CMakeLists.txt +++ b/dbms/src/Storages/tests/CMakeLists.txt @@ -14,12 +14,6 @@ include_directories (${CMAKE_CURRENT_BINARY_DIR}) -add_executable (system_numbers system_numbers.cpp) -target_link_libraries (system_numbers dbms clickhouse_storages_system) - -add_executable (storage_log storage_log.cpp) -target_link_libraries (storage_log dbms) - add_executable (hit_log hit_log.cpp) target_link_libraries (hit_log dbms) diff --git a/dbms/src/Storages/tests/storage_log.cpp b/dbms/src/Storages/tests/storage_log.cpp deleted file mode 100644 index 542d78e872a..00000000000 --- a/dbms/src/Storages/tests/storage_log.cpp +++ /dev/null @@ -1,122 +0,0 @@ -// 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 -#include -#include - - -int main(int, char **) -try -{ - using namespace DB; - - const size_t rows = 10000000; - - /// create table with a pair of columns - - NamesAndTypesList names_and_types; - names_and_types.emplace_back("a", std::make_shared()); - names_and_types.emplace_back("b", std::make_shared()); - - StoragePtr table = StorageLog::create( - "./", "test", ColumnsDescription{names_and_types}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE); - table->startup(); - - /// write into it - { - Block block; - - { - ColumnWithTypeAndName column; - column.name = "a"; - column.type = table->getColumn("a").type; - auto col = column.type->createColumn(); - ColumnUInt64::Container & vec = typeid_cast(*col).getData(); - - vec.resize(rows); - for (size_t i = 0; i < rows; ++i) - vec[i] = i; - - column.column = std::move(col); - block.insert(column); - } - - { - ColumnWithTypeAndName column; - column.name = "b"; - column.type = table->getColumn("b").type; - auto col = column.type->createColumn(); - ColumnUInt8::Container & vec = typeid_cast(*col).getData(); - - vec.resize(rows); - for (size_t i = 0; i < rows; ++i) - vec[i] = i * 2; - - column.column = std::move(col); - block.insert(column); - } - - BlockOutputStreamPtr out = table->write({}, {}); - out->write(block); - } - - /// read from it - { - Names column_names; - column_names.push_back("a"); - column_names.push_back("b"); - - QueryProcessingStage::Enum stage; - - BlockInputStreamPtr in = table->read(column_names, {}, Context::createGlobal(), stage, 8192, 1)[0]; - - Block sample; - { - ColumnWithTypeAndName col; - col.type = std::make_shared(); - sample.insert(std::move(col)); - } - { - ColumnWithTypeAndName col; - col.type = std::make_shared(); - sample.insert(std::move(col)); - } - - WriteBufferFromOStream out_buf(std::cout); - - LimitBlockInputStream in_limit(in, 10, 0); - RowOutputStreamPtr output_ = std::make_shared(out_buf, sample); - BlockOutputStreamFromRowOutputStream output(output_, sample); - - copyData(in_limit, output); - } - - return 0; -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; -} diff --git a/dbms/src/Storages/tests/system_numbers.cpp b/dbms/src/Storages/tests/system_numbers.cpp deleted file mode 100644 index 174e3bfae78..00000000000 --- a/dbms/src/Storages/tests/system_numbers.cpp +++ /dev/null @@ -1,59 +0,0 @@ -// 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 - - -int main(int, char **) -try -{ - using namespace DB; - - StoragePtr table = StorageSystemNumbers::create("numbers", false); - - Names column_names; - column_names.push_back("number"); - - Block sample; - ColumnWithTypeAndName col; - col.type = std::make_shared(); - sample.insert(std::move(col)); - - WriteBufferFromOStream out_buf(std::cout); - - QueryProcessingStage::Enum stage; - - LimitBlockInputStream input(table->read(column_names, {}, Context::createGlobal(), stage, 10, 1)[0], 10, 96); - RowOutputStreamPtr output_ = std::make_shared(out_buf, sample); - BlockOutputStreamFromRowOutputStream output(output_, sample); - - copyData(input, output); - - return 0; -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; -} From 8d406ba86848a48eb0fd04903bee66eccaee5def Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Tue, 6 Dec 2022 22:40:04 +0800 Subject: [PATCH 06/42] reduce column files num under heavy write pressure (#6432) close pingcap/tiflash#6431 --- .../DeltaMerge/Delta/ColumnFileFlushTask.cpp | 2 +- .../Delta/ColumnFilePersistedSet.cpp | 344 ++++++------------ .../DeltaMerge/Delta/ColumnFilePersistedSet.h | 25 +- .../DeltaMerge/Delta/MinorCompaction.cpp | 5 +- .../DeltaMerge/Delta/MinorCompaction.h | 4 +- .../tests/gtest_dm_delta_value_space.cpp | 47 +-- 6 files changed, 154 insertions(+), 273 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp b/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp index 71a504dc6e0..db284a41fdb 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFileFlushTask.cpp @@ -91,7 +91,7 @@ bool ColumnFileFlushTask::commit(ColumnFilePersistedSetPtr & persisted_file_set, } // serialize metadata and update persisted_file_set - if (!persisted_file_set->appendPersistedColumnFilesToLevel0(new_column_files, wbs)) + if (!persisted_file_set->appendPersistedColumnFiles(new_column_files, wbs)) return false; mem_table_set->removeColumnFilesInFlushTask(*this); diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp index 72770e79541..54cd00096a0 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp @@ -27,83 +27,60 @@ namespace DB { namespace DM { -inline ColumnFilePersisteds flattenColumnFileLevels(const ColumnFilePersistedSet::ColumnFilePersistedLevels & file_levels) -{ - ColumnFilePersisteds column_files; - // Last level first - for (auto level_it = file_levels.rbegin(); level_it != file_levels.rend(); ++level_it) - { - for (const auto & file : *level_it) - { - column_files.emplace_back(file); - } - } - return column_files; -} - -inline void serializeColumnFilePersistedLevels(WriteBatches & wbs, PageId id, const ColumnFilePersistedSet::ColumnFilePersistedLevels & file_levels) +inline void serializeColumnFilePersisteds(WriteBatches & wbs, PageId id, const ColumnFilePersisteds & persisted_files) { MemoryWriteBuffer buf(0, COLUMN_FILE_SERIALIZE_BUFFER_SIZE); - auto column_files = flattenColumnFileLevels(file_levels); - serializeSavedColumnFiles(buf, column_files); + serializeSavedColumnFiles(buf, persisted_files); auto data_size = buf.count(); wbs.meta.putPage(id, 0, buf.tryGetReadBuffer(), data_size); } void ColumnFilePersistedSet::updateColumnFileStats() { - size_t new_persisted_files_count = 0; size_t new_rows = 0; size_t new_bytes = 0; size_t new_deletes = 0; - for (auto & file_level : persisted_files_levels) + for (auto & file : persisted_files) { - new_persisted_files_count += file_level.size(); - for (auto & file : file_level) - { - new_rows += file->getRows(); - new_bytes += file->getBytes(); - new_deletes += file->getDeletes(); - } + new_rows += file->getRows(); + new_bytes += file->getBytes(); + new_deletes += file->getDeletes(); } - persisted_files_count = new_persisted_files_count; - persisted_files_level_count = persisted_files_levels.size(); + persisted_files_count = persisted_files.size(); rows = new_rows; bytes = new_bytes; deletes = new_deletes; } -void ColumnFilePersistedSet::checkColumnFiles(const ColumnFilePersistedLevels & new_column_file_levels) +void ColumnFilePersistedSet::checkColumnFiles(const ColumnFilePersisteds & new_column_files) { if constexpr (!DM_RUN_CHECK) return; size_t new_rows = 0; size_t new_deletes = 0; - for (const auto & level : new_column_file_levels) + for (const auto & file : new_column_files) { - for (const auto & file : level) - { - new_rows += file->getRows(); - new_deletes += file->isDeleteRange(); - } + new_rows += file->getRows(); + new_deletes += file->isDeleteRange(); } - if (unlikely(new_rows != rows || new_deletes != deletes)) - { - LOG_ERROR(log, "Rows and deletes check failed. Actual: rows[{}], deletes[{}]. Expected: rows[{}], deletes[{}]. Current column files: {}, new column files: {}.", new_rows, new_deletes, rows.load(), deletes.load(), columnFilesToString(flattenColumnFileLevels(persisted_files_levels)), columnFilesToString(flattenColumnFileLevels(new_column_file_levels))); - throw Exception("Rows and deletes check failed.", ErrorCodes::LOGICAL_ERROR); - } + RUNTIME_CHECK_MSG(new_rows == rows && new_deletes == deletes, + "Rows and deletes check failed. Actual: rows[{}], deletes[{}]. Expected: rows[{}], deletes[{}]. Current column files: {}, new column files: {}.", // + new_rows, + new_deletes, + rows.load(), + deletes.load(), + columnFilesToString(persisted_files), + columnFilesToString(new_column_files)); } ColumnFilePersistedSet::ColumnFilePersistedSet( // PageId metadata_id_, const ColumnFilePersisteds & persisted_column_files) : metadata_id(metadata_id_) + , persisted_files(persisted_column_files) , log(Logger::get()) { - // TODO: place column file to different levels, but it seems no need to do it currently because we only do minor compaction on really small files? - persisted_files_levels.push_back(persisted_column_files); - updateColumnFileStats(); } @@ -120,27 +97,21 @@ ColumnFilePersistedSetPtr ColumnFilePersistedSet::restore( // void ColumnFilePersistedSet::saveMeta(WriteBatches & wbs) const { - serializeColumnFilePersistedLevels(wbs, metadata_id, persisted_files_levels); + serializeColumnFilePersisteds(wbs, metadata_id, persisted_files); } void ColumnFilePersistedSet::recordRemoveColumnFilesPages(WriteBatches & wbs) const { - for (const auto & level : persisted_files_levels) - { - for (const auto & file : level) - file->removeData(wbs); - } + for (const auto & file : persisted_files) + file->removeData(wbs); } BlockPtr ColumnFilePersistedSet::getLastSchema() { - for (const auto & level : persisted_files_levels) + for (auto it = persisted_files.rbegin(); it != persisted_files.rend(); ++it) { - for (auto it = level.rbegin(); it != level.rend(); ++it) - { - if (auto * t_file = (*it)->tryToTinyFile(); t_file) - return t_file->getSchema(); - } + if (auto * t_file = (*it)->tryToTinyFile(); t_file) + return t_file->getSchema(); } return {}; } @@ -156,26 +127,15 @@ ColumnFilePersisteds ColumnFilePersistedSet::diffColumnFiles(const ColumnFiles & // that this function is called under a for_update snapshot context. RUNTIME_CHECK(previous_column_files.size() <= getColumnFileCount()); - // We check in the direction from the last level to the first level. - // In every level, we check from the begin to the last. auto it_1 = previous_column_files.begin(); - auto level_it = persisted_files_levels.rbegin(); - auto it_2 = level_it->begin(); + auto it_2 = persisted_files.begin(); bool check_success = true; if (likely(previous_column_files.size() <= persisted_files_count.load())) { - while (it_1 != previous_column_files.end() && level_it != persisted_files_levels.rend()) + while (it_1 != previous_column_files.end() && it_2 != persisted_files.end()) { - if (it_2 == level_it->end()) - { - level_it++; - if (unlikely(level_it == persisted_files_levels.rend())) - throw Exception("Delta Check head algorithm broken", ErrorCodes::LOGICAL_ERROR); - it_2 = level_it->begin(); - continue; - } // We allow passing unflushed memtable files to `previous_column_files`, these heads will be skipped anyway. - if (!(*it_2)->mayBeFlushedFrom(&**it_1) && !(*it_1)->isSame(&**it_1)) + if (!(*it_2)->mayBeFlushedFrom(&(**it_1)) && !(*it_2)->isSame(&(**it_1))) { check_success = false; break; @@ -196,21 +156,13 @@ ColumnFilePersisteds ColumnFilePersistedSet::diffColumnFiles(const ColumnFiles & if (unlikely(!check_success)) { - LOG_ERROR(log, "{}, Delta Check head failed, unexpected size. head column files: {}, level details: {}", info(), columnFilesToString(previous_column_files), levelsInfo()); + LOG_ERROR(log, "{}, Delta Check head failed, unexpected size. head column files: {}, persisted column files: {}", info(), columnFilesToString(previous_column_files), detailInfo()); throw Exception("Check head failed, unexpected size", ErrorCodes::LOGICAL_ERROR); } ColumnFilePersisteds tail; - while (level_it != persisted_files_levels.rend()) + while (it_2 != persisted_files.end()) { - if (it_2 == level_it->end()) - { - level_it++; - if (level_it == persisted_files_levels.rend()) - break; - it_2 = level_it->begin(); - continue; - } const auto & column_file = *it_2; tail.push_back(column_file); it_2++; @@ -222,15 +174,12 @@ ColumnFilePersisteds ColumnFilePersistedSet::diffColumnFiles(const ColumnFiles & size_t ColumnFilePersistedSet::getTotalCacheRows() const { size_t cache_rows = 0; - for (const auto & level : persisted_files_levels) + for (const auto & file : persisted_files) { - for (const auto & file : level) + if (auto * tf = file->tryToTinyFile(); tf) { - if (auto * tf = file->tryToTinyFile(); tf) - { - if (auto && c = tf->getCache(); c) - cache_rows += c->block.rows(); - } + if (auto && c = tf->getCache(); c) + cache_rows += c->block.rows(); } } return cache_rows; @@ -239,15 +188,12 @@ size_t ColumnFilePersistedSet::getTotalCacheRows() const size_t ColumnFilePersistedSet::getTotalCacheBytes() const { size_t cache_bytes = 0; - for (const auto & level : persisted_files_levels) + for (const auto & file : persisted_files) { - for (const auto & file : level) + if (auto * tf = file->tryToTinyFile(); tf) { - if (auto * tf = file->tryToTinyFile(); tf) - { - if (auto && c = tf->getCache(); c) - cache_bytes += c->block.allocatedBytes(); - } + if (auto && c = tf->getCache(); c) + cache_bytes += c->block.allocatedBytes(); } } return cache_bytes; @@ -256,15 +202,12 @@ size_t ColumnFilePersistedSet::getTotalCacheBytes() const size_t ColumnFilePersistedSet::getValidCacheRows() const { size_t cache_rows = 0; - for (const auto & level : persisted_files_levels) + for (const auto & file : persisted_files) { - for (const auto & file : level) + if (auto * tf = file->tryToTinyFile(); tf) { - if (auto * tf = file->tryToTinyFile(); tf) - { - if (auto && c = tf->getCache(); c) - cache_rows += tf->getRows(); - } + if (auto && c = tf->getCache(); c) + cache_rows += tf->getRows(); } } return cache_rows; @@ -281,91 +224,74 @@ bool ColumnFilePersistedSet::checkAndIncreaseFlushVersion(size_t task_flush_vers return true; } -bool ColumnFilePersistedSet::appendPersistedColumnFilesToLevel0(const ColumnFilePersisteds & column_files, WriteBatches & wbs) +bool ColumnFilePersistedSet::appendPersistedColumnFiles(const ColumnFilePersisteds & column_files, WriteBatches & wbs) { - ColumnFilePersistedLevels new_persisted_files_levels; - for (auto & level : persisted_files_levels) + ColumnFilePersisteds new_persisted_files; + for (const auto & file : persisted_files) { - auto & new_level = new_persisted_files_levels.emplace_back(); - for (auto & file : level) - new_level.push_back(file); + new_persisted_files.push_back(file); + } + for (const auto & file : column_files) + { + new_persisted_files.push_back(file); } - if (new_persisted_files_levels.empty()) - new_persisted_files_levels.emplace_back(); - auto & new_level_0 = new_persisted_files_levels[0]; - - for (const auto & f : column_files) - new_level_0.push_back(f); - /// Save the new metadata of column files to disk. - serializeColumnFilePersistedLevels(wbs, metadata_id, new_persisted_files_levels); + serializeColumnFilePersisteds(wbs, metadata_id, new_persisted_files); wbs.writeMeta(); /// Commit updates in memory. - persisted_files_levels.swap(new_persisted_files_levels); + persisted_files.swap(new_persisted_files); updateColumnFileStats(); - LOG_DEBUG(log, "{}, after append {} column files, level info: {}", info(), column_files.size(), levelsInfo()); + LOG_DEBUG(log, "{}, after append {} column files, persisted column files: {}", info(), column_files.size(), detailInfo()); return true; } MinorCompactionPtr ColumnFilePersistedSet::pickUpMinorCompaction(DMContext & context) { - // Every time we try to compact all column files in a specific level. + // Every time we try to compact all column files. // For ColumnFileTiny, we will try to combine small `ColumnFileTiny`s to a bigger one. - // For ColumnFileDeleteRange and ColumnFileBig, we will simply move them to the next level. - // And only if there exists some small `ColumnFileTiny`s which can be combined together, we will actually do the compaction. - size_t check_level_num = 0; - while (check_level_num < persisted_files_levels.size()) + // For ColumnFileDeleteRange and ColumnFileBig, we keep them intact. + // And only if there exists some small `ColumnFileTiny`s which can be combined, we will actually do the compaction. + auto compaction = std::make_shared(minor_compaction_version); + if (!persisted_files.empty()) { - check_level_num += 1; - if (next_compaction_level >= persisted_files_levels.size()) - next_compaction_level = 0; - - auto compaction = std::make_shared(next_compaction_level, minor_compaction_version); - auto & level = persisted_files_levels[next_compaction_level]; - next_compaction_level++; - if (!level.empty()) + bool is_all_trivial_move = true; + MinorCompaction::Task cur_task; + auto pack_up_cur_task = [&]() { + bool is_trivial_move = compaction->packUpTask(std::move(cur_task)); + is_all_trivial_move = is_all_trivial_move && is_trivial_move; + cur_task = {}; + }; + for (auto & file : persisted_files) { - bool is_all_trivial_move = true; - MinorCompaction::Task cur_task; - for (auto & file : level) + if (auto * t_file = file->tryToTinyFile(); t_file) { - auto pack_up_cur_task = [&]() { - bool is_trivial_move = compaction->packUpTask(std::move(cur_task)); - is_all_trivial_move = is_all_trivial_move && is_trivial_move; - cur_task = {}; - }; + bool cur_task_full = cur_task.total_rows >= context.delta_small_column_file_rows; + bool small_column_file = t_file->getRows() < context.delta_small_column_file_rows; + bool schema_ok = cur_task.to_compact.empty(); - if (auto * t_file = file->tryToTinyFile(); t_file) + if (!schema_ok) { - bool cur_task_full = cur_task.total_rows >= context.delta_small_column_file_rows; - bool small_column_file = t_file->getRows() < context.delta_small_column_file_rows; - bool schema_ok = cur_task.to_compact.empty(); - - if (!schema_ok) - { - if (auto * last_t_file = cur_task.to_compact.back()->tryToTinyFile(); last_t_file) - schema_ok = t_file->getSchema() == last_t_file->getSchema(); - } - - if (cur_task_full || !small_column_file || !schema_ok) - pack_up_cur_task(); - - cur_task.addColumnFile(file); + if (auto * last_t_file = cur_task.to_compact.back()->tryToTinyFile(); last_t_file) + schema_ok = t_file->getSchema() == last_t_file->getSchema(); } - else - { + + if (cur_task_full || !small_column_file || !schema_ok) pack_up_cur_task(); - cur_task.addColumnFile(file); - } - } - bool is_trivial_move = compaction->packUpTask(std::move(cur_task)); - is_all_trivial_move = is_all_trivial_move && is_trivial_move; - if (!is_all_trivial_move) - return compaction; + cur_task.addColumnFile(file); + } + else + { + pack_up_cur_task(); + cur_task.addColumnFile(file); + } } + pack_up_cur_task(); + + if (!is_all_trivial_move) + return compaction; } return nullptr; } @@ -378,76 +304,45 @@ bool ColumnFilePersistedSet::installCompactionResults(const MinorCompactionPtr & return false; } minor_compaction_version += 1; - LOG_DEBUG(log, "{}, before commit compaction, level info: {}", info(), levelsInfo()); - ColumnFilePersistedLevels new_persisted_files_levels; - auto compaction_src_level = compaction->getCompactionSourceLevel(); - // Copy column files in level range [0, compaction_src_level) - for (size_t i = 0; i < compaction_src_level; i++) + LOG_DEBUG(log, "{}, before commit compaction, persisted column files: {}", info(), detailInfo()); + ColumnFilePersisteds new_persisted_files; + for (const auto & task : compaction->getTasks()) { - auto & new_level = new_persisted_files_levels.emplace_back(); - for (const auto & f : persisted_files_levels[i]) - new_level.push_back(f); + if (task.is_trivial_move) + new_persisted_files.push_back(task.to_compact[0]); + else + new_persisted_files.push_back(task.result); } - // Copy the files in source level that is not in the compaction task. - // Actually, just level 0 may contain file that is not in the compaction task, because flush and compaction can happen concurrently. - // For other levels, we always compact all the files in the level. - // And because compaction is a single threaded process, so there will be no new files compacted to the source level at the same time. - const auto & old_src_level_files = persisted_files_levels[compaction_src_level]; - auto old_src_level_files_iter = old_src_level_files.begin(); + auto old_persisted_files_iter = persisted_files.begin(); for (const auto & task : compaction->getTasks()) { for (const auto & file : task.to_compact) { - if (unlikely(old_src_level_files_iter == old_src_level_files.end() - || (file->getId() != (*old_src_level_files_iter)->getId()) - || (file->getRows() != (*old_src_level_files_iter)->getRows()))) + if (unlikely(old_persisted_files_iter == persisted_files.end() + || (file->getId() != (*old_persisted_files_iter)->getId()) + || (file->getRows() != (*old_persisted_files_iter)->getRows()))) { throw Exception("Compaction algorithm broken", ErrorCodes::LOGICAL_ERROR); } - old_src_level_files_iter++; + old_persisted_files_iter++; } } - auto & src_level_files = new_persisted_files_levels.emplace_back(); - while (old_src_level_files_iter != old_src_level_files.end()) + while (old_persisted_files_iter != persisted_files.end()) { - src_level_files.emplace_back(*old_src_level_files_iter); - old_src_level_files_iter++; - } - // Add new file to the target level - auto target_level = compaction_src_level + 1; - auto & target_level_files = new_persisted_files_levels.emplace_back(); - // Copy the old column files in the target level first if exists - if (persisted_files_levels.size() > target_level) - { - for (auto & column_file : persisted_files_levels[target_level]) - target_level_files.emplace_back(column_file); - } - // Add the compaction result to new target level - for (const auto & task : compaction->getTasks()) - { - if (task.is_trivial_move) - target_level_files.push_back(task.to_compact[0]); - else - target_level_files.push_back(task.result); - } - // Copy column files in level range [target_level + 1, +inf) if exists - for (size_t i = target_level + 1; i < persisted_files_levels.size(); i++) - { - auto & new_level = new_persisted_files_levels.emplace_back(); - for (const auto & f : persisted_files_levels[i]) - new_level.push_back(f); + new_persisted_files.emplace_back(*old_persisted_files_iter); + old_persisted_files_iter++; } - checkColumnFiles(new_persisted_files_levels); + checkColumnFiles(new_persisted_files); /// Save the new metadata of column files to disk. - serializeColumnFilePersistedLevels(wbs, metadata_id, new_persisted_files_levels); + serializeColumnFilePersisteds(wbs, metadata_id, new_persisted_files); wbs.writeMeta(); /// Commit updates in memory. - persisted_files_levels.swap(new_persisted_files_levels); + persisted_files.swap(new_persisted_files); updateColumnFileStats(); - LOG_DEBUG(log, "{}, after commit compaction, level info: {}", info(), levelsInfo()); + LOG_DEBUG(log, "{}, after commit compaction, persisted column files: {}", info(), detailInfo()); return true; } @@ -461,25 +356,20 @@ ColumnFileSetSnapshotPtr ColumnFilePersistedSet::createSnapshot(const StorageSna size_t total_rows = 0; size_t total_deletes = 0; - // The read direction is from the last level to the first level, - // and in each level we read from the begin to the end. - for (auto level_it = persisted_files_levels.rbegin(); level_it != persisted_files_levels.rend(); level_it++) + for (const auto & file : persisted_files) { - for (const auto & file : *level_it) + if (auto * t = file->tryToTinyFile(); (t && t->getCache())) { - if (auto * t = file->tryToTinyFile(); (t && t->getCache())) - { - // Compact threads could update the value of ColumnTinyFile::cache, - // and since ColumnFile is not multi-threads safe, we should create a new column file object. - snap->column_files.push_back(std::make_shared(*t)); - } - else - { - snap->column_files.push_back(file); - } - total_rows += file->getRows(); - total_deletes += file->getDeletes(); + // Compact threads could update the value of ColumnTinyFile::cache, + // and since ColumnFile is not multi-threads safe, we should create a new column file object. + snap->column_files.push_back(std::make_shared(*t)); + } + else + { + snap->column_files.push_back(file); } + total_rows += file->getRows(); + total_deletes += file->getDeletes(); } if (unlikely(total_rows != rows || total_deletes != deletes)) diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.h b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.h index 17fc2ddbd2f..e7796ac55dc 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.h +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.h @@ -49,16 +49,11 @@ using ColumnFilePersistedSetPtr = std::shared_ptr; class ColumnFilePersistedSet : public std::enable_shared_from_this , private boost::noncopyable { -public: - using ColumnFilePersistedLevel = ColumnFilePersisteds; - using ColumnFilePersistedLevels = std::vector; - private: PageId metadata_id; - ColumnFilePersistedLevels persisted_files_levels; + ColumnFilePersisteds persisted_files; // TODO: check the proper memory_order when use this atomic variable std::atomic persisted_files_count = 0; - std::atomic persisted_files_level_count = 0; std::atomic rows = 0; std::atomic bytes = 0; @@ -66,7 +61,6 @@ class ColumnFilePersistedSet : public std::enable_shared_from_this private: Tasks tasks; - size_t compaction_src_level; size_t current_compaction_version; size_t total_compact_files = 0; @@ -62,7 +61,7 @@ class MinorCompaction : public std::enable_shared_from_this size_t result_compact_files = 0; public: - MinorCompaction(size_t compaction_src_level_, size_t current_compaction_version_); + explicit MinorCompaction(size_t current_compaction_version_); // Add new task and return whether this task is a trivial move inline bool packUpTask(Task && task) @@ -90,7 +89,6 @@ class MinorCompaction : public std::enable_shared_from_this const Tasks & getTasks() const { return tasks; } - size_t getCompactionSourceLevel() const { return compaction_src_level; } size_t getCompactionVersion() const { return current_compaction_version; } /// Create new column file by combining several small `ColumnFileTiny`s diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp index 100f2578fe1..d6c2ed3c99f 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp @@ -334,6 +334,10 @@ TEST_F(DeltaValueSpaceTest, MinorCompaction) { delta->appendDeleteRange(dmContext(), RowKeyRange::fromHandleRange(HandleRange(0, num_rows_write_per_batch))); } + { + appendBlockToDeltaValueSpace(dmContext(), delta, total_rows_write, num_rows_write_per_batch); + total_rows_write += num_rows_write_per_batch; + } delta->flush(dmContext()); } // build compaction task and finish prepare stage @@ -341,18 +345,21 @@ TEST_F(DeltaValueSpaceTest, MinorCompaction) { PageReader reader = dmContext().storage_pool.newLogReader(dmContext().getReadLimiter(), true, ""); compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); - ASSERT_EQ(compaction_task->getCompactionSourceLevel(), 0); - // There should be two compaction sub_tasks. - // The first task try to compact the first three column files to a larger one, - // and the second task is just a trivial move for the last column file which is a delete range. + // There should be three compaction sub_tasks. + // The first task try to compact the first three column files to a larger one. + // The second task is a trivial move for a ColumnFileDeleteRange. + // The third task is a trivial move for and a ColumnFileTiny. const auto & tasks = compaction_task->getTasks(); - ASSERT_EQ(tasks.size(), 2); + ASSERT_EQ(tasks.size(), 3); ASSERT_EQ(tasks[0].to_compact.size(), 3); ASSERT_EQ(tasks[0].is_trivial_move, false); ASSERT_EQ(tasks[1].to_compact.size(), 1); ASSERT_EQ(tasks[1].is_trivial_move, true); + ASSERT_EQ(tasks[2].to_compact.size(), 1); + ASSERT_EQ(tasks[2].is_trivial_move, true); compaction_task->prepare(dmContext(), wbs, reader); } + // another thread write more data to the delta value space and flush it { appendBlockToDeltaValueSpace(dmContext(), delta, total_rows_write, num_rows_write_per_batch); @@ -361,18 +368,27 @@ TEST_F(DeltaValueSpaceTest, MinorCompaction) ASSERT_EQ(delta->getUnsavedRows(), 0); ASSERT_EQ(persisted_file_set->getRows(), total_rows_write); ASSERT_EQ(persisted_file_set->getDeletes(), 1); - ASSERT_EQ(persisted_file_set->getColumnFileCount(), 5); + ASSERT_EQ(persisted_file_set->getColumnFileCount(), 6); } // commit the compaction task and check the status { + ASSERT_TRUE(compaction_task->commit(persisted_file_set, wbs)); + ASSERT_EQ(persisted_file_set->getRows(), total_rows_write); + ASSERT_EQ(persisted_file_set->getDeletes(), 1); + ASSERT_EQ(persisted_file_set->getColumnFileCount(), 4); + } + // now the column files in persisted_file_set should be: T_300, D_0_100, T_100, T_100 + { + compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); + PageReader reader = dmContext().storage_pool.newLogReader(dmContext().getReadLimiter(), true, ""); + compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); + compaction_task->prepare(dmContext(), wbs, reader); ASSERT_TRUE(compaction_task->commit(persisted_file_set, wbs)); ASSERT_EQ(persisted_file_set->getRows(), total_rows_write); ASSERT_EQ(persisted_file_set->getDeletes(), 1); ASSERT_EQ(persisted_file_set->getColumnFileCount(), 3); } - // after compaction, the column file in persisted_file_set should be like the following: - // level 0: T_100 - // level 1: T_300, D_0_100 + // now the column files in persisted_file_set should be: T_300, D_0_100, T_200 // so there is no compaction task to do { compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); @@ -431,7 +447,6 @@ TEST_F(DeltaValueSpaceTest, Restore) total_rows_write += num_rows_write_per_batch; } delta->flush(dmContext()); - ASSERT_EQ(persisted_file_set->getColumnFileLevelCount(), 2); ASSERT_EQ(delta->getColumnFileCount(), 3); ASSERT_EQ(delta->getRows(), total_rows_write); } @@ -475,9 +490,7 @@ TEST_F(DeltaValueSpaceTest, CloneNewlyAppendedColumnFiles) auto persisted_file_set = delta->getPersistedFileSet(); size_t total_rows_write = 0; WriteBatches wbs(dmContext().storage_pool, dmContext().getWriteLimiter()); - // create three levels in persisted_file_set { - // one column file in level 1 { appendBlockToDeltaValueSpace(dmContext(), delta, total_rows_write, num_rows_write_per_batch); total_rows_write += num_rows_write_per_batch; @@ -489,8 +502,6 @@ TEST_F(DeltaValueSpaceTest, CloneNewlyAppendedColumnFiles) delta->flush(dmContext()); delta->compact(dmContext()); ASSERT_EQ(delta->getColumnFileCount(), 1); - ASSERT_EQ(persisted_file_set->getColumnFileLevelCount(), 2); - // one column files in level 2 { appendBlockToDeltaValueSpace(dmContext(), delta, total_rows_write, num_rows_write_per_batch); total_rows_write += num_rows_write_per_batch; @@ -500,13 +511,8 @@ TEST_F(DeltaValueSpaceTest, CloneNewlyAppendedColumnFiles) total_rows_write += num_rows_write_per_batch; } delta->flush(dmContext()); - // compact two level 0 files to level 1 - delta->compact(dmContext()); - // compact two level 1 files to level 2 delta->compact(dmContext()); ASSERT_EQ(delta->getColumnFileCount(), 1); - ASSERT_EQ(persisted_file_set->getColumnFileLevelCount(), 3); - // one column files in level 1 and one column files in level 2 { appendBlockToDeltaValueSpace(dmContext(), delta, total_rows_write, num_rows_write_per_batch); total_rows_write += num_rows_write_per_batch; @@ -517,8 +523,7 @@ TEST_F(DeltaValueSpaceTest, CloneNewlyAppendedColumnFiles) } delta->flush(dmContext()); delta->compact(dmContext()); - ASSERT_EQ(delta->getColumnFileCount(), 2); - ASSERT_EQ(persisted_file_set->getColumnFileLevelCount(), 3); + ASSERT_EQ(delta->getColumnFileCount(), 1); } { auto snapshot = delta->createSnapshot(dmContext(), true, CurrentMetrics::DT_SnapshotOfRead); From 165e23e33d5f3a47b62eb6d8048f37052eea086e Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Wed, 7 Dec 2022 15:42:04 +0800 Subject: [PATCH 07/42] Throw when meet empty PK (#6392) close pingcap/tiflash#6391 --- dbms/src/Common/RedactHelpers.cpp | 17 ++++++ dbms/src/Common/RedactHelpers.h | 1 + dbms/src/Debug/MockRaftStoreProxy.cpp | 57 +++++++++++++++++-- dbms/src/Debug/MockRaftStoreProxy.h | 23 +++++--- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 2 +- dbms/src/Storages/Transaction/RegionData.cpp | 6 +- .../Transaction/tests/gtest_new_kvstore.cpp | 35 ++++++++++++ 7 files changed, 124 insertions(+), 17 deletions(-) diff --git a/dbms/src/Common/RedactHelpers.cpp b/dbms/src/Common/RedactHelpers.cpp index 7a8a3eef172..39af6aa08f3 100644 --- a/dbms/src/Common/RedactHelpers.cpp +++ b/dbms/src/Common/RedactHelpers.cpp @@ -76,3 +76,20 @@ void Redact::keyToDebugString(const char * key, const size_t size, std::ostream } oss.flags(flags); // restore flags } + +std::string Redact::hexStringToKey(const char * start, size_t len) +{ + std::string s; + if (len & 1) + throw DB::Exception("Invalid length: " + std::string(start, len), DB::ErrorCodes::LOGICAL_ERROR); + + for (size_t i = 0; i < len; i += 2) + { + int x; + std::stringstream ss; + ss << std::hex << std::string(start + i, start + i + 2); + ss >> x; + s.push_back(x); + } + return s; +} \ No newline at end of file diff --git a/dbms/src/Common/RedactHelpers.h b/dbms/src/Common/RedactHelpers.h index 13125646586..b1a681c80dc 100644 --- a/dbms/src/Common/RedactHelpers.h +++ b/dbms/src/Common/RedactHelpers.h @@ -35,6 +35,7 @@ class Redact static std::string keyToHexString(const char * key, size_t size); static void keyToDebugString(const char * key, size_t size, std::ostream & oss); + static std::string hexStringToKey(const char * start, size_t len); friend class DB::FieldVisitorToDebugString; diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 3939f0fc6bc..727b2a2d4c7 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -369,9 +369,53 @@ std::tuple MockRaftStoreProxy::normalWrite( // The new entry is committed on Proxy's side. region->updateCommitIndex(index); // We record them, as persisted raft log, for potential recovery. + std::vector new_keys; + for (size_t i = 0; i < cmd_types.size(); i++) + { + if (cmd_types[i] == WriteCmdType::Put) + { + auto cf_name = CFToName(cmd_cf[i]); + new_keys.emplace_back(RecordKVFormat::genKey(table_id, keys[i], 1)); + } + else + { + auto cf_name = CFToName(cmd_cf[i]); + new_keys.emplace_back(RecordKVFormat::genKey(table_id, keys[i], 1)); + } + } region->commands[index] = { term, - MockProxyRegion::NormalWrite{ + MockProxyRegion::RawWrite{ + new_keys, + vals, + cmd_types, + cmd_cf, + }}; + } + return std::make_tuple(index, term); +} + +std::tuple MockRaftStoreProxy::rawWrite( + UInt64 region_id, + std::vector && keys, + std::vector && vals, + std::vector && cmd_types, + std::vector && cmd_cf) +{ + uint64_t index = 0; + uint64_t term = 0; + { + auto region = getRegion(region_id); + assert(region != nullptr); + // We have a new entry. + index = region->getLatestCommitIndex() + 1; + term = region->getLatestCommitTerm(); + // The new entry is committed on Proxy's side. + region->updateCommitIndex(index); + // We record them, as persisted raft log, for potential recovery. + region->commands[index] = { + term, + MockProxyRegion::RawWrite{ keys, vals, cmd_types, @@ -381,6 +425,7 @@ std::tuple MockRaftStoreProxy::normalWrite( return std::make_tuple(index, term); } + std::tuple MockRaftStoreProxy::compactLog(UInt64 region_id, UInt64 compact_index) { uint64_t index = 0; @@ -427,9 +472,9 @@ void MockRaftStoreProxy::doApply( raft_cmdpb::RaftCmdRequest request; auto & cmd = region->commands[index]; auto term = cmd.term; - if (cmd.has_write_request()) + if (cmd.has_raw_write_request()) { - auto & c = cmd.write(); + auto & c = cmd.raw_write(); auto & keys = c.keys; auto & vals = c.vals; auto & cmd_types = c.cmd_types; @@ -444,14 +489,14 @@ void MockRaftStoreProxy::doApply( if (cmd_types[i] == WriteCmdType::Put) { auto cf_name = CFToName(cmd_cf[i]); - auto key = RecordKVFormat::genKey(table_id, keys[i], 1); + auto key = TiKVKey(keys[i].data(), keys[i].size()); TiKVValue value = std::move(vals[i]); RegionBench::setupPutRequest(request.add_requests(), cf_name, key, value); } else { auto cf_name = CFToName(cmd_cf[i]); - auto key = RecordKVFormat::genKey(table_id, keys[i], 1); + auto key = TiKVKey(keys[i].data(), keys[i].size()); RegionBench::setupDelRequest(request.add_requests(), cf_name, key); } } @@ -465,7 +510,7 @@ void MockRaftStoreProxy::doApply( auto old_applied = kvs.getRegion(region_id)->appliedIndex(); auto old_applied_term = kvs.getRegion(region_id)->appliedIndexTerm(); - if (cmd.has_write_request()) + if (cmd.has_raw_write_request()) { // TiFlash write kvs.handleWriteRaftCmd(std::move(request), region_id, index, term, tmt); diff --git a/dbms/src/Debug/MockRaftStoreProxy.h b/dbms/src/Debug/MockRaftStoreProxy.h index 423372b7a0e..6207e7e8b17 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.h +++ b/dbms/src/Debug/MockRaftStoreProxy.h @@ -36,15 +36,13 @@ struct MockProxyRegion : MutexLockWrap void updateCommitIndex(uint64_t index); void setSate(raft_serverpb::RegionLocalState); explicit MockProxyRegion(uint64_t id); - - struct NormalWrite + struct RawWrite { - std::vector keys; + std::vector keys; std::vector vals; std::vector cmd_types; std::vector cmd_cf; }; - struct AdminCommand { raft_cmdpb::AdminRequest request; @@ -58,16 +56,16 @@ struct MockProxyRegion : MutexLockWrap struct CachedCommand { uint64_t term; - std::variant inner; + std::variant inner; bool has_admin_request() const { return std::holds_alternative(inner); } - bool has_write_request() const + bool has_raw_write_request() const { - return std::holds_alternative(inner); + return std::holds_alternative(inner); } AdminCommand & admin() @@ -75,9 +73,9 @@ struct MockProxyRegion : MutexLockWrap return std::get(inner); } - NormalWrite & write() + RawWrite & raw_write() { - return std::get(inner); + return std::get(inner); } }; @@ -190,6 +188,13 @@ struct MockRaftStoreProxy : MutexLockWrap std::vector && cmd_types, std::vector && cmd_cf); + std::tuple rawWrite( + UInt64 region_id, + std::vector && keys, + std::vector && vals, + std::vector && cmd_types, + std::vector && cmd_cf); + /// Create a compactLog admin command, returns (index, term) of the admin command itself. std::tuple compactLog(UInt64 region_id, UInt64 compact_index); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 809878b7a30..7b1049225d1 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -561,7 +561,7 @@ void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_ // to fit the segment. auto [cur_offset, cur_limit] = rowkey_range.getPosRange(handle_column, offset, rows - offset); if (unlikely(cur_offset != offset)) - throw Exception("cur_offset does not equal to offset", ErrorCodes::LOGICAL_ERROR); + throw Exception(fmt::format("cur_offset does not equal to offset. is_common_handle {} start_key {} cur_offset {} cur_limit {} rows {} offset {} rowkey_range {}", is_common_handle, start_key.toRowKeyValue().toString(), cur_offset, cur_limit, rows, offset, rowkey_range.toDebugString()), ErrorCodes::LOGICAL_ERROR); limit = cur_limit; auto alloc_bytes = block.bytes(offset, limit); diff --git a/dbms/src/Storages/Transaction/RegionData.cpp b/dbms/src/Storages/Transaction/RegionData.cpp index 26bd2e93fdf..c27dcb82efa 100644 --- a/dbms/src/Storages/Transaction/RegionData.cpp +++ b/dbms/src/Storages/Transaction/RegionData.cpp @@ -115,7 +115,11 @@ RegionDataReadInfo RegionData::readDataByWriteIt(const ConstWriteCFIter & write_ const auto & [pk, ts] = write_it->first; std::ignore = value; - std::ignore = value; + + if (pk->empty()) + { + throw Exception("Observe empty PK: raw key " + key->toDebugString(), ErrorCodes::ILLFORMAT_RAFT_ROW); + } if (!need_value) return std::make_tuple(pk, decoded_val.write_type, ts, nullptr); diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index f3ffc3e8346..36361db53bf 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -154,6 +154,41 @@ TEST_F(RegionKVStoreTest, KVStoreFailRecovery) } } +TEST_F(RegionKVStoreTest, KVStoreInvalidWrites) +{ + auto ctx = TiFlashTestEnv::getGlobalContext(); + { + auto region_id = 1; + { + initStorages(); + KVStore & kvs = getKVS(); + proxy_instance->bootstrap_table(ctx, kvs, ctx.getTMTContext()); + proxy_instance->bootstrap(kvs, ctx.getTMTContext(), region_id); + + MockRaftStoreProxy::FailCond cond; + + auto kvr1 = kvs.getRegion(region_id); + auto r1 = proxy_instance->getRegion(region_id); + ASSERT_NE(r1, nullptr); + ASSERT_NE(kvr1, nullptr); + ASSERT_EQ(r1->getLatestAppliedIndex(), kvr1->appliedIndex()); + { + r1->getLatestAppliedIndex(); + // This key has empty PK which is actually truncated. + std::string k = "7480000000000001FFBD5F720000000000FAF9ECEFDC3207FFFC"; + std::string v = "4486809092ACFEC38906"; + auto strKey = Redact::hexStringToKey(k.data(), k.size()); + auto strVal = Redact::hexStringToKey(v.data(), v.size()); + + auto [index, term] = proxy_instance->rawWrite(region_id, {strKey}, {strVal}, {WriteCmdType::Put}, {ColumnFamilyType::Write}); + EXPECT_THROW(proxy_instance->doApply(kvs, ctx.getTMTContext(), cond, region_id, index), Exception); + UNUSED(term); + EXPECT_THROW(ReadRegionCommitCache(kvr1, true), Exception); + } + } + } +} + TEST_F(RegionKVStoreTest, KVStoreAdminCommands) { auto ctx = TiFlashTestEnv::getGlobalContext(); From 98988131be16a22ac816c06378bf786d12340144 Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Thu, 8 Dec 2022 12:18:05 +0800 Subject: [PATCH 08/42] limit the queued task number and queued duration of coprocess task. (#6394) ref pingcap/tiflash#6438 --- contrib/client-c | 2 +- dbms/src/Flash/FlashService.cpp | 24 ++++++++++++++++++++++++ dbms/src/Interpreters/Settings.h | 2 ++ 3 files changed, 27 insertions(+), 1 deletion(-) diff --git a/contrib/client-c b/contrib/client-c index 9563b2d37f4..425148c0392 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 9563b2d37f410f356ecc3f9af7ad666a79839c34 +Subproject commit 425148c03929f4c960e50ed2c877f6695b105278 diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index cb62855d6c1..8ab12d5b906 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -137,7 +137,31 @@ grpc::Status FlashService::Coprocessor( context->setMockStorage(mock_storage); + const auto & settings = context->getSettingsRef(); + auto handle_limit = settings.cop_pool_handle_limit != 0 ? settings.cop_pool_handle_limit.get() : 10 * cop_pool->size(); + auto max_queued_duration_seconds = std::min(settings.cop_pool_max_queued_seconds, 20); + + if (handle_limit > 0) + { + // We use this atomic variable metrics from the prometheus-cpp library to mark the number of queued queries. + // TODO: Use grpc asynchronous server and a more fully-featured thread pool. + if (auto current = GET_METRIC(tiflash_coprocessor_handling_request_count, type_cop).Value(); current > handle_limit) + { + response->mutable_region_error()->mutable_server_is_busy()->set_reason(fmt::format("tiflash cop pool queued too much, current = {}, limit = {}", current, handle_limit)); + return grpc::Status::OK; + } + } + + grpc::Status ret = executeInThreadPool(*cop_pool, [&] { + if (max_queued_duration_seconds > 0) + { + if (auto current = watch.elapsedSeconds(); current > max_queued_duration_seconds) + { + response->mutable_region_error()->mutable_server_is_busy()->set_reason(fmt::format("this task queued in tiflash cop pool too long, current = {}, limit = {}", current, max_queued_duration_seconds)); + return grpc::Status::OK; + } + } auto [db_context, status] = createDBContext(grpc_context); if (!status.ok()) { diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 32bb62375ec..ea6d1c89cd7 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -65,6 +65,8 @@ struct Settings M(SettingUInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.") \ M(SettingMaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.") \ M(SettingUInt64, cop_pool_size, 0, "The number of threads to handle cop requests. By default, it is determined automatically.") \ + M(SettingInt64, cop_pool_handle_limit, 0, "The maximum number of requests can be handled by cop pool, include executing and queuing tasks. More cop requests will get error \"TiFlash Server is Busy\". -1 means unlimited, 0 means determined automatically (10 times of cop-pool-size).") \ + M(SettingInt64, cop_pool_max_queued_seconds, 15, "The maximum queuing duration of coprocessor task, unit is second. When task starts to run, it checks whether queued more than this config, if so, it will directly return error \"TiFlash Server is Busy\". <=0 means unlimited, default is 15. The upper limit of this config is 20.") \ M(SettingUInt64, batch_cop_pool_size, 0, "The number of threads to handle batch cop requests. By default, it is determined automatically.") \ M(SettingUInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.") \ M(SettingUInt64, max_distributed_connections, DEFAULT_MAX_DISTRIBUTED_CONNECTIONS, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).") \ From 966e7e228ee70e4392139d513529fc8f330d61ce Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 8 Dec 2022 15:30:04 +0800 Subject: [PATCH 09/42] Get correct cpu cores in k8s pod (#6430) close pingcap/tiflash#6434 --- .../src/Common/getNumberOfLogicalCPUCores.cpp | 34 ++++++++++ ...PUCores.h => getNumberOfLogicalCPUCores.h} | 13 +++- .../Common/getNumberOfPhysicalCPUCores.cpp | 65 ------------------- dbms/src/Flash/FlashService.cpp | 3 +- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 3 +- dbms/src/Interpreters/SettingsCommon.h | 13 +--- dbms/src/Server/Server.cpp | 4 +- dbms/src/Server/ServerInfo.h | 3 +- 8 files changed, 55 insertions(+), 83 deletions(-) create mode 100644 dbms/src/Common/getNumberOfLogicalCPUCores.cpp rename dbms/src/Common/{getNumberOfPhysicalCPUCores.h => getNumberOfLogicalCPUCores.h} (66%) delete mode 100644 dbms/src/Common/getNumberOfPhysicalCPUCores.cpp diff --git a/dbms/src/Common/getNumberOfLogicalCPUCores.cpp b/dbms/src/Common/getNumberOfLogicalCPUCores.cpp new file mode 100644 index 00000000000..16854909636 --- /dev/null +++ b/dbms/src/Common/getNumberOfLogicalCPUCores.cpp @@ -0,0 +1,34 @@ +// 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 + +namespace CPUCores +{ +UInt16 number_of_logical_cpu_cores = std::thread::hardware_concurrency(); +} // namespace CPUCores + + +UInt16 getNumberOfLogicalCPUCores() +{ + return CPUCores::number_of_logical_cpu_cores; +} + +// We should call this function before Context has been created, +// which will call `getNumberOfLogicalCPUCores`, or we can not +// set cpu cores any more. +void setNumberOfLogicalCPUCores(UInt16 max_logical_cpu_cores) +{ + CPUCores::number_of_logical_cpu_cores = max_logical_cpu_cores; +} diff --git a/dbms/src/Common/getNumberOfPhysicalCPUCores.h b/dbms/src/Common/getNumberOfLogicalCPUCores.h similarity index 66% rename from dbms/src/Common/getNumberOfPhysicalCPUCores.h rename to dbms/src/Common/getNumberOfLogicalCPUCores.h index b3ab65a66e5..d50d13c2596 100644 --- a/dbms/src/Common/getNumberOfPhysicalCPUCores.h +++ b/dbms/src/Common/getNumberOfLogicalCPUCores.h @@ -14,6 +14,13 @@ #pragma once -/// Get number of CPU cores without hyper-threading. -/// Note: do not support environment under resource isolation mechanism like Docker, CGroup. -unsigned getNumberOfPhysicalCPUCores(); +#include + +#include + +UInt16 getNumberOfLogicalCPUCores(); + +// We should call this function before Context has been created, +// which will call `getNumberOfLogicalCPUCores`, or we can not +// set cpu cores any more. +void setNumberOfLogicalCPUCores(UInt16 max_logical_cpu_cores); diff --git a/dbms/src/Common/getNumberOfPhysicalCPUCores.cpp b/dbms/src/Common/getNumberOfPhysicalCPUCores.cpp deleted file mode 100644 index de98f4faa10..00000000000 --- a/dbms/src/Common/getNumberOfPhysicalCPUCores.cpp +++ /dev/null @@ -1,65 +0,0 @@ -// 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 - -#if defined(__x86_64__) -#include -#endif - -namespace DB -{ -namespace ErrorCodes -{ -extern const int CPUID_ERROR; -} -} // namespace DB - -unsigned getNumberOfPhysicalCPUCores() -{ - unsigned res = 0; -#if defined(__x86_64__) - - cpu_raw_data_t raw_data; - cpu_id_t data; - if (0 == cpuid_get_raw_data(&raw_data) && 0 == cpu_identify(&raw_data, &data) && data.num_logical_cpus != 0) - { - res = data.num_cores * data.total_logical_cpus / data.num_logical_cpus; - } - - /// Also, libcpuid gives strange result on Google Compute Engine VMs. - /// Example: - /// num_cores = 12, /// number of physical cores on current CPU socket - /// total_logical_cpus = 1, /// total number of logical cores on all sockets - /// num_logical_cpus = 24. /// number of logical cores on current CPU socket - /// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1. - if (res != 0) - { - return res; - } - // else fallback -#endif - - /// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system. - /// (Actually, only Aarch64 is supported). - res = std::thread::hardware_concurrency(); - if (res == 0) - { - throw DB::Exception("Cannot Identify CPU: Unsupported processor", DB::ErrorCodes::CPUID_ERROR); - } - return res; -} diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 8ab12d5b906..ed13fcfe1c9 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -74,7 +75,7 @@ void FlashService::init(const TiFlashSecurityConfig & security_config_, Context auto settings = context->getSettingsRef(); enable_local_tunnel = settings.enable_local_tunnel; enable_async_grpc_client = settings.enable_async_grpc_client; - const size_t default_size = 2 * getNumberOfPhysicalCPUCores(); + const size_t default_size = getNumberOfLogicalCPUCores(); auto cop_pool_size = static_cast(settings.cop_pool_size); cop_pool_size = cop_pool_size ? cop_pool_size : default_size; diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 32816d90cd3..8107a9e3712 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include @@ -35,7 +36,7 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac , active_set_soft_limit(active_set_soft_limit_) , log(Logger::get()) { - auto cores = getNumberOfPhysicalCPUCores(); + auto cores = static_cast(getNumberOfLogicalCPUCores() / 2); if (active_set_soft_limit == 0 || active_set_soft_limit > 10 * cores) { /// set active_set_soft_limit to a reasonable value diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index cdc7b1b6fbd..4510159da57 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -16,7 +16,7 @@ #include #include -#include +#include #include #include #include @@ -26,7 +26,6 @@ #include #include - namespace DB { namespace ErrorCodes @@ -166,18 +165,12 @@ struct SettingMaxThreads is_auto = true; } - UInt64 getAutoValue() const + static UInt64 getAutoValue() { - static auto res = getAutoValueImpl(); + static auto res = getNumberOfLogicalCPUCores(); return res; } - /// Executed once for all time. Executed from one thread. - UInt64 getAutoValueImpl() const - { - return getNumberOfPhysicalCPUCores(); - } - UInt64 get() const { return value; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 7a0368d0beb..5c729c7084d 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -32,7 +32,7 @@ #include #include #include -#include +#include #include #include #include @@ -882,10 +882,12 @@ int Server::main(const std::vector & /*args*/) auto * helper = tiflash_instance_wrap.proxy_helper; helper->fn_server_info(helper->proxy_ptr, strIntoView(&req), &response); server_info.parseSysInfo(response); + setNumberOfLogicalCPUCores(server_info.cpu_info.logical_cores); LOG_INFO(log, "ServerInfo: {}", server_info.debugString()); } else { + setNumberOfLogicalCPUCores(std::thread::hardware_concurrency()); LOG_INFO(log, "TiFlashRaftProxyHelper is null, failed to get server info"); } diff --git a/dbms/src/Server/ServerInfo.h b/dbms/src/Server/ServerInfo.h index 9663bd37568..d9731e37bb8 100644 --- a/dbms/src/Server/ServerInfo.h +++ b/dbms/src/Server/ServerInfo.h @@ -13,7 +13,6 @@ // limitations under the License. #pragma once -#include #include #include @@ -38,7 +37,7 @@ class ServerInfo /// number of logical CPU cores UInt16 logical_cores = std::thread::hardware_concurrency(); /// number of physical CPU cores - UInt16 physical_cores = getNumberOfPhysicalCPUCores(); + UInt16 physical_cores = std::thread::hardware_concurrency() / 2; /// number of L1 cache size /// units: Byte UInt32 l1_cache_size = 16384; // 16KB (typical value) From 1a69ba8e5c4599ab0ba3ec5fdc3cebb5e8c437b1 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Thu, 8 Dec 2022 20:40:05 +0800 Subject: [PATCH 10/42] *: Refine TiDBTableScan (#6448) ref pingcap/tiflash#6233 --- dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp | 6 +++--- dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h | 2 +- dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp | 6 +++--- dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp | 2 +- dbms/src/Flash/Coprocessor/RemoteRequest.cpp | 2 +- dbms/src/Flash/Coprocessor/TiDBTableScan.cpp | 2 +- dbms/src/Flash/Coprocessor/TiDBTableScan.h | 7 +++++-- dbms/src/Storages/Transaction/TiDB.cpp | 9 +++++++++ dbms/src/Storages/Transaction/TiDB.h | 1 + 9 files changed, 25 insertions(+), 12 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 48774f0c457..498eb0e0a89 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -842,7 +842,7 @@ String DAGExpressionAnalyzer::appendTimeZoneCast( bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( const ExpressionActionsPtr & actions, const std::vector & need_cast_column, - const ::google::protobuf::RepeatedPtrField & table_scan_columns) + const ColumnInfos & table_scan_columns) { bool has_cast = false; @@ -867,9 +867,9 @@ bool DAGExpressionAnalyzer::buildExtraCastsAfterTS( if (need_cast_column[i] == ExtraCastAfterTSMode::AppendDurationCast) { - if (table_scan_columns[i].decimal() > 6) + if (table_scan_columns[i].decimal > 6) throw Exception("fsp must <= 6", ErrorCodes::LOGICAL_ERROR); - auto fsp = table_scan_columns[i].decimal() < 0 ? 0 : table_scan_columns[i].decimal(); + const auto fsp = table_scan_columns[i].decimal < 0 ? 0 : table_scan_columns[i].decimal; tipb::Expr fsp_expr = constructInt64LiteralTiExpr(fsp); fsp_col = getActions(fsp_expr, actions); String casted_name = appendDurationCast(fsp_col, source_columns[i].name, dur_func_name, actions); diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index f0d64b2232e..13e751c4b85 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -281,7 +281,7 @@ class DAGExpressionAnalyzer : private boost::noncopyable bool buildExtraCastsAfterTS( const ExpressionActionsPtr & actions, const std::vector & need_cast_column, - const ::google::protobuf::RepeatedPtrField & table_scan_columns); + const ColumnInfos & table_scan_columns); std::pair buildJoinKey( const ExpressionActionsPtr & actions, diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 4c8adefebc3..4f6ed972235 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -1041,7 +1041,7 @@ std::tuple> DAGStorageIn for (Int32 i = 0; i < table_scan.getColumnSize(); ++i) { auto const & ci = table_scan.getColumns()[i]; - ColumnID cid = ci.column_id(); + const ColumnID cid = ci.id; // Column ID -1 return the handle column String name; @@ -1062,9 +1062,9 @@ std::tuple> DAGStorageIn source_columns_tmp.emplace_back(std::move(pair)); } required_columns_tmp.emplace_back(std::move(name)); - if (cid != -1 && ci.tp() == TiDB::TypeTimestamp) + if (cid != -1 && ci.tp == TiDB::TypeTimestamp) need_cast_column.push_back(ExtraCastAfterTSMode::AppendTimeZoneCast); - else if (cid != -1 && ci.tp() == TiDB::TypeTime) + else if (cid != -1 && ci.tp == TiDB::TypeTime) need_cast_column.push_back(ExtraCastAfterTSMode::AppendDurationCast); else need_cast_column.push_back(ExtraCastAfterTSMode::None); diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp index f21c74dbfaa..f638190b47c 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -40,7 +40,7 @@ NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan, const StringRef names_and_types.reserve(table_scan.getColumnSize()); for (Int32 i = 0; i < table_scan.getColumnSize(); ++i) { - auto column_info = TiDB::toTiDBColumnInfo(table_scan.getColumns()[i]); + const auto column_info = table_scan.getColumns()[i]; switch (column_info.id) { case TiDBPkColumnID: diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp index 2ecedee82fe..e0111641a4f 100644 --- a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp @@ -59,7 +59,7 @@ RemoteRequest RemoteRequest::build( for (int i = 0; i < table_scan.getColumnSize(); ++i) { const auto & col = table_scan.getColumns()[i]; - auto col_id = col.column_id(); + auto col_id = col.id; if (col_id == DB::TiDBPkColumnID) { diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp index 20a12770c73..97af20992be 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp @@ -23,7 +23,7 @@ TiDBTableScan::TiDBTableScan( : table_scan(table_scan_) , executor_id(executor_id_) , is_partition_table_scan(table_scan->tp() == tipb::TypePartitionTableScan) - , columns(is_partition_table_scan ? table_scan->partition_table_scan().columns() : table_scan->tbl_scan().columns()) + , columns(is_partition_table_scan ? std::move(TiDB::toTiDBColumnInfos(table_scan->partition_table_scan().columns())) : std::move(TiDB::toTiDBColumnInfos(table_scan->tbl_scan().columns()))) // Only No-partition table need keep order when tablescan executor required keep order. // If keep_order is not set, keep order for safety. , keep_order(!is_partition_table_scan && (table_scan->tbl_scan().keep_order() || !table_scan->tbl_scan().has_keep_order())) diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.h b/dbms/src/Flash/Coprocessor/TiDBTableScan.h index 574d4b0a15f..84e6f41581f 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.h +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.h @@ -18,6 +18,9 @@ namespace DB { + +using ColumnInfos = std::vector; + /// TiDBTableScan is a wrap to hide the difference of `TableScan` and `PartitionTableScan` class TiDBTableScan { @@ -34,7 +37,7 @@ class TiDBTableScan { return columns.size(); } - const google::protobuf::RepeatedPtrField & getColumns() const + const ColumnInfos & getColumns() const { return columns; } @@ -65,7 +68,7 @@ class TiDBTableScan const tipb::Executor * table_scan; String executor_id; bool is_partition_table_scan; - const google::protobuf::RepeatedPtrField & columns; + const ColumnInfos columns; /// logical_table_id is the table id for a TiDB' table, while if the /// TiDB table is partition, each partition is a physical table, and /// the partition's table id is the physical table id. diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index 2f9790bcb37..745839a2476 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -1143,4 +1143,13 @@ ColumnInfo toTiDBColumnInfo(const tipb::ColumnInfo & tipb_column_info) return tidb_column_info; } +std::vector toTiDBColumnInfos(const ::google::protobuf::RepeatedPtrField & tipb_column_infos) +{ + std::vector tidb_column_infos; + tidb_column_infos.reserve(tipb_column_infos.size()); + for (const auto & tipb_column_info : tipb_column_infos) + tidb_column_infos.emplace_back(toTiDBColumnInfo(tipb_column_info)); + return tidb_column_infos; +} + } // namespace TiDB diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index 9bd78abeed3..cd428e57e6e 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -423,5 +423,6 @@ String genJsonNull(); tipb::FieldType columnInfoToFieldType(const ColumnInfo & ci); ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type); ColumnInfo toTiDBColumnInfo(const tipb::ColumnInfo & tipb_column_info); +std::vector toTiDBColumnInfos(const ::google::protobuf::RepeatedPtrField & tipb_column_infos); } // namespace TiDB From f248fac2bffd2416a8c56206ed37783b77d28093 Mon Sep 17 00:00:00 2001 From: JaySon Date: Thu, 8 Dec 2022 22:00:05 +0800 Subject: [PATCH 11/42] PageStorage: background version compact for v2 (#6446) close pingcap/tiflash#6407 --- dbms/src/Common/CurrentMetrics.cpp | 2 + dbms/src/Common/ProfileEvents.cpp | 1 + dbms/src/Common/TiFlashMetrics.h | 5 +- dbms/src/Interpreters/Context.cpp | 10 + dbms/src/Interpreters/Context.h | 1 + .../DeltaMerge/Delta/DeltaValueSpace.cpp | 9 +- dbms/src/Storages/DeltaMerge/StoragePool.cpp | 26 +- dbms/src/Storages/Page/PageStorage.cpp | 4 +- dbms/src/Storages/Page/PageStorage.h | 3 + dbms/src/Storages/Page/V2/PageEntries.h | 34 ++- dbms/src/Storages/Page/V2/PageStorage.cpp | 37 ++- dbms/src/Storages/Page/V2/PageStorage.h | 12 + .../PageEntriesVersionSetWithDelta.cpp | 44 ++-- .../PageEntriesVersionSetWithDelta.h | 57 +++- .../Storages/Page/V2/gc/LegacyCompactor.cpp | 2 +- .../Page/V2/tests/gtest_data_compactor.cpp | 6 +- .../Page/V2/tests/gtest_legacy_compactor.cpp | 17 +- .../V2/tests/gtest_page_map_version_set.cpp | 205 ++++++++------- .../Page/V2/tests/gtest_page_storage.cpp | 12 +- .../tests/gtest_page_storage_multi_paths.cpp | 21 +- .../gtest_page_storage_multi_writers.cpp | 28 +- .../Page/tools/PageCtl/PageStorageCtlV2.cpp | 4 +- .../src/Storages/Page/workload/PSWorkload.cpp | 3 +- dbms/src/Storages/Page/workload/PSWorkload.h | 2 + .../Storages/Transaction/RegionPersister.cpp | 6 +- metrics/grafana/tiflash_proxy_details.json | 244 +++++++++++++++++- 26 files changed, 604 insertions(+), 191 deletions(-) diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index a69491d1329..4073f80f2db 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -26,6 +26,8 @@ M(MemoryCapacity) \ M(PSMVCCNumSnapshots) \ M(PSMVCCSnapshotsList) \ + M(PSMVCCNumDelta) \ + M(PSMVCCNumBase) \ M(RWLockWaitingReaders) \ M(RWLockWaitingWriters) \ M(RWLockActiveReaders) \ diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 83035b93699..5944d83fafb 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -66,6 +66,7 @@ M(PSMVCCCompactOnDelta) \ M(PSMVCCCompactOnDeltaRebaseRejected) \ M(PSMVCCCompactOnBase) \ + M(PSMVCCCompactOnBaseCommit) \ \ M(DMWriteBlock) \ M(DMWriteBlockNS) \ diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 2ef8fd6cb01..4c27a0f3025 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -124,7 +124,7 @@ namespace DB F(type_seg_split_bg, {"type", "seg_split_bg"}), \ F(type_seg_split_fg, {"type", "seg_split_fg"}), \ F(type_seg_split_ingest, {"type", "seg_split_ingest"}), \ - F(type_seg_merge_bg_gc, {"type", "seg_merge_bg_gc"}), \ + F(type_seg_merge_bg_gc, {"type", "seg_merge_bg_gc"}), \ F(type_place_index_update, {"type", "place_index_update"})) \ M(tiflash_storage_subtask_duration_seconds, "Bucketed histogram of storage's sub task duration", Histogram, \ F(type_delta_merge_bg, {{"type", "delta_merge_bg"}}, ExpBuckets{0.001, 2, 20}), \ @@ -164,7 +164,8 @@ namespace DB F(type_v3_bs_full_gc, {"type", "v3_bs_full_gc"})) \ M(tiflash_storage_page_gc_duration_seconds, "Bucketed histogram of page's gc task duration", Histogram, \ F(type_v2, {{"type", "v2"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_v2_compact, {{"type", "v2_compact"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_v2_data_compact, {{"type", "v2_data_compact"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_v2_ver_compact, {{"type", "v2_ver_compact"}}, ExpBuckets{0.0005, 2, 20}), \ /* Below are metrics for PageStorage V3 */ \ F(type_compact_wal, {{"type", "compact_wal"}}, ExpBuckets{0.0005, 2, 20}), \ F(type_compact_directory, {{"type", "compact_directory"}}, ExpBuckets{0.0005, 2, 20}), \ diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 456778af526..c88e739921f 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -145,6 +145,7 @@ struct ContextShared ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. BackgroundProcessingPoolPtr background_pool; /// The thread pool for the background work performed by the tables. BackgroundProcessingPoolPtr blockable_background_pool; /// The thread pool for the blockable background work performed by the tables. + BackgroundProcessingPoolPtr ps_compact_background_pool; /// The thread pool for the background work performed by the ps v2. mutable TMTContextPtr tmt_context; /// Context of TiFlash. Note that this should be free before background_pool. MultiVersion macros; /// Substitutions extracted from config. size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default) @@ -1374,6 +1375,15 @@ BackgroundProcessingPool & Context::getBlockableBackgroundPool() return *shared->blockable_background_pool; } +BackgroundProcessingPool & Context::getPSBackgroundPool() +{ + auto lock = getLock(); + // use the same size as `background_pool_size` + if (!shared->ps_compact_background_pool) + shared->ps_compact_background_pool = std::make_shared(settings.background_pool_size, "bg-page-"); + return *shared->ps_compact_background_pool; +} + void Context::createTMTContext(const TiFlashRaftConfig & raft_config, pingcap::ClusterConfig && cluster_config) { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 6a16e86bc96..9f8a440de3f 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -387,6 +387,7 @@ class Context BackgroundProcessingPool & getBackgroundPool(); BackgroundProcessingPool & initializeBlockableBackgroundPool(UInt16 pool_size); BackgroundProcessingPool & getBlockableBackgroundPool(); + BackgroundProcessingPool & getPSBackgroundPool(); void createTMTContext(const TiFlashRaftConfig & raft_config, pingcap::ClusterConfig && cluster_config); diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp index 43c5c2cd9b0..687044e5183 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp @@ -379,10 +379,13 @@ bool DeltaValueSpace::compact(DMContext & context) log_storage_snap = context.storage_pool.logReader()->getSnapshot(/*tracing_id*/ fmt::format("minor_compact_{}", simpleInfo())); } - // do compaction task WriteBatches wbs(context.storage_pool, context.getWriteLimiter()); - const auto & reader = context.storage_pool.newLogReader(context.getReadLimiter(), log_storage_snap); - compaction_task->prepare(context, wbs, reader); + { + // do compaction task + const auto & reader = context.storage_pool.newLogReader(context.getReadLimiter(), log_storage_snap); + compaction_task->prepare(context, wbs, reader); + log_storage_snap.reset(); // release the snapshot ASAP + } { std::scoped_lock lock(mutex); diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.cpp b/dbms/src/Storages/DeltaMerge/StoragePool.cpp index 14a06534af5..22ca45179a8 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.cpp +++ b/dbms/src/Storages/DeltaMerge/StoragePool.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include @@ -90,16 +91,19 @@ GlobalStoragePool::GlobalStoragePool(const PathPool & path_pool, Context & globa path_pool.getPSDiskDelegatorGlobalMulti("log"), extractConfig(settings, StorageType::Log), global_ctx.getFileProvider(), + global_ctx, true)) , data_storage(PageStorage::create("__global__.data", path_pool.getPSDiskDelegatorGlobalMulti("data"), extractConfig(settings, StorageType::Data), global_ctx.getFileProvider(), + global_ctx, true)) , meta_storage(PageStorage::create("__global__.meta", path_pool.getPSDiskDelegatorGlobalMulti("meta"), extractConfig(settings, StorageType::Meta), global_ctx.getFileProvider(), + global_ctx, true)) , global_context(global_ctx) { @@ -184,15 +188,18 @@ StoragePool::StoragePool(Context & global_ctx, NamespaceId ns_id_, StoragePathPo log_storage_v2 = PageStorage::create(name + ".log", storage_path_pool.getPSDiskDelegatorMulti("log"), extractConfig(global_context.getSettingsRef(), StorageType::Log), - global_context.getFileProvider()); + global_context.getFileProvider(), + global_context); data_storage_v2 = PageStorage::create(name + ".data", storage_path_pool.getPSDiskDelegatorSingle("data"), // keep for behavior not changed extractConfig(global_context.getSettingsRef(), StorageType::Data), - global_ctx.getFileProvider()); + global_context.getFileProvider(), + global_context); meta_storage_v2 = PageStorage::create(name + ".meta", storage_path_pool.getPSDiskDelegatorMulti("meta"), extractConfig(global_context.getSettingsRef(), StorageType::Meta), - global_ctx.getFileProvider()); + global_context.getFileProvider(), + global_context); log_storage_reader = std::make_shared(run_mode, ns_id, log_storage_v2, /*storage_v3_*/ nullptr, nullptr); data_storage_reader = std::make_shared(run_mode, ns_id, data_storage_v2, /*storage_v3_*/ nullptr, nullptr); meta_storage_reader = std::make_shared(run_mode, ns_id, meta_storage_v2, /*storage_v3_*/ nullptr, nullptr); @@ -246,18 +253,21 @@ StoragePool::StoragePool(Context & global_ctx, NamespaceId ns_id_, StoragePathPo storage_path_pool.getPSDiskDelegatorMulti("log"), extractConfig(global_context.getSettingsRef(), StorageType::Log), global_context.getFileProvider(), + global_context, /* use_v3 */ false, /* no_more_write_to_v2 */ true); data_storage_v2 = PageStorage::create(name + ".data", storage_path_pool.getPSDiskDelegatorMulti("data"), extractConfig(global_context.getSettingsRef(), StorageType::Data), - global_ctx.getFileProvider(), + global_context.getFileProvider(), + global_context, /* use_v3 */ false, /* no_more_write_to_v2 */ true); meta_storage_v2 = PageStorage::create(name + ".meta", storage_path_pool.getPSDiskDelegatorMulti("meta"), extractConfig(global_context.getSettingsRef(), StorageType::Meta), - global_ctx.getFileProvider(), + global_context.getFileProvider(), + global_context, /* use_v3 */ false, /* no_more_write_to_v2 */ true); } @@ -626,6 +636,12 @@ void StoragePool::shutdown() global_context.getBackgroundPool().removeTask(gc_handle); gc_handle = nullptr; } + if (run_mode != PageStorageRunMode::ONLY_V3) + { + meta_storage_v2->shutdown(); + log_storage_v2->shutdown(); + data_storage_v2->shutdown(); + } } void StoragePool::drop() diff --git a/dbms/src/Storages/Page/PageStorage.cpp b/dbms/src/Storages/Page/PageStorage.cpp index 0d323feba7f..1103daef4ce 100644 --- a/dbms/src/Storages/Page/PageStorage.cpp +++ b/dbms/src/Storages/Page/PageStorage.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -23,13 +24,14 @@ PageStoragePtr PageStorage::create( PSDiskDelegatorPtr delegator, const PageStorageConfig & config, const FileProviderPtr & file_provider, + Context & global_ctx, bool use_v3, bool no_more_insert_to_v2) { if (use_v3) return std::make_shared(name, delegator, config, file_provider); else - return std::make_shared(name, delegator, config, file_provider, no_more_insert_to_v2); + return std::make_shared(name, delegator, config, file_provider, global_ctx.getPSBackgroundPool(), no_more_insert_to_v2); } /*************************** diff --git a/dbms/src/Storages/Page/PageStorage.h b/dbms/src/Storages/Page/PageStorage.h index bb6835298ce..9368b8eb30f 100644 --- a/dbms/src/Storages/Page/PageStorage.h +++ b/dbms/src/Storages/Page/PageStorage.h @@ -99,6 +99,7 @@ class PageStorage : private boost::noncopyable PSDiskDelegatorPtr delegator, const PageStorageConfig & config, const FileProviderPtr & file_provider, + Context & global_ctx, bool use_v3 = false, bool no_more_insert_to_v2 = false); @@ -198,6 +199,8 @@ class PageStorage : private boost::noncopyable return gcImpl(not_skip, write_limiter, read_limiter); } + virtual void shutdown() {} + // Register and unregister external pages GC callbacks // Note that user must ensure that it is safe to call `scanner` and `remover` even after unregister. virtual void registerExternalPagesCallbacks(const ExternalPageCallbacks & callbacks) = 0; diff --git a/dbms/src/Storages/Page/V2/PageEntries.h b/dbms/src/Storages/Page/V2/PageEntries.h index 0a0504b0cb5..7f82a6907a0 100644 --- a/dbms/src/Storages/Page/V2/PageEntries.h +++ b/dbms/src/Storages/Page/V2/PageEntries.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -31,6 +32,11 @@ #include +namespace CurrentMetrics +{ +extern const int PSMVCCNumDelta; +extern const int PSMVCCNumBase; +} // namespace CurrentMetrics namespace DB { namespace ErrorCodes @@ -118,12 +124,30 @@ class PageEntriesMixin { public: explicit PageEntriesMixin(bool is_base_) - : normal_pages() - , page_ref() - , ref_deletions() - , max_page_id(0) + : max_page_id(0) , is_base(is_base_) - {} + { + if (is_base) + { + CurrentMetrics::add(CurrentMetrics::PSMVCCNumBase); + } + else + { + CurrentMetrics::add(CurrentMetrics::PSMVCCNumDelta); + } + } + + virtual ~PageEntriesMixin() + { + if (is_base) + { + CurrentMetrics::sub(CurrentMetrics::PSMVCCNumBase); + } + else + { + CurrentMetrics::sub(CurrentMetrics::PSMVCCNumDelta); + } + } public: static std::shared_ptr createBase() { return std::make_shared(true); } diff --git a/dbms/src/Storages/Page/V2/PageStorage.cpp b/dbms/src/Storages/Page/V2/PageStorage.cpp index cc98561a300..287f6b8d4c2 100644 --- a/dbms/src/Storages/Page/V2/PageStorage.cpp +++ b/dbms/src/Storages/Page/V2/PageStorage.cpp @@ -161,12 +161,14 @@ PageStorage::PageStorage(String name, PSDiskDelegatorPtr delegator_, // const PageStorageConfig & config_, const FileProviderPtr & file_provider_, + BackgroundProcessingPool & ver_compact_pool_, bool no_more_insert_) : DB::PageStorage(name, delegator_, config_, file_provider_) , write_files(std::max(1UL, config_.num_write_slots.get())) , page_file_log(&Poco::Logger::get("PageFile")) , log(&Poco::Logger::get("PageStorage")) , versioned_page_entries(storage_name, config.version_set_config, log) + , ver_compact_pool(ver_compact_pool_) , no_more_insert(no_more_insert_) { // at least 1 write slots @@ -184,6 +186,10 @@ PageStorage::PageStorage(String name, config.num_write_slots = num_paths * 2; } write_files.resize(config.num_write_slots); + + // If there is no snapshot released, check with default interval (10s) and exit quickly + // If snapshot released, wakeup this handle to compact the version list + ver_compact_handle = ver_compact_pool.addTask([this] { return compactInMemVersions(); }, /*multi*/ false); } @@ -361,7 +367,7 @@ void PageStorage::restore() PageId PageStorage::getMaxId() { std::lock_guard write_lock(write_mutex); - return versioned_page_entries.getSnapshot("")->version()->maxId(); + return versioned_page_entries.getSnapshot("", ver_compact_handle)->version()->maxId(); } PageId PageStorage::getNormalPageIdImpl(NamespaceId /*ns_id*/, PageId page_id, SnapshotPtr snapshot, bool throw_on_not_exist) @@ -588,13 +594,13 @@ void PageStorage::writeImpl(DB::WriteBatch && wb, const WriteLimiterPtr & write_ DB::PageStorage::SnapshotPtr PageStorage::getSnapshot(const String & tracing_id) { - return versioned_page_entries.getSnapshot(tracing_id); + return versioned_page_entries.getSnapshot(tracing_id, ver_compact_handle); } PageStorage::VersionedPageEntries::SnapshotPtr PageStorage::getConcreteSnapshot() { - return versioned_page_entries.getSnapshot(/*tracing_id*/ ""); + return versioned_page_entries.getSnapshot(/*tracing_id*/ "", ver_compact_handle); } SnapshotsStatistics PageStorage::getSnapshotsStat() const @@ -942,6 +948,28 @@ WriteBatch::SequenceID PageStorage::WritingFilesSnapshot::minPersistedSequence() return seq; } +void PageStorage::shutdown() +{ + if (ver_compact_handle) + { + ver_compact_pool.removeTask(ver_compact_handle); + ver_compact_handle = nullptr; + } +} + +bool PageStorage::compactInMemVersions() +{ + Stopwatch watch; + // try compact the in-mem version list + bool done_anything = versioned_page_entries.tryCompact(); + if (done_anything) + { + auto elapsed_sec = watch.elapsedSeconds(); + GET_METRIC(tiflash_storage_page_gc_duration_seconds, type_v2_ver_compact).Observe(elapsed_sec); + } + return done_anything; +} + bool PageStorage::gcImpl(bool not_skip, const WriteLimiterPtr & write_limiter, const ReadLimiterPtr & read_limiter) { // If another thread is running gc, just return; @@ -954,7 +982,6 @@ bool PageStorage::gcImpl(bool not_skip, const WriteLimiterPtr & write_limiter, c gc_is_running.compare_exchange_strong(is_running, false); }); - /// Get all pending external pages and PageFiles. Note that we should get external pages before PageFiles. ExternalPageCallbacks::PathAndIdsVec external_pages; if (external_pages_scanner) @@ -1185,7 +1212,7 @@ bool PageStorage::gcImpl(bool not_skip, const WriteLimiterPtr & write_limiter, c // We only care about those time cost in actually doing compaction on page data. if (gc_context.compact_result.do_compaction) { - GET_METRIC(tiflash_storage_page_gc_duration_seconds, type_v2_compact).Observe(watch_migrate.elapsedSeconds()); + GET_METRIC(tiflash_storage_page_gc_duration_seconds, type_v2_data_compact).Observe(watch_migrate.elapsedSeconds()); } } diff --git a/dbms/src/Storages/Page/V2/PageStorage.h b/dbms/src/Storages/Page/V2/PageStorage.h index 669e2efb629..bf59c53a6f1 100644 --- a/dbms/src/Storages/Page/V2/PageStorage.h +++ b/dbms/src/Storages/Page/V2/PageStorage.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -89,6 +90,7 @@ class PageStorage : public DB::PageStorage PSDiskDelegatorPtr delegator, // const PageStorageConfig & config_, const FileProviderPtr & file_provider_, + BackgroundProcessingPool & ver_compact_pool_, bool no_more_insert_ = false); ~PageStorage() override = default; @@ -128,6 +130,8 @@ class PageStorage : public DB::PageStorage bool gcImpl(bool not_skip, const WriteLimiterPtr & write_limiter, const ReadLimiterPtr & read_limiter) override; + void shutdown() override; + void registerExternalPagesCallbacks(const ExternalPageCallbacks & callbacks) override; FileProviderPtr getFileProvider() const { return file_provider; } @@ -238,6 +242,10 @@ class PageStorage : public DB::PageStorage template friend class DataCompactor; + // Try compact in memory versions. + // Return true if compact is executed. + bool compactInMemVersions(); + #ifndef DBMS_PUBLIC_GTEST private: #endif @@ -272,6 +280,10 @@ class PageStorage : public DB::PageStorage StatisticsInfo last_gc_statistics; + // background pool for running compact on `versioned_page_entries` + BackgroundProcessingPool & ver_compact_pool; + BackgroundProcessingPool::TaskHandle ver_compact_handle = nullptr; + // true means this instance runs under mix mode bool no_more_insert = false; diff --git a/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.cpp b/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.cpp index b5efeb71edf..41f95a37a34 100644 --- a/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.cpp +++ b/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -32,6 +33,7 @@ namespace ProfileEvents extern const Event PSMVCCCompactOnDelta; extern const Event PSMVCCCompactOnDeltaRebaseRejected; extern const Event PSMVCCCompactOnBase; +extern const Event PSMVCCCompactOnBaseCommit; extern const Event PSMVCCApplyOnCurrentBase; extern const Event PSMVCCApplyOnCurrentDelta; extern const Event PSMVCCApplyOnNewDelta; @@ -108,12 +110,12 @@ SnapshotsStatistics PageEntriesVersionSetWithDelta::getSnapshotsStat() const } -PageEntriesVersionSetWithDelta::SnapshotPtr PageEntriesVersionSetWithDelta::getSnapshot(const String & tracing_id) +PageEntriesVersionSetWithDelta::SnapshotPtr PageEntriesVersionSetWithDelta::getSnapshot(const String & tracing_id, BackgroundProcessingPool::TaskHandle handle) { // acquire for unique_lock since we need to add all snapshots to link list std::unique_lock lock(read_write_mutex); - auto s = std::make_shared(this, current, tracing_id); + auto s = std::make_shared(this, current, tracing_id, handle); // Register a weak_ptr to snapshot into VersionSet so that we can get all living PageFiles // by `PageEntriesVersionSetWithDelta::listAllLiveFiles`, and it remove useless weak_ptr of snapshots. // Do not call `vset->removeExpiredSnapshots` inside `~Snapshot`, or it may cause incursive deadlock @@ -165,7 +167,7 @@ std::unique_lock PageEntriesVersionSetWithDelta::acquireForLo return std::unique_lock(read_write_mutex); } -bool PageEntriesVersionSetWithDelta::isValidVersion(const VersionPtr tail) const +bool PageEntriesVersionSetWithDelta::isValidVersion(VersionPtr tail) const { for (auto node = current; node != nullptr; node = std::atomic_load(&node->prev)) { @@ -177,7 +179,7 @@ bool PageEntriesVersionSetWithDelta::isValidVersion(const VersionPtr tail) const return false; } -void PageEntriesVersionSetWithDelta::compactOnDeltaRelease(VersionPtr tail) +void PageEntriesVersionSetWithDelta::compactUntil(VersionPtr tail) { if (tail == nullptr || tail->isBase()) return; @@ -205,20 +207,27 @@ void PageEntriesVersionSetWithDelta::compactOnDeltaRelease(VersionPtr tail) tail = tmp; tmp.reset(); } + + if (!tail->shouldCompactToBase(config)) + { + return; + } + // do compact on base - if (tail->shouldCompactToBase(config)) + ProfileEvents::increment(ProfileEvents::PSMVCCCompactOnBase); + auto old_base = std::atomic_load(&tail->prev); + assert(old_base != nullptr); + // create a new_base and copy the entries from `old_base` and `tail` + VersionPtr new_base = PageEntriesForDelta::compactDeltaAndBase(old_base, tail); + // replace nodes [head, tail] by new_base + if (this->rebase(tail, new_base) == RebaseResult::INVALID_VERSION) { - ProfileEvents::increment(ProfileEvents::PSMVCCCompactOnBase); - auto old_base = std::atomic_load(&tail->prev); - assert(old_base != nullptr); - VersionPtr new_base = PageEntriesForDelta::compactDeltaAndBase(old_base, tail); - // replace nodes [head, tail] -> new_base - if (this->rebase(tail, new_base) == RebaseResult::INVALID_VERSION) - { - // Another thread may have done compaction and rebase, then we just release `tail`. In case we may add more code after do compaction on base - ProfileEvents::increment(ProfileEvents::PSMVCCCompactOnDeltaRebaseRejected); - return; - } + // Another thread may have done compaction and rebase, then we just release `tail`. In case we may add more code after do compaction on base + ProfileEvents::increment(ProfileEvents::PSMVCCCompactOnDeltaRebaseRejected); + } + else + { + ProfileEvents::increment(ProfileEvents::PSMVCCCompactOnBaseCommit); } } @@ -343,7 +352,8 @@ PageEntriesVersionSetWithDelta::listAllLiveFiles(std::unique_lock(this, current, "")); + // release this temporary snapshot won't cause version-list compact + valid_snapshots.emplace_back(std::make_shared(this, current, "", nullptr)); lock.unlock(); // Notice: unlock and we should free those valid snapshots without locking diff --git a/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.h b/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.h index bc83c6fca25..9045aa01b26 100644 --- a/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.h +++ b/dbms/src/Storages/Page/V2/VersionSet/PageEntriesVersionSetWithDelta.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -77,6 +78,29 @@ class PageEntriesVersionSetWithDelta size_t sizeUnlocked() const; + bool tryCompact() + { + const auto release_idx = last_released_snapshot_index.load(); + const auto last_try_idx = last_try_compact_index.load(); + if (release_idx <= last_try_idx) + { + return false; + } + + // some new snapshot are released, let's try + // compact the versions. + last_try_compact_index.store(release_idx); + + // compact version list with the latest snapshot. + // do NOT increase the index by this snapshot or it will + // cause inf loop + auto snap = getSnapshot("ps-mem-compact", nullptr); + compactUntil(snap->view.getSharedTailVersion()); + + // try compact again + return true; + } + SnapshotsStatistics getSnapshotsStat() const; std::string toDebugString() const @@ -126,23 +150,36 @@ class PageEntriesVersionSetWithDelta private: const TimePoint create_time; + // it should be a weak_ptr because the handle may be released before snapshot released + std::weak_ptr compact_handle; + public: - Snapshot(PageEntriesVersionSetWithDelta * vset_, VersionPtr tail_, const String & tracing_id_) + Snapshot(PageEntriesVersionSetWithDelta * vset_, VersionPtr tail_, const String & tracing_id_, BackgroundProcessingPool::TaskHandle handle) : vset(vset_) , view(std::move(tail_)) , create_thread(Poco::ThreadNumber::get()) , tracing_id(tracing_id_) , create_time(std::chrono::steady_clock::now()) + , compact_handle(handle) { CurrentMetrics::add(CurrentMetrics::PSMVCCNumSnapshots); } // Releasing a snapshot object may do compaction on vset's versions. - ~Snapshot() + ~Snapshot() override { - vset->compactOnDeltaRelease(view.getSharedTailVersion()); - // Remove snapshot from linked list + if (auto handle = compact_handle.lock(); handle) + { + // increase the index so that upper level know it should try + // the version compact. + vset->last_released_snapshot_index.fetch_add(1); + // Do vset->compactUntil on background pool + handle->wake(); + } + // else if the handle is nullptr (handle is not set or task has been removed from bkg pool), + // just skip the version list compact. + // Remove snapshot from linked list view.release(); CurrentMetrics::sub(CurrentMetrics::PSMVCCNumSnapshots); @@ -164,7 +201,7 @@ class PageEntriesVersionSetWithDelta using SnapshotPtr = std::shared_ptr; using SnapshotWeakPtr = std::weak_ptr; - SnapshotPtr getSnapshot(const String & tracing_id = ""); + SnapshotPtr getSnapshot(const String & tracing_id, BackgroundProcessingPool::TaskHandle handle); std::pair, std::set> gcApply(PageEntriesEdit & edit, bool need_scan_page_ids = true); @@ -203,11 +240,12 @@ class PageEntriesVersionSetWithDelta std::unique_lock acquireForLock(); // Return true if `tail` is in current version-list - bool isValidVersion(const VersionPtr tail) const; + bool isValidVersion(VersionPtr tail) const; // If `tail` is in the latest versions-list, do compaction on version-list [head, tail]. - // If there some versions after tail, use vset's `rebase` to concat them. - void compactOnDeltaRelease(VersionPtr tail); + // If there some versions after tail, use vset's `rebase` to concat those version to the + // new compacted version-list. + void compactUntil(VersionPtr tail); // Scan over all `snapshots`, remove the invalid snapshots and get some statistics // of all living snapshots and the oldest living snapshot. @@ -223,6 +261,9 @@ class PageEntriesVersionSetWithDelta bool need_scan_page_ids); private: + std::atomic last_released_snapshot_index{0}; + std::atomic last_try_compact_index{0}; + mutable std::shared_mutex read_write_mutex; VersionPtr current; mutable std::list snapshots; diff --git a/dbms/src/Storages/Page/V2/gc/LegacyCompactor.cpp b/dbms/src/Storages/Page/V2/gc/LegacyCompactor.cpp index fb6f0d2ca1c..c249ab481b7 100644 --- a/dbms/src/Storages/Page/V2/gc/LegacyCompactor.cpp +++ b/dbms/src/Storages/Page/V2/gc/LegacyCompactor.cpp @@ -80,7 +80,7 @@ LegacyCompactor::tryCompact( } // Build a version_set with snapshot - auto snapshot = version_set.getSnapshot(/*tracing_id*/ ""); + auto snapshot = version_set.getSnapshot(/*tracing_id*/ "", nullptr); auto wb = prepareCheckpointWriteBatch(snapshot, checkpoint_sequence); { diff --git a/dbms/src/Storages/Page/V2/tests/gtest_data_compactor.cpp b/dbms/src/Storages/Page/V2/tests/gtest_data_compactor.cpp index 531695aa0bb..e36f2f164a8 100644 --- a/dbms/src/Storages/Page/V2/tests/gtest_data_compactor.cpp +++ b/dbms/src/Storages/Page/V2/tests/gtest_data_compactor.cpp @@ -63,7 +63,8 @@ try const auto file_provider = ctx.getFileProvider(); PSDiskDelegatorPtr delegate = std::make_shared(test_paths); - PageStorage storage("data_compact_test", delegate, config, file_provider); + auto bkg_pool = std::make_shared(4, "bg-page-"); + PageStorage storage("data_compact_test", delegate, config, file_provider, *bkg_pool); #ifdef GENERATE_TEST_DATA // Codes to generate a directory of test data storage.restore(); @@ -174,7 +175,8 @@ try { // Try to recover from disk, check whether page 1, 2, 3, 4, 5, 6 is valid or not. - PageStorage ps("data_compact_test", delegate, config, file_provider); + auto bkg_pool = std::make_shared(4, "bg-page-"); + PageStorage ps("data_compact_test", delegate, config, file_provider, *bkg_pool); ps.restore(); // Page 1, 2 have been migrated to PageFile_2_1 PageEntry entry = ps.getEntry(1, nullptr); diff --git a/dbms/src/Storages/Page/V2/tests/gtest_legacy_compactor.cpp b/dbms/src/Storages/Page/V2/tests/gtest_legacy_compactor.cpp index ac7ee624970..4ca0980bf64 100644 --- a/dbms/src/Storages/Page/V2/tests/gtest_legacy_compactor.cpp +++ b/dbms/src/Storages/Page/V2/tests/gtest_legacy_compactor.cpp @@ -32,7 +32,7 @@ namespace DB::PS::V2::tests { -TEST(LegacyCompactor_test, WriteMultipleBatchRead) +TEST(LegacyCompactorTest, WriteMultipleBatchRead) try { PageStorageConfig config; @@ -72,7 +72,7 @@ try // Restore a new version set with snapshot WriteBatch WriteBatch::SequenceID seq_write = 0x1234; { - auto snapshot = original_version.getSnapshot(); + auto snapshot = original_version.getSnapshot("", nullptr); WriteBatch wb = LegacyCompactor::prepareCheckpointWriteBatch(snapshot, seq_write); EXPECT_EQ(wb.getSequence(), seq_write); @@ -100,9 +100,9 @@ try // Compare the two versions above { - auto original_snapshot = original_version.getSnapshot(); + auto original_snapshot = original_version.getSnapshot("", nullptr); const auto * original = original_snapshot->version(); - auto restored_snapshot = version_restored_with_snapshot.getSnapshot(); + auto restored_snapshot = version_restored_with_snapshot.getSnapshot("", nullptr); const auto * restored = restored_snapshot->version(); auto original_normal_page_ids = original->validNormalPageIds(); @@ -166,14 +166,15 @@ try CATCH // TODO: enable this test -TEST(LegacyCompactor_test, DISABLED_CompactAndRestore) +TEST(LegacyCompactorTest, DISABLED_CompactAndRestore) try { auto ctx = DB::tests::TiFlashTestEnv::getContext(); const FileProviderPtr file_provider = ctx.getFileProvider(); StoragePathPool spool = ctx.getPathPool().withTable("test", "t", false); auto delegator = spool.getPSDiskDelegatorSingle("meta"); - PageStorage storage("compact_test", delegator, PageStorageConfig{}, file_provider); + auto bkg_pool = std::make_shared(4, "bg-page-"); + PageStorage storage("compact_test", delegator, PageStorageConfig{}, file_provider, *bkg_pool); PageStorage::ListPageFilesOption opt; opt.ignore_checkpoint = false; @@ -211,8 +212,8 @@ try (void)page_files_to_remove; { - auto s0 = compactor.version_set.getSnapshot(); - auto s1 = vset_restored.getSnapshot(); + auto s0 = compactor.version_set.getSnapshot("", nullptr); + auto s1 = vset_restored.getSnapshot("", nullptr); ASSERT_EQ(s0->version()->numPages(), s1->version()->numPages()); ASSERT_EQ(s0->version()->numNormalPages(), s1->version()->numNormalPages()); diff --git a/dbms/src/Storages/Page/V2/tests/gtest_page_map_version_set.cpp b/dbms/src/Storages/Page/V2/tests/gtest_page_map_version_set.cpp index 9a8c623da28..622251f32db 100644 --- a/dbms/src/Storages/Page/V2/tests/gtest_page_map_version_set.cpp +++ b/dbms/src/Storages/Page/V2/tests/gtest_page_map_version_set.cpp @@ -13,39 +13,43 @@ // limitations under the License. #include +#include #include #include +#include #include namespace DB::PS::V2::tests { template -class PageMapVersionSet_test : public ::testing::Test +class PageMapVersionSetTest : public ::testing::Test { public: - PageMapVersionSet_test() - : log(&Poco::Logger::get("PageMapVersionSet_test")) + PageMapVersionSetTest() + : log(&Poco::Logger::get("PageMapVersionSetTest")) {} static void SetUpTestCase() {} void SetUp() override { - config_.compact_hint_delta_entries = 1; - config_.compact_hint_delta_deletions = 1; + config.compact_hint_delta_entries = 1; + config.compact_hint_delta_deletions = 1; + bkg_pool = std::make_shared(4, "bg-page-"); } protected: - DB::MVCC::VersionSetConfig config_; + DB::MVCC::VersionSetConfig config; + std::shared_ptr bkg_pool; Poco::Logger * log; }; -TYPED_TEST_CASE_P(PageMapVersionSet_test); +TYPED_TEST_CASE_P(PageMapVersionSetTest); -TYPED_TEST_P(PageMapVersionSet_test, ApplyEdit) +TYPED_TEST_P(PageMapVersionSetTest, ApplyEdit) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); LOG_TRACE(&Poco::Logger::root(), "init :" + versions.toDebugString()); { PageEntriesEdit edit; @@ -67,7 +71,7 @@ TYPED_TEST_P(PageMapVersionSet_test, ApplyEdit) versions.apply(edit); } LOG_TRACE(&Poco::Logger::root(), "apply B:" + versions.toDebugString()); - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); EXPECT_EQ(versions.size(), 1UL); auto entry = s2->version()->at(0); ASSERT_EQ(entry.checksum, 0x123UL); @@ -81,10 +85,15 @@ TYPED_TEST_P(PageMapVersionSet_test, ApplyEdit) /// Generate two different snapshot(s1, s2) with apply new edits. /// s2 released first, then release s1 -TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock) +TYPED_TEST_P(PageMapVersionSetTest, ApplyEditWithReadLock) { - TypeParam versions("vset_test", this->config_, this->log); - auto s1 = versions.getSnapshot(); + TypeParam versions("vset_test", this->config, this->log); + auto ver_compact_handle + = this->bkg_pool->addTask([&] { return false; }, /*multi*/ false); + SCOPE_EXIT({ + this->bkg_pool->removeTask(ver_compact_handle); + }); + auto s1 = versions.getSnapshot("", ver_compact_handle); EXPECT_EQ(versions.size(), 1UL); LOG_TRACE(&Poco::Logger::root(), "snapshot 1:" + versions.toDebugString()); { @@ -98,7 +107,7 @@ TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock) LOG_TRACE(&Poco::Logger::root(), "apply B:" + versions.toDebugString()); // Get snapshot for checking edit is success - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", ver_compact_handle); LOG_TRACE(&Poco::Logger::root(), "snapshot 2:" + versions.toDebugString()); auto entry = s2->version()->at(0); ASSERT_EQ(entry.checksum, 0x123UL); @@ -107,18 +116,20 @@ TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock) s2.reset(); LOG_TRACE(&Poco::Logger::root(), "rel snap 2:" + versions.toDebugString()); - /// For VersionDeltaSet, size is 1 since we do a compaction on delta - EXPECT_EQ(versions.size(), 1UL); + /// For VersionDeltaSet, size is 1 since we always do compact with latest tail + versions.tryCompact(); + EXPECT_EQ(versions.size(), 1); s1.reset(); LOG_TRACE(&Poco::Logger::root(), "rel snap 1:" + versions.toDebugString()); // VersionSet, old version removed from version set // VersionSetWithDelta, delta version merged - EXPECT_EQ(versions.size(), 1UL); + versions.tryCompact(); + EXPECT_EQ(versions.size(), 1); // Ensure that after old snapshot released, new snapshot get the same content - auto s3 = versions.getSnapshot(); + auto s3 = versions.getSnapshot("", ver_compact_handle); entry = s3->version()->at(0); ASSERT_EQ(entry.checksum, 0x123UL); s3.reset(); @@ -133,18 +144,24 @@ TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock) LOG_TRACE(&Poco::Logger::root(), "apply C:" + versions.toDebugString()); // VersionSet, new version gen and old version remove at the same time // VersionSetWithDelta, C merge to delta - EXPECT_EQ(versions.size(), 1UL); - auto s4 = versions.getSnapshot(); + versions.tryCompact(); + EXPECT_EQ(versions.size(), 1); + auto s4 = versions.getSnapshot("", nullptr); entry = s4->version()->at(0); ASSERT_EQ(entry.checksum, 0x456UL); } /// Generate two different snapshot(s1, s2) with apply new edits. /// s1 released first, then release s2 -TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock2) +TYPED_TEST_P(PageMapVersionSetTest, ApplyEditWithReadLock2) { - TypeParam versions("vset_test", this->config_, this->log); - auto s1 = versions.getSnapshot(); + TypeParam versions("vset_test", this->config, this->log); + auto ver_compact_handle + = this->bkg_pool->addTask([&] { return false; }, /*multi*/ false); + SCOPE_EXIT({ + this->bkg_pool->removeTask(ver_compact_handle); + }); + auto s1 = versions.getSnapshot("", ver_compact_handle); LOG_TRACE(&Poco::Logger::root(), "snapshot 1:" + versions.toDebugString()); PageEntriesEdit edit; PageEntry e; @@ -152,27 +169,34 @@ TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock2) edit.put(0, e); versions.apply(edit); LOG_TRACE(&Poco::Logger::root(), "apply B:" + versions.toDebugString()); - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", ver_compact_handle); auto entry = s2->version()->at(0); ASSERT_EQ(entry.checksum, 0x123UL); s1.reset(); LOG_TRACE(&Poco::Logger::root(), "rel snap 1:" + versions.toDebugString()); - // VersionSetWithDelta, size is 2 since we can not do a compaction on delta - EXPECT_EQ(versions.size(), 2UL); + // VersionSetWithDelta, size is 1 since we always do compact with latest tail + versions.tryCompact(); + EXPECT_EQ(versions.size(), 1); s2.reset(); LOG_TRACE(&Poco::Logger::root(), "rel snap 2:" + versions.toDebugString()); - EXPECT_EQ(versions.size(), 1UL); + versions.tryCompact(); + EXPECT_EQ(versions.size(), 1); } /// Generate two different snapshot(s1, s2) with apply new edits. /// s1 released first, then release s2 -TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock3) +TYPED_TEST_P(PageMapVersionSetTest, ApplyEditWithReadLock3) { - TypeParam versions("vset_test", this->config_, this->log); - auto s1 = versions.getSnapshot(); + TypeParam versions("vset_test", this->config, this->log); + auto ver_compact_handle + = this->bkg_pool->addTask([&] { return false; }, /*multi*/ false); + SCOPE_EXIT({ + this->bkg_pool->removeTask(ver_compact_handle); + }); + auto s1 = versions.getSnapshot("", ver_compact_handle); LOG_TRACE(&Poco::Logger::root(), "snapshot 1:" + versions.toDebugString()); { PageEntriesEdit edit; @@ -182,7 +206,7 @@ TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock3) versions.apply(edit); } LOG_TRACE(&Poco::Logger::root(), "apply B:" + versions.toDebugString()); - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", ver_compact_handle); auto entry = s2->version()->at(0); ASSERT_EQ(entry.checksum, 0x123UL); @@ -194,23 +218,26 @@ TYPED_TEST_P(PageMapVersionSet_test, ApplyEditWithReadLock3) versions.apply(edit); } LOG_TRACE(&Poco::Logger::root(), "apply C:" + versions.toDebugString()); - auto s3 = versions.getSnapshot(); + auto s3 = versions.getSnapshot("", ver_compact_handle); entry = s3->version()->at(1); ASSERT_EQ(entry.checksum, 0xFFUL); s1.reset(); LOG_TRACE(&Poco::Logger::root(), "rel snap 1:" + versions.toDebugString()); - // VersionSetWithDelta, size is 3 since we can not do a compaction on delta - EXPECT_EQ(versions.size(), 3UL); + // VersionSetWithDelta, size is 1 since we always do compact with latest tail + versions.tryCompact(); + EXPECT_EQ(versions.size(), 1); s2.reset(); LOG_TRACE(&Poco::Logger::root(), "rel snap 2:" + versions.toDebugString()); - EXPECT_EQ(versions.size(), 2UL); + versions.tryCompact(); + EXPECT_EQ(versions.size(), 1); s3.reset(); LOG_TRACE(&Poco::Logger::root(), "rel snap 3:" + versions.toDebugString()); - EXPECT_EQ(versions.size(), 1UL); + versions.tryCompact(); + EXPECT_EQ(versions.size(), 1); } namespace @@ -227,9 +254,9 @@ std::set getNormalPageIDs(const PageEntriesVersionSetWithDelta::Snapshot } // namespace -TYPED_TEST_P(PageMapVersionSet_test, Restore) +TYPED_TEST_P(PageMapVersionSetTest, Restore) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); // For PageEntriesVersionSetWithDelta, we directly apply edit to versions { PageEntriesEdit edit; @@ -249,7 +276,7 @@ TYPED_TEST_P(PageMapVersionSet_test, Restore) versions.apply(edit); } - auto s = versions.getSnapshot(); + auto s = versions.getSnapshot("", nullptr); auto entry = s->version()->find(1); ASSERT_EQ(entry, std::nullopt); auto entry2 = s->version()->find(2); @@ -264,9 +291,9 @@ TYPED_TEST_P(PageMapVersionSet_test, Restore) ASSERT_TRUE(valid_normal_page_ids.count(3) > 0); } -TYPED_TEST_P(PageMapVersionSet_test, PutOrDelRefPage) +TYPED_TEST_P(PageMapVersionSetTest, PutOrDelRefPage) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); { PageEntriesEdit edit; PageEntry e; @@ -274,7 +301,7 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOrDelRefPage) edit.put(2, e); versions.apply(edit); } - auto s1 = versions.getSnapshot(); + auto s1 = versions.getSnapshot("", nullptr); ASSERT_EQ(s1->version()->at(2).checksum, 0xfUL); // Put RefPage3 -> Page2 @@ -283,7 +310,7 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOrDelRefPage) edit.ref(3, 2); versions.apply(edit); } - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); auto ensure_snapshot2_status = [&s2]() { // Check the ref-count auto entry3 = s2->version()->at(3); @@ -311,7 +338,7 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOrDelRefPage) edit.del(2); versions.apply(edit); } - auto s3 = versions.getSnapshot(); + auto s3 = versions.getSnapshot("", nullptr); auto ensure_snapshot3_status = [&s3]() { // Check that NormalPage2's ref-count is decreased. auto entry3 = s3->version()->at(3); @@ -338,7 +365,7 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOrDelRefPage) edit.del(3); versions.apply(edit); } - auto s4 = versions.getSnapshot(); + auto s4 = versions.getSnapshot("", nullptr); auto ensure_snapshot4_status = [&s4]() { auto entry3 = s4->version()->find(3); ASSERT_FALSE(entry3); @@ -368,9 +395,9 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOrDelRefPage) ensure_snapshot4_status(); } -TYPED_TEST_P(PageMapVersionSet_test, IdempotentDel) +TYPED_TEST_P(PageMapVersionSetTest, IdempotentDel) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); { PageEntriesEdit edit; PageEntry e; @@ -379,7 +406,7 @@ TYPED_TEST_P(PageMapVersionSet_test, IdempotentDel) edit.ref(3, 2); versions.apply(edit); } - auto s1 = versions.getSnapshot(); + auto s1 = versions.getSnapshot("", nullptr); ASSERT_EQ(s1->version()->at(2).checksum, 0xfUL); // Del Page2 @@ -388,7 +415,7 @@ TYPED_TEST_P(PageMapVersionSet_test, IdempotentDel) edit.del(2); versions.apply(edit); } - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); { auto ref_entry = s2->version()->at(3); ASSERT_EQ(ref_entry.checksum, 0xfUL); @@ -404,7 +431,7 @@ TYPED_TEST_P(PageMapVersionSet_test, IdempotentDel) edit.del(2); versions.apply(edit); } - auto s3 = versions.getSnapshot(); + auto s3 = versions.getSnapshot("", nullptr); { auto ref_entry = s3->version()->at(3); ASSERT_EQ(ref_entry.checksum, 0xfUL); @@ -415,10 +442,10 @@ TYPED_TEST_P(PageMapVersionSet_test, IdempotentDel) } } -TYPED_TEST_P(PageMapVersionSet_test, GcConcurrencyDelPage) +TYPED_TEST_P(PageMapVersionSetTest, GcConcurrencyDelPage) { PageId pid = 0; - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); // Page0 is in PageFile{2, 0} at first { PageEntriesEdit init_edit; @@ -447,7 +474,7 @@ TYPED_TEST_P(PageMapVersionSet_test, GcConcurrencyDelPage) versions.gcApply(gc_edit); // Page0 don't update to page_map - auto snapshot = versions.getSnapshot(); + auto snapshot = versions.getSnapshot("", nullptr); auto entry = snapshot->version()->find(pid); ASSERT_EQ(entry, std::nullopt); } @@ -460,14 +487,14 @@ static void EXPECT_PagePos_LT(PageFileIdAndLevel p0, PageFileIdAndLevel p1) } #pragma clang diagnostic pop -TYPED_TEST_P(PageMapVersionSet_test, GcPageMove) +TYPED_TEST_P(PageMapVersionSetTest, GcPageMove) { EXPECT_PagePos_LT({4, 0}, {5, 1}); EXPECT_PagePos_LT({5, 0}, {5, 1}); EXPECT_PagePos_LT({5, 1}, {6, 1}); EXPECT_PagePos_LT({5, 2}, {6, 1}); - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); const PageId pid = 0; const PageId ref_pid = 1; @@ -493,7 +520,7 @@ TYPED_TEST_P(PageMapVersionSet_test, GcPageMove) } // Page get updated - auto snapshot = versions.getSnapshot(); + auto snapshot = versions.getSnapshot("", nullptr); PageEntry entry = snapshot->version()->at(pid); ASSERT_TRUE(entry.isValid()); ASSERT_EQ(entry.file_id, 5ULL); @@ -508,10 +535,10 @@ TYPED_TEST_P(PageMapVersionSet_test, GcPageMove) ASSERT_EQ(entry.ref, 2u); } -TYPED_TEST_P(PageMapVersionSet_test, GcConcurrencySetPage) +TYPED_TEST_P(PageMapVersionSetTest, GcConcurrencySetPage) { const PageId pid = 0; - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); // gc move Page0 -> PageFile{5,1} @@ -537,16 +564,16 @@ TYPED_TEST_P(PageMapVersionSet_test, GcConcurrencySetPage) versions.gcApply(gc_edit); // read - auto snapshot = versions.getSnapshot(); + auto snapshot = versions.getSnapshot("", nullptr); const PageEntry entry = snapshot->version()->at(pid); ASSERT_TRUE(entry.isValid()); ASSERT_EQ(entry.file_id, 6ULL); ASSERT_EQ(entry.level, 0U); } -TYPED_TEST_P(PageMapVersionSet_test, UpdateOnRefPage) +TYPED_TEST_P(PageMapVersionSetTest, UpdateOnRefPage) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); { PageEntriesEdit edit; PageEntry e; @@ -555,7 +582,7 @@ TYPED_TEST_P(PageMapVersionSet_test, UpdateOnRefPage) edit.ref(3, 2); versions.apply(edit); } - auto s1 = versions.getSnapshot(); + auto s1 = versions.getSnapshot("", nullptr); ASSERT_EQ(s1->version()->at(2).checksum, 0xfUL); ASSERT_EQ(s1->version()->at(3).checksum, 0xfUL); @@ -567,12 +594,12 @@ TYPED_TEST_P(PageMapVersionSet_test, UpdateOnRefPage) edit.put(3, e); versions.apply(edit); } - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); ASSERT_EQ(s2->version()->at(3).checksum, 0xffUL); ASSERT_EQ(s2->version()->at(2).checksum, 0xffUL); s2.reset(); s1.reset(); - auto s3 = versions.getSnapshot(); + auto s3 = versions.getSnapshot("", nullptr); ASSERT_EQ(s3->version()->at(3).checksum, 0xffUL); ASSERT_EQ(s3->version()->at(2).checksum, 0xffUL); //s3.reset(); @@ -583,7 +610,7 @@ TYPED_TEST_P(PageMapVersionSet_test, UpdateOnRefPage) edit.del(2); versions.apply(edit); } - auto s4 = versions.getSnapshot(); + auto s4 = versions.getSnapshot("", nullptr); ASSERT_EQ(s4->version()->find(2), std::nullopt); ASSERT_EQ(s4->version()->at(3).checksum, 0xffUL); s4.reset(); @@ -591,14 +618,14 @@ TYPED_TEST_P(PageMapVersionSet_test, UpdateOnRefPage) ASSERT_EQ(s3->version()->at(3).checksum, 0xffUL); s3.reset(); - auto s5 = versions.getSnapshot(); + auto s5 = versions.getSnapshot("", nullptr); ASSERT_EQ(s5->version()->find(2), std::nullopt); ASSERT_EQ(s5->version()->at(3).checksum, 0xffUL); } -TYPED_TEST_P(PageMapVersionSet_test, UpdateOnRefPage2) +TYPED_TEST_P(PageMapVersionSetTest, UpdateOnRefPage2) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); { PageEntriesEdit edit; PageEntry e; @@ -608,7 +635,7 @@ TYPED_TEST_P(PageMapVersionSet_test, UpdateOnRefPage2) edit.del(2); versions.apply(edit); } - auto s1 = versions.getSnapshot(); + auto s1 = versions.getSnapshot("", nullptr); ASSERT_EQ(s1->version()->find(2), std::nullopt); ASSERT_EQ(s1->version()->at(3).checksum, 0xfUL); @@ -620,14 +647,14 @@ TYPED_TEST_P(PageMapVersionSet_test, UpdateOnRefPage2) edit.del(2); versions.apply(edit); } - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); ASSERT_EQ(s2->version()->find(2), std::nullopt); ASSERT_EQ(s2->version()->at(3).checksum, 0x9UL); } -TYPED_TEST_P(PageMapVersionSet_test, IsRefId) +TYPED_TEST_P(PageMapVersionSetTest, IsRefId) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); { PageEntriesEdit edit; PageEntry e; @@ -636,7 +663,7 @@ TYPED_TEST_P(PageMapVersionSet_test, IsRefId) edit.ref(2, 1); versions.apply(edit); } - auto s1 = versions.getSnapshot(); + auto s1 = versions.getSnapshot("", nullptr); bool is_ref; PageId normal_page_id; std::tie(is_ref, normal_page_id) = s1->version()->isRefId(2); @@ -648,14 +675,14 @@ TYPED_TEST_P(PageMapVersionSet_test, IsRefId) edit.del(2); versions.apply(edit); } - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); std::tie(is_ref, normal_page_id) = s2->version()->isRefId(2); ASSERT_FALSE(is_ref); } -TYPED_TEST_P(PageMapVersionSet_test, Snapshot) +TYPED_TEST_P(PageMapVersionSetTest, Snapshot) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); ASSERT_EQ(versions.size(), 1UL); { PageEntriesEdit init_edit; @@ -668,7 +695,7 @@ TYPED_TEST_P(PageMapVersionSet_test, Snapshot) ASSERT_EQ(versions.size(), 1UL); } - auto s1 = versions.getSnapshot(); + auto s1 = versions.getSnapshot("", nullptr); // Apply edit that // * update Page 0 with checksum = 0x456 @@ -687,7 +714,7 @@ TYPED_TEST_P(PageMapVersionSet_test, Snapshot) ASSERT_EQ(s1->version()->at(0).checksum, 0x123UL); ASSERT_EQ(s1->version()->at(1).checksum, 0x1234UL); - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); auto p0 = s2->version()->find(0); ASSERT_NE(p0, std::nullopt); ASSERT_EQ(p0->checksum, 0x456UL); // entry is updated in snapshot 2 @@ -731,9 +758,9 @@ String livePagesToString(const std::set & ids) } // namespace -TYPED_TEST_P(PageMapVersionSet_test, LiveFiles) +TYPED_TEST_P(PageMapVersionSetTest, LiveFiles) { - TypeParam versions("vset_test", this->config_, this->log); + TypeParam versions("vset_test", this->config, this->log); { PageEntriesEdit edit; @@ -747,7 +774,7 @@ TYPED_TEST_P(PageMapVersionSet_test, LiveFiles) edit.put(2, e); versions.apply(edit); } - auto s1 = versions.getSnapshot(); + auto s1 = versions.getSnapshot("", nullptr); { PageEntriesEdit edit; edit.del(0); @@ -757,13 +784,13 @@ TYPED_TEST_P(PageMapVersionSet_test, LiveFiles) edit.put(3, e); versions.apply(edit); } - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); { PageEntriesEdit edit; edit.del(3); versions.apply(edit); } - auto s3 = versions.getSnapshot(); + auto s3 = versions.getSnapshot("", nullptr); s3.reset(); // do compact on version-list, and //std::cerr << "s3 reseted." << std::endl; auto [livefiles, live_normal_pages] = versions.listAllLiveFiles(versions.acquireForLock()); @@ -801,7 +828,7 @@ TYPED_TEST_P(PageMapVersionSet_test, LiveFiles) EXPECT_GT(live_normal_pages.count(2), 0UL); } -TYPED_TEST_P(PageMapVersionSet_test, PutOnTombstonePageEntry) +TYPED_TEST_P(PageMapVersionSetTest, PutOnTombstonePageEntry) { if constexpr (std::is_same_v) { @@ -817,7 +844,7 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOnTombstonePageEntry) edit.put(page_id, e); versions.apply(edit); } - auto s1 = versions.getSnapshot(); + auto s1 = versions.getSnapshot("", nullptr); { // Then delete that page, because there is read lock on previouse version, @@ -826,7 +853,7 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOnTombstonePageEntry) edit.del(page_id); versions.apply(edit); // Now there is a tombstone on current version. - auto s2 = versions.getSnapshot(); + auto s2 = versions.getSnapshot("", nullptr); auto entry = s2->version()->find(page_id); ASSERT_FALSE(entry); // Get tombstone by find return nullopt auto normal_entry = s2->version()->findNormalPageEntry(page_id); @@ -842,7 +869,7 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOnTombstonePageEntry) e.checksum = 0x6; edit.put(page_id, e); versions.apply(edit); - auto s3 = versions.getSnapshot(); + auto s3 = versions.getSnapshot("", nullptr); auto entry = s3->version()->find(page_id); ASSERT_TRUE(entry); ASSERT_EQ(entry->ref, 1UL); @@ -852,7 +879,7 @@ TYPED_TEST_P(PageMapVersionSet_test, PutOnTombstonePageEntry) } } -REGISTER_TYPED_TEST_CASE_P(PageMapVersionSet_test, +REGISTER_TYPED_TEST_CASE_P(PageMapVersionSetTest, ApplyEdit, ApplyEditWithReadLock, ApplyEditWithReadLock2, @@ -871,6 +898,6 @@ REGISTER_TYPED_TEST_CASE_P(PageMapVersionSet_test, PutOnTombstonePageEntry); using VersionSetTypes = ::testing::Types; -INSTANTIATE_TYPED_TEST_CASE_P(VersionSetTypedTest, PageMapVersionSet_test, VersionSetTypes); +INSTANTIATE_TYPED_TEST_CASE_P(VersionSetTypedTest, PageMapVersionSetTest, VersionSetTypes); } // namespace DB::PS::V2::tests diff --git a/dbms/src/Storages/Page/V2/tests/gtest_page_storage.cpp b/dbms/src/Storages/Page/V2/tests/gtest_page_storage.cpp index 4793304c530..ebea32b69f1 100644 --- a/dbms/src/Storages/Page/V2/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V2/tests/gtest_page_storage.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -52,15 +53,17 @@ class PageStorage_test : public DB::base::TiFlashStorageTestBasic { public: PageStorage_test() - : storage() - , file_provider{DB::tests::TiFlashTestEnv::getContext().getFileProvider()} + : file_provider{DB::tests::TiFlashTestEnv::getContext().getFileProvider()} {} protected: - static void SetUpTestCase() {} + static void SetUpTestCase() + { + } void SetUp() override { + bkg_pool = std::make_shared(4, "bg-page-"); TiFlashStorageTestBasic::SetUp(); // drop dir if exists path_pool = std::make_unique(db_context->getPathPool().withTable("test", "t1", false)); @@ -74,13 +77,14 @@ class PageStorage_test : public DB::base::TiFlashStorageTestBasic std::shared_ptr reopenWithConfig(const PageStorageConfig & config_) { auto delegator = path_pool->getPSDiskDelegatorSingle("log"); - auto storage = std::make_shared("test.t", delegator, config_, file_provider); + auto storage = std::make_shared("test.t", delegator, config_, file_provider, *bkg_pool); storage->restore(); return storage; } protected: PageStorageConfig config; + std::shared_ptr bkg_pool; std::shared_ptr storage; std::unique_ptr path_pool; const FileProviderPtr file_provider; diff --git a/dbms/src/Storages/Page/V2/tests/gtest_page_storage_multi_paths.cpp b/dbms/src/Storages/Page/V2/tests/gtest_page_storage_multi_paths.cpp index 4b985dd525b..178c3a3edcd 100644 --- a/dbms/src/Storages/Page/V2/tests/gtest_page_storage_multi_paths.cpp +++ b/dbms/src/Storages/Page/V2/tests/gtest_page_storage_multi_paths.cpp @@ -43,13 +43,12 @@ namespace DB::PS::V2::tests { using PSPtr = std::shared_ptr; -class PageStorageMultiPaths_test : public DB::base::TiFlashStorageTestBasic +class PageStorageMultiPathsTest : public DB::base::TiFlashStorageTestBasic , public ::testing::WithParamInterface { public: - PageStorageMultiPaths_test() - : storage() - , file_provider{DB::tests::TiFlashTestEnv::getContext().getFileProvider()} + PageStorageMultiPathsTest() + : file_provider{DB::tests::TiFlashTestEnv::getContext().getFileProvider()} {} static void SetUpTestCase() {} @@ -59,6 +58,7 @@ class PageStorageMultiPaths_test : public DB::base::TiFlashStorageTestBasic { // drop dir if exists dropDataOnDisk(getTemporaryPath()); + bkg_pool = std::make_shared(4, "bg-page-"); // default test config config.file_roll_size = 4 * MB; config.gc_max_valid_rate = 0.5; @@ -73,18 +73,19 @@ class PageStorageMultiPaths_test : public DB::base::TiFlashStorageTestBasic return paths; } - String getParentPathForTable(const String & /*db*/, const String & table = "table") + static String getParentPathForTable(const String & /*db*/, const String & table = "table") { return Poco::Path{getTemporaryPath() + "/ps_multi_paths/data" + toString(0) + "/" + table + "/log"}.toString(); } protected: PageStorageConfig config; + std::shared_ptr bkg_pool; std::shared_ptr storage; const FileProviderPtr file_provider; }; -TEST_P(PageStorageMultiPaths_test, DeltaWriteReadRestore) +TEST_P(PageStorageMultiPathsTest, DeltaWriteReadRestore) try { config.file_roll_size = 128 * MB; @@ -94,7 +95,7 @@ try auto capacity = std::make_shared(0, all_paths, std::vector{}, Strings{}, std::vector{}); StoragePathPool pool = PathPool(all_paths, all_paths, Strings{}, capacity, file_provider).withTable("test", "table", false); - storage = std::make_shared("test.table", pool.getPSDiskDelegatorMulti("log"), config, file_provider); + storage = std::make_shared("test.table", pool.getPSDiskDelegatorMulti("log"), config, file_provider, *bkg_pool); storage->restore(); const UInt64 tag = 0; @@ -132,7 +133,7 @@ try } // restore - storage = std::make_shared("test.t", pool.getPSDiskDelegatorMulti("log"), config, file_provider); + storage = std::make_shared("test.t", pool.getPSDiskDelegatorMulti("log"), config, file_provider, *bkg_pool); storage->restore(); // Read again @@ -188,7 +189,7 @@ try } // Restore. This ensure last write is correct. - storage = std::make_shared("test.t", pool.getPSDiskDelegatorMulti("log"), config, file_provider); + storage = std::make_shared("test.t", pool.getPSDiskDelegatorMulti("log"), config, file_provider, *bkg_pool); storage->restore(); // Read again to check all data. @@ -218,6 +219,6 @@ try } CATCH -INSTANTIATE_TEST_CASE_P(DifferentNumberOfDeltaPaths, PageStorageMultiPaths_test, testing::Range(1UL, 7UL)); +INSTANTIATE_TEST_CASE_P(DifferentNumberOfDeltaPaths, PageStorageMultiPathsTest, testing::Range(1UL, 7UL)); } // namespace DB::PS::V2::tests diff --git a/dbms/src/Storages/Page/V2/tests/gtest_page_storage_multi_writers.cpp b/dbms/src/Storages/Page/V2/tests/gtest_page_storage_multi_writers.cpp index 63a750e1c23..5cc7be170d6 100644 --- a/dbms/src/Storages/Page/V2/tests/gtest_page_storage_multi_writers.cpp +++ b/dbms/src/Storages/Page/V2/tests/gtest_page_storage_multi_writers.cpp @@ -45,8 +45,7 @@ class PageStorageMultiWriters_test : public DB::base::TiFlashStorageTestBasic { public: PageStorageMultiWriters_test() - : storage() - , file_provider{DB::tests::TiFlashTestEnv::getContext().getFileProvider()} + : file_provider{DB::tests::TiFlashTestEnv::getContext().getFileProvider()} {} protected: @@ -55,6 +54,7 @@ class PageStorageMultiWriters_test : public DB::base::TiFlashStorageTestBasic void SetUp() override { TiFlashStorageTestBasic::SetUp(); + bkg_pool = std::make_shared(4, "bg-page-"); // default test config config.file_roll_size = 4 * MB; config.gc_max_valid_rate = 0.5; @@ -67,20 +67,21 @@ class PageStorageMultiWriters_test : public DB::base::TiFlashStorageTestBasic { auto spool = db_context->getPathPool().withTable("test", "t", false); auto delegator = spool.getPSDiskDelegatorSingle("log"); - auto storage = std::make_shared("test.t", delegator, config_, file_provider); + auto storage = std::make_shared("test.t", delegator, config_, file_provider, *bkg_pool); storage->restore(); return storage; } protected: PageStorageConfig config; + std::shared_ptr bkg_pool; std::shared_ptr storage; const FileProviderPtr file_provider; }; struct TestContext { - const PageId MAX_PAGE_ID = 2000; + static constexpr PageId MAX_PAGE_ID = 2000; std::atomic running_without_exception = true; std::atomic running_without_timeout = true; @@ -113,7 +114,6 @@ class PSWriter : public Poco::Runnable PSWriter(const PSPtr & storage_, DB::UInt32 idx, TestContext & ctx_) : index(idx) , storage(storage_) - , gen() , bytes_written(0) , pages_written(0) , ctx(ctx_) @@ -141,16 +141,16 @@ class PSWriter : public Poco::Runnable static void fillAllPages(const PSPtr & storage, TestContext & ctx) { - for (PageId pageId = 0; pageId < ctx.MAX_PAGE_ID; ++pageId) + for (PageId page_id = 0; page_id < ctx.MAX_PAGE_ID; ++page_id) { MemHolder holder; - DB::ReadBufferPtr buff = genRandomData(pageId, holder); + DB::ReadBufferPtr buff = genRandomData(page_id, holder); WriteBatch wb; - wb.putPage(pageId, 0, buff, buff->buffer().size()); + wb.putPage(page_id, 0, buff, buff->buffer().size()); storage->write(std::move(wb)); - if (pageId % 100 == 0) - LOG_INFO(&Poco::Logger::get("root"), "writer wrote page" + DB::toString(pageId)); + if (page_id % 100 == 0) + LOG_INFO(&Poco::Logger::get("root"), "writer wrote page" + DB::toString(page_id)); } } @@ -160,13 +160,13 @@ class PSWriter : public Poco::Runnable { assert(storage != nullptr); std::normal_distribution<> d{ctx.MAX_PAGE_ID / 2.0, 150}; - const PageId pageId = static_cast(std::round(d(gen))) % ctx.MAX_PAGE_ID; + const PageId page_id = static_cast(std::round(d(gen))) % ctx.MAX_PAGE_ID; MemHolder holder; - DB::ReadBufferPtr buff = genRandomData(pageId, holder); + DB::ReadBufferPtr buff = genRandomData(page_id, holder); WriteBatch wb; - wb.putPage(pageId, 0, buff, buff->buffer().size()); + wb.putPage(page_id, 0, buff, buff->buffer().size()); storage->write(std::move(wb)); ++pages_written; bytes_written += buff->buffer().size(); @@ -283,7 +283,7 @@ class PSGc struct StressTimeout { TestContext & ctx; - StressTimeout(TestContext & ctx_) + explicit StressTimeout(TestContext & ctx_) : ctx(ctx_) {} void onTime(Poco::Timer & /* t */) diff --git a/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.cpp b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.cpp index 10f95b83bcc..1b4d22a8397 100644 --- a/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.cpp +++ b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -173,8 +174,9 @@ try return 0; } + auto bkg_pool = std::make_shared(4, "bg-page-"); DB::PageStorageConfig config = parse_storage_config(argc, argv, logger); - PageStorage storage("PageCtl", delegator, config, file_provider); + PageStorage storage("PageCtl", delegator, config, file_provider, *bkg_pool); storage.restore(); switch (mode) { diff --git a/dbms/src/Storages/Page/workload/PSWorkload.cpp b/dbms/src/Storages/Page/workload/PSWorkload.cpp index 009106621f7..df8ffb45c3f 100644 --- a/dbms/src/Storages/Page/workload/PSWorkload.cpp +++ b/dbms/src/Storages/Page/workload/PSWorkload.cpp @@ -116,7 +116,8 @@ void StressWorkload::initPageStorage(DB::PageStorageConfig & config, String path if (options.running_ps_version == 2) { - ps = std::make_shared("stress_test", delegator, config, file_provider); + bkg_pool = std::make_shared(4, "bg-page-"); + ps = std::make_shared("stress_test", delegator, config, file_provider, *bkg_pool); } else if (options.running_ps_version == 3) { diff --git a/dbms/src/Storages/Page/workload/PSWorkload.h b/dbms/src/Storages/Page/workload/PSWorkload.h index 35248b6188b..746cb6c2262 100644 --- a/dbms/src/Storages/Page/workload/PSWorkload.h +++ b/dbms/src/Storages/Page/workload/PSWorkload.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -121,6 +122,7 @@ class StressWorkload StressEnv options; Poco::ThreadPool pool; + std::shared_ptr bkg_pool; PSPtr ps; DB::PSDiskDelegatorPtr delegator; diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index a8b99e6e3b4..0aa55cf74da 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -243,7 +243,8 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH "RegionPersister", delegator, config, - provider); + provider, + global_context.getPSBackgroundPool()); page_storage_v2->restore(); page_writer = std::make_shared(global_run_mode, page_storage_v2, /*storage_v3_*/ nullptr); page_reader = std::make_shared(global_run_mode, ns_id, page_storage_v2, /*storage_v3_*/ nullptr, /*readlimiter*/ global_context.getReadLimiter()); @@ -282,7 +283,8 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH "RegionPersister", delegator, PageStorage::getEasyGCConfig(), - provider); + provider, + global_context.getPSBackgroundPool()); // V3 should not used getPSDiskDelegatorRaft // Because V2 will delete all invalid(unrecognized) file when it restore auto page_storage_v3 = std::make_shared( // diff --git a/metrics/grafana/tiflash_proxy_details.json b/metrics/grafana/tiflash_proxy_details.json index bc172a55f64..f33fdb0c0df 100644 --- a/metrics/grafana/tiflash_proxy_details.json +++ b/metrics/grafana/tiflash_proxy_details.json @@ -14,7 +14,7 @@ "type": "grafana", "id": "grafana", "name": "Grafana", - "version": "6.1.6" + "version": "7.5.11" }, { "type": "panel", @@ -52,11 +52,12 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1577960059869, + "iteration": 1670499325053, "links": [], "panels": [ { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -75,7 +76,12 @@ "description": "The CPU usage of each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -83,6 +89,7 @@ "x": 0, "y": 1 }, + "hiddenSeries": false, "id": 1708, "legend": { "alignAsTable": true, @@ -102,7 +109,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -172,7 +183,12 @@ "description": "The memory usage per TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -180,6 +196,7 @@ "x": 12, "y": 1 }, + "hiddenSeries": false, "id": 1709, "legend": { "alignAsTable": true, @@ -199,7 +216,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -269,7 +290,12 @@ "description": "The I/O utilization per TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -277,6 +303,7 @@ "x": 0, "y": 9 }, + "hiddenSeries": false, "id": 1710, "legend": { "alignAsTable": true, @@ -296,7 +323,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -366,7 +397,12 @@ "description": "TiKV uptime since the last restart", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -374,6 +410,7 @@ "x": 12, "y": 9 }, + "hiddenSeries": false, "id": 4106, "legend": { "alignAsTable": true, @@ -393,7 +430,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -463,7 +504,12 @@ "description": " \tThe number of leaders on each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -471,6 +517,7 @@ "x": 0, "y": 17 }, + "hiddenSeries": false, "id": 1715, "legend": { "alignAsTable": true, @@ -490,7 +537,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -574,7 +625,12 @@ "description": "The number of Regions on each TiKV instance", "editable": true, "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "grid": {}, "gridPos": { "h": 8, @@ -582,6 +638,7 @@ "x": 12, "y": 17 }, + "hiddenSeries": false, "id": 1714, "legend": { "alignAsTable": true, @@ -601,7 +658,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -660,6 +721,115 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The memory usage of raft entry cache per TiFlash instance", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 25 + }, + "hiddenSeries": false, + "id": 4536, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tiflash_proxy_tikv_server_mem_trace_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"raftstore-.*\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{name}} {{instance}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Entry Cache", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, @@ -668,6 +838,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -1688,6 +1859,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -2925,6 +3097,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -3537,6 +3710,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -5096,6 +5270,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -5484,6 +5659,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -6103,6 +6279,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -6434,6 +6611,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -7028,6 +7206,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -8037,6 +8216,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -8445,6 +8625,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -8725,6 +8906,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -9167,6 +9349,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -9622,6 +9805,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -10675,6 +10859,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -11189,6 +11374,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -11600,6 +11786,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -11996,6 +12183,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -16184,6 +16372,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -16663,40 +16852,54 @@ "list": [ { "allValue": null, - "current": { }, + "current": {}, "datasource": "${DS_TEST-CLUSTER}", + "definition": "", + "description": null, + "error": null, "hide": 2, "includeAll": false, "label": "K8s-cluster", "multi": false, "name": "k8s_cluster", - "options": [ ], - "query": "label_values(tiflash_proxy_tikv_engine_block_cache_size_bytes, k8s_cluster)", + "options": [], + "query": { + "query": "label_values(tiflash_proxy_tikv_engine_block_cache_size_bytes, k8s_cluster)", + "refId": "ldz-test-k8s_cluster-Variable-Query" + }, "refresh": 2, "regex": "", + "skipUrlSync": false, "sort": 1, "tagValuesQuery": "", - "tags": [ ], + "tags": [], "tagsQuery": "", "type": "query", "useTags": false }, { "allValue": null, - "current": { }, + "current": {}, "datasource": "${DS_TEST-CLUSTER}", + "definition": "", + "description": null, + "error": null, "hide": 2, "includeAll": false, "label": "tidb_cluster", "multi": false, "name": "tidb_cluster", - "options": [ ], - "query": "label_values(tiflash_proxy_tikv_engine_block_cache_size_bytes{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", + "options": [], + "query": { + "query": "label_values(tiflash_proxy_tikv_engine_block_cache_size_bytes{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", + "refId": "ldz-test-tidb_cluster-Variable-Query" + }, "refresh": 2, "regex": "", + "skipUrlSync": false, "sort": 1, "tagValuesQuery": "", - "tags": [ ], + "tags": [], "tagsQuery": "", "type": "query", "useTags": false @@ -16706,13 +16909,18 @@ "current": {}, "datasource": "${DS_TEST-CLUSTER}", "definition": "", + "description": null, + "error": null, "hide": 0, "includeAll": true, "label": "db", "multi": true, "name": "db", "options": [], - "query": "label_values(tiflash_proxy_tikv_engine_block_cache_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, db)", + "query": { + "query": "label_values(tiflash_proxy_tikv_engine_block_cache_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, db)", + "refId": "ldz-test-db-Variable-Query" + }, "refresh": 1, "regex": "", "skipUrlSync": false, @@ -16728,13 +16936,18 @@ "current": {}, "datasource": "${DS_TEST-CLUSTER}", "definition": "", + "description": null, + "error": null, "hide": 0, "includeAll": true, "label": "command", "multi": true, "name": "command", "options": [], - "query": "label_values(tiflash_proxy_tikv_storage_command_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, type)", + "query": { + "query": "label_values(tiflash_proxy_tikv_storage_command_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, type)", + "refId": "ldz-test-command-Variable-Query" + }, "refresh": 1, "regex": "prewrite|commit|rollback", "skipUrlSync": false, @@ -16750,13 +16963,18 @@ "current": {}, "datasource": "${DS_TEST-CLUSTER}", "definition": "", + "description": null, + "error": null, "hide": 0, "includeAll": true, "label": "Instance", "multi": false, "name": "instance", "options": [], - "query": "label_values(tiflash_proxy_tikv_engine_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, instance)", + "query": { + "query": "label_values(tiflash_proxy_tikv_engine_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, instance)", + "refId": "ldz-test-instance-Variable-Query" + }, "refresh": 1, "regex": "", "skipUrlSync": false, From d8c369cb8979efeda8df9ef6b67dfd9c38a7d9c9 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 8 Dec 2022 22:46:07 +0800 Subject: [PATCH 12/42] Support disaggregated tiflash mode (#6248) close pingcap/tiflash#6441 --- contrib/client-c | 2 +- dbms/src/Core/TiFlashDisaggregatedMode.cpp | 57 ++++ dbms/src/Core/TiFlashDisaggregatedMode.h | 37 +++ dbms/src/Debug/dbgQueryExecutor.cpp | 2 +- dbms/src/Flash/BatchCoprocessorHandler.cpp | 2 + .../src/Flash/Coprocessor/CoprocessorReader.h | 2 +- dbms/src/Flash/Coprocessor/DAGContext.h | 12 + .../Coprocessor/DAGQueryBlockInterpreter.cpp | 21 +- .../Coprocessor/DAGStorageInterpreter.cpp | 55 +--- .../Flash/Coprocessor/DAGStorageInterpreter.h | 8 +- .../Flash/Coprocessor/FineGrainedShuffle.h | 2 +- .../Flash/Coprocessor/GenSchemaAndColumn.cpp | 22 ++ .../Flash/Coprocessor/GenSchemaAndColumn.h | 4 + .../Flash/Coprocessor/InterpreterUtils.cpp | 51 +++ dbms/src/Flash/Coprocessor/InterpreterUtils.h | 13 + dbms/src/Flash/Coprocessor/RemoteRequest.cpp | 36 ++- dbms/src/Flash/Coprocessor/RemoteRequest.h | 12 +- .../StorageDisaggregatedInterpreter.h | 57 ++++ .../Flash/Coprocessor/TablesRegionsInfo.cpp | 5 + dbms/src/Flash/Coprocessor/TiDBTableScan.h | 5 + .../gtest_ti_remote_block_inputstream.cpp | 11 + dbms/src/Flash/CoprocessorHandler.cpp | 2 + dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 12 +- dbms/src/Flash/Mpp/ExchangeReceiver.h | 12 +- dbms/src/Flash/Mpp/GRPCReceiverContext.cpp | 111 ++++++- dbms/src/Flash/Mpp/GRPCReceiverContext.h | 17 + dbms/src/Flash/Mpp/MPPHandler.cpp | 16 +- dbms/src/Flash/Mpp/MPPTask.cpp | 3 + .../Flash/Planner/plans/PhysicalTableScan.cpp | 21 +- .../Flash/Planner/plans/PhysicalTableScan.h | 1 + dbms/src/Flash/Statistics/TableScanImpl.cpp | 23 +- dbms/src/Flash/Statistics/TableScanImpl.h | 4 +- dbms/src/Flash/tests/gtest_interpreter.cpp | 1 - .../tests/gtest_storage_disaggregated.cpp | 117 +++++++ dbms/src/Interpreters/Context.cpp | 1 + dbms/src/Interpreters/Context.h | 15 + dbms/src/Server/Server.cpp | 55 ++-- dbms/src/Storages/StorageDisaggregated.cpp | 291 ++++++++++++++++++ dbms/src/Storages/StorageDisaggregated.h | 96 ++++++ dbms/src/Storages/Transaction/TMTContext.cpp | 7 +- dbms/src/Storages/Transaction/TMTContext.h | 4 +- etc/config-template.toml | 1 + 42 files changed, 1094 insertions(+), 132 deletions(-) create mode 100644 dbms/src/Core/TiFlashDisaggregatedMode.cpp create mode 100644 dbms/src/Core/TiFlashDisaggregatedMode.h create mode 100644 dbms/src/Flash/Coprocessor/StorageDisaggregatedInterpreter.h create mode 100644 dbms/src/Flash/tests/gtest_storage_disaggregated.cpp create mode 100644 dbms/src/Storages/StorageDisaggregated.cpp create mode 100644 dbms/src/Storages/StorageDisaggregated.h diff --git a/contrib/client-c b/contrib/client-c index 425148c0392..d6cc312f69d 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 425148c03929f4c960e50ed2c877f6695b105278 +Subproject commit d6cc312f69da3f09ac74cc9db3742be9cfd62595 diff --git a/dbms/src/Core/TiFlashDisaggregatedMode.cpp b/dbms/src/Core/TiFlashDisaggregatedMode.cpp new file mode 100644 index 00000000000..af37ba3ae39 --- /dev/null +++ b/dbms/src/Core/TiFlashDisaggregatedMode.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 +{ +DisaggregatedMode getDisaggregatedMode(const Poco::Util::LayeredConfiguration & config) +{ + static const std::string config_key = "flash.disaggregated_mode"; + DisaggregatedMode mode = DisaggregatedMode::None; + if (config.has(config_key)) + { + std::string mode_str = config.getString(config_key); + RUNTIME_ASSERT(mode_str == DISAGGREGATED_MODE_STORAGE || mode_str == DISAGGREGATED_MODE_COMPUTE, + "Expect disaggregated_mode is {} or {}, got: {}", + DISAGGREGATED_MODE_STORAGE, + DISAGGREGATED_MODE_COMPUTE, + mode_str); + if (mode_str == DISAGGREGATED_MODE_COMPUTE) + { + mode = DisaggregatedMode::Compute; + } + else + { + mode = DisaggregatedMode::Storage; + } + } + return mode; +} + +std::string getProxyLabelByDisaggregatedMode(DisaggregatedMode mode) +{ + switch (mode) + { + case DisaggregatedMode::Compute: + return DISAGGREGATED_MODE_COMPUTE_PROXY_LABEL; + case DisaggregatedMode::Storage: + case DisaggregatedMode::None: + return DEF_PROXY_LABEL; + default: + __builtin_unreachable(); + }; +} +} // namespace DB diff --git a/dbms/src/Core/TiFlashDisaggregatedMode.h b/dbms/src/Core/TiFlashDisaggregatedMode.h new file mode 100644 index 00000000000..eb53e9d5391 --- /dev/null +++ b/dbms/src/Core/TiFlashDisaggregatedMode.h @@ -0,0 +1,37 @@ +// 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. + +#pragma once + +#include + +#include + +#define DEF_PROXY_LABEL "tiflash" +#define DISAGGREGATED_MODE_COMPUTE_PROXY_LABEL DISAGGREGATED_MODE_COMPUTE +#define DISAGGREGATED_MODE_STORAGE "tiflash_storage" +#define DISAGGREGATED_MODE_COMPUTE "tiflash_compute" + +namespace DB +{ +enum class DisaggregatedMode +{ + None, + Compute, + Storage, +}; + +DisaggregatedMode getDisaggregatedMode(const Poco::Util::LayeredConfiguration & config); +std::string getProxyLabelByDisaggregatedMode(DisaggregatedMode mode); +} // namespace DB diff --git a/dbms/src/Debug/dbgQueryExecutor.cpp b/dbms/src/Debug/dbgQueryExecutor.cpp index 579c9adbacb..38656747664 100644 --- a/dbms/src/Debug/dbgQueryExecutor.cpp +++ b/dbms/src/Debug/dbgQueryExecutor.cpp @@ -333,4 +333,4 @@ bool runAndCompareDagReq(const coprocessor::Request & req, const coprocessor::Re } return unequal_flag; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/BatchCoprocessorHandler.cpp b/dbms/src/Flash/BatchCoprocessorHandler.cpp index b7dbcf153a1..a25c14dc84e 100644 --- a/dbms/src/Flash/BatchCoprocessorHandler.cpp +++ b/dbms/src/Flash/BatchCoprocessorHandler.cpp @@ -48,6 +48,8 @@ grpc::Status BatchCoprocessorHandler::execute() try { + RUNTIME_CHECK_MSG(!cop_context.db_context.isDisaggregatedComputeMode(), "cannot run cop or batchCop request on tiflash_compute node"); + switch (cop_request->tp()) { case COP_REQ_TYPE_DAG: diff --git a/dbms/src/Flash/Coprocessor/CoprocessorReader.h b/dbms/src/Flash/Coprocessor/CoprocessorReader.h index e1e41b8fbab..5c0a05345eb 100644 --- a/dbms/src/Flash/Coprocessor/CoprocessorReader.h +++ b/dbms/src/Flash/Coprocessor/CoprocessorReader.h @@ -80,7 +80,7 @@ class CoprocessorReader CoprocessorReader( const DAGSchema & schema_, pingcap::kv::Cluster * cluster, - std::vector tasks, + std::vector tasks, bool has_enforce_encode_type_, int concurrency) : schema(schema_) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 1aaf583db14..48472665578 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -313,6 +313,15 @@ class DAGContext } void addCoprocessorReader(const CoprocessorReaderPtr & coprocessor_reader); std::vector & getCoprocessorReaders(); + void setDisaggregatedComputeExchangeReceiver(const String & executor_id, const ExchangeReceiverPtr & receiver) + { + disaggregated_compute_exchange_receiver = std::make_pair(executor_id, receiver); + } + std::optional> getDisaggregatedComputeExchangeReceiver() + { + return disaggregated_compute_exchange_receiver; + } + void addSubquery(const String & subquery_id, SubqueryForSet && subquery); bool hasSubquery() const { return !subqueries.empty(); } @@ -395,6 +404,9 @@ class DAGContext /// vector of SubqueriesForSets(such as join build subquery). /// The order of the vector is also the order of the subquery. std::vector subqueries; + // In disaggregated tiflash mode, table_scan in tiflash_compute node will be converted ExchangeReceiver. + // Record here so we can add to receiver_set and cancel/close it. + std::optional> disaggregated_compute_exchange_receiver; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index a88fc1503b3..db7f2a0f9ae 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -43,12 +43,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include namespace DB { @@ -98,8 +100,8 @@ AnalysisResult analyzeExpressions( { AnalysisResult res; ExpressionActionsChain chain; - // selection on table scan had been executed in handleTableScan - // In test mode, filter is not pushed down to table scan + // selection on table scan had been executed in handleTableScan. + // In test mode, filter is not pushed down to table scan. if (query_block.selection && (!query_block.isTableScanSource() || context.isTest())) { std::vector where_conditions; @@ -185,10 +187,19 @@ void DAGQueryBlockInterpreter::handleTableScan(const TiDBTableScan & table_scan, { const auto push_down_filter = PushDownFilter::pushDownFilterFrom(query_block.selection_name, query_block.selection); - DAGStorageInterpreter storage_interpreter(context, table_scan, push_down_filter, max_streams); - storage_interpreter.execute(pipeline); + if (context.isDisaggregatedComputeMode()) + { + StorageDisaggregatedInterpreter disaggregated_tiflash_interpreter(context, table_scan, push_down_filter, max_streams); + disaggregated_tiflash_interpreter.execute(pipeline); + analyzer = std::move(disaggregated_tiflash_interpreter.analyzer); + } + else + { + DAGStorageInterpreter storage_interpreter(context, table_scan, push_down_filter, max_streams); + storage_interpreter.execute(pipeline); - analyzer = std::move(storage_interpreter.analyzer); + analyzer = std::move(storage_interpreter.analyzer); + } } void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & pipeline, SubqueryForSet & right_query, size_t fine_grained_shuffle_count) diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 4f6ed972235..3e36418bef5 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -380,7 +380,7 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) /// handle pushed down filter for local and remote table scan. if (push_down_filter.hasValue()) { - executePushedDownFilter(remote_read_streams_start_index, pipeline); + ::DB::executePushedDownFilter(remote_read_streams_start_index, push_down_filter, *analyzer, log, pipeline); recordProfileStreams(pipeline, push_down_filter.executor_id); } } @@ -419,51 +419,6 @@ void DAGStorageInterpreter::prepare() analyzer = std::make_unique(std::move(source_columns), context); } -std::tuple DAGStorageInterpreter::buildPushDownFilter() -{ - assert(push_down_filter.hasValue()); - - ExpressionActionsChain chain; - analyzer->initChain(chain, analyzer->getCurrentInputColumns()); - String filter_column_name = analyzer->appendWhere(chain, push_down_filter.conditions); - ExpressionActionsPtr before_where = chain.getLastActions(); - chain.addStep(); - - // remove useless tmp column and keep the schema of local streams and remote streams the same. - NamesWithAliases project_cols; - for (const auto & col : analyzer->getCurrentInputColumns()) - { - chain.getLastStep().required_output.push_back(col.name); - project_cols.emplace_back(col.name, col.name); - } - chain.getLastActions()->add(ExpressionAction::project(project_cols)); - ExpressionActionsPtr project_after_where = chain.getLastActions(); - chain.finalize(); - chain.clear(); - - return {before_where, filter_column_name, project_after_where}; -} - -void DAGStorageInterpreter::executePushedDownFilter( - size_t remote_read_streams_start_index, - DAGPipeline & pipeline) -{ - auto [before_where, filter_column_name, project_after_where] = buildPushDownFilter(); - - assert(pipeline.streams_with_non_joined_data.empty()); - assert(remote_read_streams_start_index <= pipeline.streams.size()); - // for remote read, filter had been pushed down, don't need to execute again. - for (size_t i = 0; i < remote_read_streams_start_index; ++i) - { - auto & stream = pipeline.streams[i]; - stream = std::make_shared(stream, before_where, filter_column_name, log->identifier()); - stream->setExtraInfo("push down filter"); - // after filter, do project action to keep the schema of local streams and remote streams the same. - stream = std::make_shared(stream, project_after_where, log->identifier()); - stream->setExtraInfo("projection after push down filter"); - } -} - void DAGStorageInterpreter::executeCastAfterTableScan( size_t remote_read_streams_start_index, DAGPipeline & pipeline) @@ -492,7 +447,7 @@ void DAGStorageInterpreter::executeCastAfterTableScan( } } -std::vector DAGStorageInterpreter::buildCopTasks(const std::vector & remote_requests) +std::vector DAGStorageInterpreter::buildCopTasks(const std::vector & remote_requests) { assert(!remote_requests.empty()); #ifndef NDEBUG @@ -514,7 +469,7 @@ std::vector DAGStorageInterpreter::buildCopTasks( } #endif pingcap::kv::Cluster * cluster = tmt.getKVCluster(); - std::vector all_tasks; + std::vector all_tasks; for (const auto & remote_request : remote_requests) { pingcap::coprocessor::RequestPtr req = std::make_shared(); @@ -539,7 +494,7 @@ std::vector DAGStorageInterpreter::buildCopTasks( void DAGStorageInterpreter::buildRemoteStreams(const std::vector & remote_requests, DAGPipeline & pipeline) { - std::vector all_tasks = buildCopTasks(remote_requests); + std::vector all_tasks = buildCopTasks(remote_requests); const DAGSchema & schema = remote_requests[0].schema; pingcap::kv::Cluster * cluster = tmt.getKVCluster(); @@ -554,7 +509,7 @@ void DAGStorageInterpreter::buildRemoteStreams(const std::vector task_end++; if (task_end == task_start) continue; - std::vector tasks(all_tasks.begin() + task_start, all_tasks.begin() + task_end); + std::vector tasks(all_tasks.begin() + task_start, all_tasks.begin() + task_end); auto coprocessor_reader = std::make_shared(schema, cluster, tasks, has_enforce_encode_type, 1); context.getDAGContext()->addCoprocessorReader(coprocessor_reader); diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index efe78c25918..aa026461e31 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -91,19 +91,13 @@ class DAGStorageInterpreter void recordProfileStreams(DAGPipeline & pipeline, const String & key); - std::vector buildCopTasks(const std::vector & remote_requests); + std::vector buildCopTasks(const std::vector & remote_requests); void buildRemoteStreams(const std::vector & remote_requests, DAGPipeline & pipeline); void executeCastAfterTableScan( size_t remote_read_streams_start_index, DAGPipeline & pipeline); - // before_where, filter_column_name, after_where - std::tuple buildPushDownFilter(); - void executePushedDownFilter( - size_t remote_read_streams_start_index, - DAGPipeline & pipeline); - void prepare(); void executeImpl(DAGPipeline & pipeline); diff --git a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h index 6f794375a30..4d3583c47b3 100644 --- a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h +++ b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h @@ -42,4 +42,4 @@ struct FineGrainedShuffle const UInt64 stream_count; const UInt64 batch_size; }; -} // 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 f638190b47c..7ab505233d8 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -34,6 +34,28 @@ DataTypePtr getPkType(const ColumnInfo & column_info) } } // namespace +NamesAndTypes genNamesAndTypesForTableScan(const TiDBTableScan & table_scan) +{ + return genNamesAndTypes(table_scan, "table_scan"); +} + +NamesAndTypes genNamesAndTypesForExchangeReceiver(const TiDBTableScan & table_scan) +{ + NamesAndTypes names_and_types; + names_and_types.reserve(table_scan.getColumnSize()); + for (Int32 i = 0; i < table_scan.getColumnSize(); ++i) + { + const auto & column_info = table_scan.getColumns()[i]; + names_and_types.emplace_back(genNameForExchangeReceiver(i), getDataTypeByColumnInfoForComputingLayer(column_info)); + } + return names_and_types; +} + +String genNameForExchangeReceiver(Int32 col_index) +{ + return "exchange_receiver_" + std::to_string(col_index); +} + NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan, const StringRef & column_prefix) { NamesAndTypes names_and_types; diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h index cdd032b3c8c..7c6d08113a4 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h @@ -23,6 +23,10 @@ namespace DB { +NamesAndTypes genNamesAndTypesForExchangeReceiver(const TiDBTableScan & table_scan); +NamesAndTypes genNamesAndTypesForTableScan(const TiDBTableScan & table_scan); +String genNameForExchangeReceiver(Int32 col_index); + NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan, const StringRef & column_prefix); ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_types); NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema); diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index d71f8b073d1..899362521cb 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -194,4 +195,54 @@ void executeCreatingSets( log->identifier()); } } + +std::tuple buildPushDownFilter( + const PushDownFilter & push_down_filter, + DAGExpressionAnalyzer & analyzer) +{ + assert(push_down_filter.hasValue()); + + ExpressionActionsChain chain; + analyzer.initChain(chain, analyzer.getCurrentInputColumns()); + String filter_column_name = analyzer.appendWhere(chain, push_down_filter.conditions); + ExpressionActionsPtr before_where = chain.getLastActions(); + chain.addStep(); + + // remove useless tmp column and keep the schema of local streams and remote streams the same. + NamesWithAliases project_cols; + for (const auto & col : analyzer.getCurrentInputColumns()) + { + chain.getLastStep().required_output.push_back(col.name); + project_cols.emplace_back(col.name, col.name); + } + chain.getLastActions()->add(ExpressionAction::project(project_cols)); + ExpressionActionsPtr project_after_where = chain.getLastActions(); + chain.finalize(); + chain.clear(); + + return {before_where, filter_column_name, project_after_where}; +} + +void executePushedDownFilter( + size_t remote_read_streams_start_index, + const PushDownFilter & push_down_filter, + DAGExpressionAnalyzer & analyzer, + LoggerPtr log, + DAGPipeline & pipeline) +{ + auto [before_where, filter_column_name, project_after_where] = ::DB::buildPushDownFilter(push_down_filter, analyzer); + + assert(pipeline.streams_with_non_joined_data.empty()); + assert(remote_read_streams_start_index <= pipeline.streams.size()); + // for remote read, filter had been pushed down, don't need to execute again. + for (size_t i = 0; i < remote_read_streams_start_index; ++i) + { + auto & stream = pipeline.streams[i]; + stream = std::make_shared(stream, before_where, filter_column_name, log->identifier()); + stream->setExtraInfo("push down filter"); + // after filter, do project action to keep the schema of local streams and remote streams the same. + stream = std::make_shared(stream, project_after_where, log->identifier()); + stream->setExtraInfo("projection after push down filter"); + } +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 87672e81dfa..66f86de4c40 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -16,7 +16,9 @@ #include #include +#include #include +#include #include namespace DB @@ -66,4 +68,15 @@ void executeCreatingSets( const Context & context, size_t max_streams, const LoggerPtr & log); + +std::tuple buildPushDownFilter( + const PushDownFilter & push_down_filter, + DAGExpressionAnalyzer & analyzer); + +void executePushedDownFilter( + size_t remote_read_streams_start_index, + const PushDownFilter & push_down_filter, + DAGExpressionAnalyzer & analyzer, + LoggerPtr log, + DAGPipeline & pipeline); } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp index e0111641a4f..6a0766f900e 100644 --- a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp @@ -28,18 +28,7 @@ RemoteRequest RemoteRequest::build( const PushDownFilter & push_down_filter, const LoggerPtr & log) { - auto print_retry_regions = [&retry_regions, &table_info] { - FmtBuffer buffer; - buffer.fmtAppend("Start to build remote request for {} regions (", retry_regions.size()); - buffer.joinStr( - retry_regions.cbegin(), - retry_regions.cend(), - [](const auto & r, FmtBuffer & fb) { fb.fmtAppend("{}", r.get().region_id); }, - ","); - buffer.fmtAppend(") for table {}", table_info.id); - return buffer.toString(); - }; - LOG_INFO(log, "{}", print_retry_regions()); + LOG_INFO(log, "{}", printRetryRegions(retry_regions, table_info.id)); DAGSchema schema; tipb::DAGRequest dag_req; @@ -94,6 +83,13 @@ RemoteRequest RemoteRequest::build( dag_req.set_time_zone_name(original_dag_req.time_zone_name()); if (original_dag_req.has_time_zone_offset()) dag_req.set_time_zone_offset(original_dag_req.time_zone_offset()); + + std::vector key_ranges = buildKeyRanges(retry_regions); + return {std::move(dag_req), std::move(schema), std::move(key_ranges)}; +} + +std::vector RemoteRequest::buildKeyRanges(const RegionRetryList & retry_regions) +{ std::vector key_ranges; for (const auto & region : retry_regions) { @@ -101,6 +97,20 @@ RemoteRequest RemoteRequest::build( key_ranges.emplace_back(*range.first, *range.second); } sort(key_ranges.begin(), key_ranges.end()); - return {std::move(dag_req), std::move(schema), std::move(key_ranges)}; + return key_ranges; } + +std::string RemoteRequest::printRetryRegions(const RegionRetryList & retry_regions, TableID table_id) +{ + FmtBuffer buffer; + buffer.fmtAppend("Start to build remote request for {} regions (", retry_regions.size()); + buffer.joinStr( + retry_regions.cbegin(), + retry_regions.cend(), + [](const auto & r, FmtBuffer & fb) { fb.fmtAppend("{}", r.get().region_id); }, + ","); + buffer.fmtAppend(") for table {}", table_id); + return buffer.toString(); +} + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.h b/dbms/src/Flash/Coprocessor/RemoteRequest.h index ce78e0b62fe..ad6d425010c 100644 --- a/dbms/src/Flash/Coprocessor/RemoteRequest.h +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.h @@ -44,10 +44,7 @@ struct RemoteRequest , schema(std::move(schema_)) , key_ranges(std::move(key_ranges_)) {} - tipb::DAGRequest dag_request; - DAGSchema schema; - /// the sorted key ranges - std::vector key_ranges; + static RemoteRequest build( const RegionRetryList & retry_regions, DAGContext & dag_context, @@ -55,5 +52,12 @@ struct RemoteRequest const TiDB::TableInfo & table_info, const PushDownFilter & push_down_filter, const LoggerPtr & log); + static std::vector buildKeyRanges(const RegionRetryList & retry_regions); + static std::string printRetryRegions(const RegionRetryList & retry_regions, TableID table_id); + + tipb::DAGRequest dag_request; + DAGSchema schema; + /// the sorted key ranges + std::vector key_ranges; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/StorageDisaggregatedInterpreter.h b/dbms/src/Flash/Coprocessor/StorageDisaggregatedInterpreter.h new file mode 100644 index 00000000000..2818baeac67 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/StorageDisaggregatedInterpreter.h @@ -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. + +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +// For TableScan in disaggregated tiflash mode, +// we convert it to ExchangeReceiver(executed in tiflash_compute node), +// and ExchangeSender + TableScan(executed in tiflash_storage node). +class StorageDisaggregatedInterpreter +{ +public: + StorageDisaggregatedInterpreter( + Context & context_, + const TiDBTableScan & table_scan_, + const PushDownFilter & push_down_filter_, + size_t max_streams_) + : context(context_) + , storage(std::make_unique(context_, table_scan_, push_down_filter_)) + , max_streams(max_streams_) + {} + + void execute(DAGPipeline & pipeline) + { + auto stage = QueryProcessingStage::Enum::FetchColumns; + pipeline.streams = storage->read(Names(), SelectQueryInfo(), context, stage, 0, max_streams); + analyzer = std::move(storage->analyzer); + } + + // Members will be transferred to DAGQueryBlockInterpreter after execute + std::unique_ptr analyzer; + +private: + Context & context; + std::unique_ptr storage; + size_t max_streams; +}; +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp index b61a19c3177..5b3de63c7b9 100644 --- a/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp +++ b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -47,6 +48,10 @@ const SingleTableRegions & TablesRegionsInfo::getTableRegionInfoByTableID(Int64 static bool needRemoteRead(const RegionInfo & region_info, const TMTContext & tmt_context) { fiu_do_on(FailPoints::force_no_local_region_for_mpp_task, { return true; }); + // For tiflash_compute node, all regions will be fetched from tiflash_storage node. + // So treat all regions as remote regions. + if (tmt_context.getContext().isDisaggregatedComputeMode()) + return true; RegionPtr current_region = tmt_context.getKVStore()->getRegion(region_info.region_id); if (current_region == nullptr || current_region->peerState() != raft_serverpb::PeerState::Normal) return true; diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.h b/dbms/src/Flash/Coprocessor/TiDBTableScan.h index 84e6f41581f..11c5b974476 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.h +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.h @@ -64,6 +64,11 @@ class TiDBTableScan return is_fast_scan; } + const tipb::Executor * getTableScanPB() const + { + return table_scan; + } + private: const tipb::Executor * table_scan; String executor_id; diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp index e0300ec67aa..2434feeba26 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -210,6 +211,16 @@ struct MockReceiverContext return std::make_shared(queue); } + void cancelMPPTaskOnTiFlashStorageNode(LoggerPtr) + { + throw Exception("cancelMPPTaskOnTiFlashStorageNode not implemented for MockReceiverContext"); + } + + void sendMPPTaskToTiFlashStorageNode(LoggerPtr, const std::vector &) + { + throw Exception("sendMPPTaskToTiFlashStorageNode not implemented for MockReceiverContext"); + } + static Status getStatusOK() { return ::grpc::Status(); diff --git a/dbms/src/Flash/CoprocessorHandler.cpp b/dbms/src/Flash/CoprocessorHandler.cpp index 706314fafb6..f428add83c0 100644 --- a/dbms/src/Flash/CoprocessorHandler.cpp +++ b/dbms/src/Flash/CoprocessorHandler.cpp @@ -73,6 +73,8 @@ grpc::Status CoprocessorHandler::execute() try { + RUNTIME_CHECK_MSG(!cop_context.db_context.isDisaggregatedComputeMode(), "cannot run cop or batchCop request on tiflash_compute node"); + switch (cop_request->tp()) { case COP_REQ_TYPE_DAG: diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 20de428c5f1..adbbad0e48b 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -411,7 +411,8 @@ ExchangeReceiverBase::ExchangeReceiverBase( size_t max_streams_, const String & req_id, const String & executor_id, - uint64_t fine_grained_shuffle_stream_count_) + uint64_t fine_grained_shuffle_stream_count_, + const std::vector & disaggregated_dispatch_reqs_) : rpc_context(std::move(rpc_context_)) , source_num(source_num_) , enable_fine_grained_shuffle_flag(enableFineGrainedShuffle(fine_grained_shuffle_stream_count_)) @@ -422,6 +423,7 @@ ExchangeReceiverBase::ExchangeReceiverBase( , state(ExchangeReceiverState::NORMAL) , exc_log(Logger::get(req_id, executor_id)) , collected(false) + , disaggregated_dispatch_reqs(disaggregated_dispatch_reqs_) { try { @@ -436,6 +438,8 @@ ExchangeReceiverBase::ExchangeReceiverBase( { msg_channels.push_back(std::make_unique>>(max_buffer_size)); } + if (isReceiverForTiFlashStorage()) + rpc_context->sendMPPTaskToTiFlashStorageNode(exc_log, disaggregated_dispatch_reqs); rpc_context->fillSchema(schema); setUpConnection(); } @@ -471,7 +475,11 @@ ExchangeReceiverBase::~ExchangeReceiverBase() template void ExchangeReceiverBase::cancel() { - setEndState(ExchangeReceiverState::CANCELED); + if (setEndState(ExchangeReceiverState::CANCELED)) + { + if (isReceiverForTiFlashStorage()) + rpc_context->cancelMPPTaskOnTiFlashStorageNode(exc_log); + } cancelAllMsgChannels(); } diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.h b/dbms/src/Flash/Mpp/ExchangeReceiver.h index f1255a355fb..a05bff8a0cc 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.h +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.h @@ -130,7 +130,8 @@ class ExchangeReceiverBase size_t max_streams_, const String & req_id, const String & executor_id, - uint64_t fine_grained_shuffle_stream_count); + uint64_t fine_grained_shuffle_stream_count, + const std::vector & disaggregated_dispatch_reqs_ = {}); ~ExchangeReceiverBase(); @@ -189,6 +190,12 @@ class ExchangeReceiverBase std::unique_ptr & decoder_ptr); private: + bool isReceiverForTiFlashStorage() + { + // If not empty, need to send MPPTask to tiflash_storage. + return !disaggregated_dispatch_reqs.empty(); + } + std::shared_ptr rpc_context; const tipb::ExchangeReceiver pb_exchange_receiver; @@ -213,6 +220,9 @@ class ExchangeReceiverBase bool collected = false; int thread_count = 0; + + // For tiflash_compute node, need to send MPPTask to tiflash_storage node. + std::vector disaggregated_dispatch_reqs; }; class ExchangeReceiver : public ExchangeReceiverBase diff --git a/dbms/src/Flash/Mpp/GRPCReceiverContext.cpp b/dbms/src/Flash/Mpp/GRPCReceiverContext.cpp index 7a16fd70506..8ad03aaff59 100644 --- a/dbms/src/Flash/Mpp/GRPCReceiverContext.cpp +++ b/dbms/src/Flash/Mpp/GRPCReceiverContext.cpp @@ -13,8 +13,10 @@ // limitations under the License. #include +#include #include #include +#include #include #include @@ -221,6 +223,113 @@ ExchangeRecvRequest GRPCReceiverContext::makeRequest(int index) const return req; } +void GRPCReceiverContext::sendMPPTaskToTiFlashStorageNode( + LoggerPtr log, + const std::vector & disaggregated_dispatch_reqs) +{ + if (disaggregated_dispatch_reqs.empty()) + throw Exception("unexpected disaggregated_dispatch_reqs, it's empty."); + + std::shared_ptr thread_manager = newThreadManager(); + for (const StorageDisaggregated::RequestAndRegionIDs & dispatch_req : disaggregated_dispatch_reqs) + { + LOG_DEBUG(log, "tiflash_compute node start to send MPPTask({})", std::get<0>(dispatch_req)->DebugString()); + thread_manager->schedule(/*propagate_memory_tracker=*/false, "", [&dispatch_req, this] { + // When send req succeed or backoff timeout, need_retry is false. + bool need_retry = true; + pingcap::kv::Backoffer bo(pingcap::kv::copNextMaxBackoff); + while (need_retry) + { + try + { + pingcap::kv::RpcCall rpc_call(std::get<0>(dispatch_req)); + this->cluster->rpc_client->sendRequest(std::get<0>(dispatch_req)->meta().address(), rpc_call, /*timeout=*/60); + need_retry = false; + const auto & resp = rpc_call.getResp(); + if (resp->has_error()) + { + this->setDispatchMPPTaskErrMsg(resp->error().msg()); + return; + } + for (const auto & retry_region : resp->retry_regions()) + { + auto region_id = pingcap::kv::RegionVerID( + retry_region.id(), + retry_region.region_epoch().conf_ver(), + retry_region.region_epoch().version()); + this->cluster->region_cache->dropRegion(region_id); + } + } + catch (...) + { + std::string local_err_msg = getCurrentExceptionMessage(true); + try + { + bo.backoff(pingcap::kv::boTiFlashRPC, pingcap::Exception(local_err_msg)); + } + catch (...) + { + need_retry = false; + this->setDispatchMPPTaskErrMsg(local_err_msg); + this->cluster->region_cache->onSendReqFailForBatchRegions(std::get<1>(dispatch_req), std::get<2>(dispatch_req)); + } + } + } + }); + } + + thread_manager->wait(); + + // No need to lock, because all concurrent threads are done. + if (!dispatch_mpp_task_err_msg.empty()) + throw Exception(dispatch_mpp_task_err_msg); +} + +void GRPCReceiverContext::setDispatchMPPTaskErrMsg(const std::string & err) +{ + std::lock_guard lock(dispatch_mpp_task_err_msg_mu); + // Only record first dispatch_mpp_task_err_msg. + if (dispatch_mpp_task_err_msg.empty()) + { + dispatch_mpp_task_err_msg = err; + } +} + +void GRPCReceiverContext::cancelMPPTaskOnTiFlashStorageNode(LoggerPtr log) +{ + auto sender_task_size = exchange_receiver_meta.encoded_task_meta_size(); + auto thread_manager = newThreadManager(); + for (auto i = 0; i < sender_task_size; ++i) + { + auto sender_task = std::make_unique(); + if (unlikely(!sender_task->ParseFromString(exchange_receiver_meta.encoded_task_meta(i)))) + { + LOG_WARNING(log, "parse exchange_receiver_meta.encoded_task_meta failed when canceling MPPTask on tiflash_storage node, will ignore this error"); + return; + } + auto cancel_req = std::make_shared(); + cancel_req->set_allocated_meta(sender_task.release()); + auto rpc_call = std::make_shared>(cancel_req); + thread_manager->schedule(/*propagate_memory_tracker=*/false, "", [cancel_req, log, this] { + try + { + auto rpc_call = pingcap::kv::RpcCall(cancel_req); + // No need to retry. + this->cluster->rpc_client->sendRequest(cancel_req->meta().address(), rpc_call, /*timeout=*/30); + const auto & resp = rpc_call.getResp(); + if (resp->has_error()) + throw Exception(resp->error().msg()); + } + catch (...) + { + String cancel_err_msg = getCurrentExceptionMessage(true); + LOG_WARNING(log, "cancel MPPTasks on tiflash_storage nodes failed: {}. will ignore this error", cancel_err_msg); + } + }); + } + thread_manager->wait(); +} + bool GRPCReceiverContext::supportAsync(const ExchangeRecvRequest & request) const { return enable_async_grpc && !request.is_local; @@ -263,7 +372,7 @@ void GRPCReceiverContext::fillSchema(DAGSchema & schema) const schema.clear(); for (int i = 0; i < exchange_receiver_meta.field_types_size(); ++i) { - String name = "exchange_receiver_" + std::to_string(i); + String name = genNameForExchangeReceiver(i); ColumnInfo info = TiDB::fieldTypeToColumnInfo(exchange_receiver_meta.field_types(i)); schema.emplace_back(std::move(name), std::move(info)); } diff --git a/dbms/src/Flash/Mpp/GRPCReceiverContext.h b/dbms/src/Flash/Mpp/GRPCReceiverContext.h index 59eae4eaaf8..29a7eb169de 100644 --- a/dbms/src/Flash/Mpp/GRPCReceiverContext.h +++ b/dbms/src/Flash/Mpp/GRPCReceiverContext.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -98,12 +99,28 @@ class GRPCReceiverContext void fillSchema(DAGSchema & schema) const; + // Only for tiflash_compute mode, make sure disaggregated_dispatch_reqs is not empty. + void sendMPPTaskToTiFlashStorageNode( + LoggerPtr log, + const std::vector & disaggregated_dispatch_reqs); + + // Normally cancel will be sent by TiDB to all MPPTasks, so ExchangeReceiver no need to cancel. + // But in disaggregated mode, TableScan in tiflash_compute node will be converted to ExchangeReceiver(executed in tiflash_compute node), + // and ExchangeSender+TableScan(executed in tiflash_storage node). + // So when we cancel the former MPPTask, the latter MPPTask needs to be handled by the tiflash_compute node itself. + void cancelMPPTaskOnTiFlashStorageNode(LoggerPtr log); + private: + void setDispatchMPPTaskErrMsg(const std::string & err); + tipb::ExchangeReceiver exchange_receiver_meta; mpp::TaskMeta task_meta; pingcap::kv::Cluster * cluster; std::shared_ptr task_manager; bool enable_local_tunnel; bool enable_async_grpc; + + std::mutex dispatch_mpp_task_err_msg_mu; + String dispatch_mpp_task_err_msg; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPHandler.cpp b/dbms/src/Flash/Mpp/MPPHandler.cpp index a3d14ba8726..f01e2b76ad2 100644 --- a/dbms/src/Flash/Mpp/MPPHandler.cpp +++ b/dbms/src/Flash/Mpp/MPPHandler.cpp @@ -55,14 +55,18 @@ grpc::Status MPPHandler::execute(const ContextPtr & context, mpp::DispatchTaskRe task = MPPTask::newTask(task_request.meta(), context); task->prepare(task_request); - for (const auto & table_region_info : context->getDAGContext()->tables_regions_info.getTableRegionsInfoMap()) + // For tiflash_compute mode, all regions are fetched from remote, so no need to refresh TiDB's region cache. + if (!context->isDisaggregatedComputeMode()) { - for (const auto & region : table_region_info.second.remote_regions) + for (const auto & table_region_info : context->getDAGContext()->tables_regions_info.getTableRegionsInfoMap()) { - auto * retry_region = response->add_retry_regions(); - retry_region->set_id(region.region_id); - retry_region->mutable_region_epoch()->set_conf_ver(region.region_conf_version); - retry_region->mutable_region_epoch()->set_version(region.region_version); + for (const auto & region : table_region_info.second.remote_regions) + { + auto * retry_region = response->add_retry_regions(); + retry_region->set_id(region.region_id); + retry_region->mutable_region_epoch()->set_conf_ver(region.region_conf_version); + retry_region->mutable_region_epoch()->set_version(region.region_version); + } } } if (task->isRootMPPTask()) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index f8f1829545e..990da4c7ad8 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -333,6 +333,9 @@ void MPPTask::preprocess() throw Exception("task not in running state, may be cancelled"); for (auto & r : dag_context->getCoprocessorReaders()) receiver_set->addCoprocessorReader(r); + const auto & receiver_opt = dag_context->getDisaggregatedComputeExchangeReceiver(); + if (receiver_opt.has_value()) + receiver_set->addExchangeReceiver(receiver_opt->first, receiver_opt->second); new_thread_count_of_mpp_receiver += receiver_set->getExternalThreadCnt(); } auto end_time = Clock::now(); diff --git a/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp b/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp index bca67369208..06710f4dc98 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -41,7 +42,7 @@ PhysicalPlanNodePtr PhysicalTableScan::build( const LoggerPtr & log, const TiDBTableScan & table_scan) { - auto schema = genNamesAndTypes(table_scan, "table_scan"); + auto schema = genNamesAndTypesForTableScan(table_scan); auto physical_table_scan = std::make_shared( executor_id, schema, @@ -55,10 +56,22 @@ void PhysicalTableScan::transformImpl(DAGPipeline & pipeline, Context & context, { assert(pipeline.streams.empty() && pipeline.streams_with_non_joined_data.empty()); - DAGStorageInterpreter storage_interpreter(context, tidb_table_scan, push_down_filter, max_streams); - storage_interpreter.execute(pipeline); + if (context.isDisaggregatedComputeMode()) + { + StorageDisaggregatedInterpreter disaggregated_tiflash_interpreter(context, tidb_table_scan, push_down_filter, max_streams); + disaggregated_tiflash_interpreter.execute(pipeline); + buildProjection(context, pipeline, disaggregated_tiflash_interpreter.analyzer->getCurrentInputColumns()); + } + else + { + DAGStorageInterpreter storage_interpreter(context, tidb_table_scan, push_down_filter, max_streams); + storage_interpreter.execute(pipeline); + buildProjection(context, pipeline, storage_interpreter.analyzer->getCurrentInputColumns()); + } +} - const auto & storage_schema = storage_interpreter.analyzer->getCurrentInputColumns(); +void PhysicalTableScan::buildProjection(Context & context, DAGPipeline & pipeline, const NamesAndTypes & storage_schema) +{ RUNTIME_CHECK( storage_schema.size() == schema.size(), storage_schema.size(), diff --git a/dbms/src/Flash/Planner/plans/PhysicalTableScan.h b/dbms/src/Flash/Planner/plans/PhysicalTableScan.h index 947750950e8..4724e1c267a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTableScan.h +++ b/dbms/src/Flash/Planner/plans/PhysicalTableScan.h @@ -46,6 +46,7 @@ class PhysicalTableScan : public PhysicalLeaf private: void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + void buildProjection(Context & context, DAGPipeline & pipeline, const NamesAndTypes & storage_schema); private: PushDownFilter push_down_filter; diff --git a/dbms/src/Flash/Statistics/TableScanImpl.cpp b/dbms/src/Flash/Statistics/TableScanImpl.cpp index fca3fb0e83d..f6242a4a4f0 100644 --- a/dbms/src/Flash/Statistics/TableScanImpl.cpp +++ b/dbms/src/Flash/Statistics/TableScanImpl.cpp @@ -32,7 +32,7 @@ void TableScanStatistics::appendExtraJson(FmtBuffer & fmt_buffer) const fmt_buffer.fmtAppend( R"("connection_details":[{},{}])", local_table_scan_detail.toJson(), - cop_table_scan_detail.toJson()); + remote_table_scan_detail.toJson()); } void TableScanStatistics::collectExtraRuntimeDetail() @@ -43,12 +43,21 @@ void TableScanStatistics::collectExtraRuntimeDetail() { for (const auto & io_stream : it->second) { - if (auto * cop_stream = dynamic_cast(io_stream.get()); cop_stream) + auto * cop_stream = dynamic_cast(io_stream.get()); + /// In tiflash_compute node, TableScan will be converted to ExchangeReceiver. + auto * exchange_stream = dynamic_cast(io_stream.get()); + if (cop_stream || exchange_stream) { - for (const auto & connection_profile_info : cop_stream->getConnectionProfileInfos()) + const std::vector * connection_profile_infos = nullptr; + if (cop_stream) + connection_profile_infos = &cop_stream->getConnectionProfileInfos(); + else if (exchange_stream) + connection_profile_infos = &exchange_stream->getConnectionProfileInfos(); + + for (const auto & connection_profile_info : *connection_profile_infos) { - cop_table_scan_detail.packets += connection_profile_info.packets; - cop_table_scan_detail.bytes += connection_profile_info.bytes; + remote_table_scan_detail.packets += connection_profile_info.packets; + remote_table_scan_detail.bytes += connection_profile_info.bytes; } } else if (auto * local_stream = dynamic_cast(io_stream.get()); local_stream) @@ -56,6 +65,10 @@ void TableScanStatistics::collectExtraRuntimeDetail() /// local read input stream also is IProfilingBlockInputStream local_table_scan_detail.bytes += local_stream->getProfileInfo().bytes; } + else + { + /// Streams like: NullBlockInputStream. + } } } } diff --git a/dbms/src/Flash/Statistics/TableScanImpl.h b/dbms/src/Flash/Statistics/TableScanImpl.h index 409b27bbf27..a68ea84eade 100644 --- a/dbms/src/Flash/Statistics/TableScanImpl.h +++ b/dbms/src/Flash/Statistics/TableScanImpl.h @@ -51,10 +51,10 @@ class TableScanStatistics : public TableScanStatisticsBase private: TableScanDetail local_table_scan_detail{true}; - TableScanDetail cop_table_scan_detail{false}; + TableScanDetail remote_table_scan_detail{false}; protected: void appendExtraJson(FmtBuffer &) const override; void collectExtraRuntimeDetail() override; }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index 784d3a78ded..736166929bc 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -675,6 +675,5 @@ Union: } } CATCH - } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp b/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp new file mode 100644 index 00000000000..c9855e61ec6 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp @@ -0,0 +1,117 @@ +// 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 + +namespace DB +{ +namespace tests +{ + +class StorageDisaggregatedTest : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + db_name = "test_db"; + table_name = "test_table"; + ExecutorTest::initializeContext(); + context.addMockTable({db_name, table_name}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); + } + + String db_name; + String table_name; +}; + +TEST_F(StorageDisaggregatedTest, BasicTest) +try +{ + ::mpp::DispatchTaskRequest dispatch_req; + auto dag_req = context.scan(db_name, table_name).aggregation({Count(col("s1"))}, {}).exchangeSender(tipb::PassThrough).build(context); + const auto & sender = dag_req->root_executor(); + ASSERT_EQ(sender.tp(), ::tipb::TypeExchangeSender); + const auto & hash_agg = sender.exchange_sender().child(); + ASSERT_EQ(hash_agg.tp(), ::tipb::TypeAggregation); + const auto & table_scan = hash_agg.aggregation().child(); + ASSERT_EQ(table_scan.tp(), ::tipb::TypeTableScan); + + // Mock batch_cop_task. + ::pingcap::coprocessor::RegionInfo mock_region_info; + mock_region_info.region_id = pingcap::kv::RegionVerID{100, 1, 1}; + ::pingcap::coprocessor::BatchCopTask mock_batch_cop_task; + mock_batch_cop_task.store_addr = "127.0.0.1:9000"; + mock_batch_cop_task.store_id = 1; + mock_batch_cop_task.region_infos = std::vector<::pingcap::coprocessor::RegionInfo>{mock_region_info}; + + // Mock DispatchTaskRequest.Meta. + auto meta = ::mpp::TaskMeta(); + meta.set_start_ts(437520160532463617); + meta.set_task_id(1); + meta.set_address("127.0.0.1:3930"); + + // Setup dag_context, becase StorageDisaggregated will use it to get ::mpp::TaskMeta. + auto dag_context = std::make_shared(*dag_req, meta, true); + auto * ori_dag_context = TiFlashTestEnv::getGlobalContext().getDAGContext(); + TiFlashTestEnv::getGlobalContext().setDAGContext(dag_context.get()); + TiDBTableScan tidb_table_scan(&table_scan, table_scan.executor_id(), *dag_context); + + PushDownFilter filter; + StorageDisaggregated storage(TiFlashTestEnv::getGlobalContext(), tidb_table_scan, filter); + + uint64_t store_id; + std::vector region_ids; + std::shared_ptr<::mpp::DispatchTaskRequest> tiflash_storage_dispatch_req; + std::tie(tiflash_storage_dispatch_req, region_ids, store_id) = storage.buildDispatchMPPTaskRequest(mock_batch_cop_task); + ASSERT_EQ(region_ids.size(), 1); + ASSERT_EQ(region_ids[0].id, 100); + ASSERT_EQ(store_id, 1); + + // Check if field number of DispatchTaskRequest and DAGRequest is correct. + // In case we add/remove filed but forget to update build processing of StorageDisaggregated. + const auto * dispatch_req_desc = tiflash_storage_dispatch_req->GetDescriptor(); + ASSERT_EQ(dispatch_req_desc->field_count(), 6); + + ::tipb::DAGRequest sender_dag_req; + sender_dag_req.ParseFromString(tiflash_storage_dispatch_req->encoded_plan()); + const auto * sender_dag_req_desc = sender_dag_req.GetDescriptor(); + ASSERT_EQ(sender_dag_req_desc->field_count(), 17); + + const auto & sender1 = sender_dag_req.root_executor(); + ASSERT_EQ(sender1.tp(), ::tipb::TypeExchangeSender); + const auto & table_scan1 = sender1.exchange_sender().child(); + ASSERT_EQ(table_scan1.tp(), ::tipb::TypeTableScan); + + TiFlashTestEnv::getGlobalContext().setDAGContext(ori_dag_context); +} +CATCH + +TEST_F(StorageDisaggregatedTest, LabelTest) +try +{ + ASSERT_EQ(getProxyLabelByDisaggregatedMode(DisaggregatedMode::Compute), "tiflash_compute"); + ASSERT_EQ(getProxyLabelByDisaggregatedMode(DisaggregatedMode::Compute), DISAGGREGATED_MODE_COMPUTE_PROXY_LABEL); + + ASSERT_EQ(getProxyLabelByDisaggregatedMode(DisaggregatedMode::Storage), "tiflash"); + ASSERT_EQ(getProxyLabelByDisaggregatedMode(DisaggregatedMode::Storage), DEF_PROXY_LABEL); + + ASSERT_EQ(getProxyLabelByDisaggregatedMode(DisaggregatedMode::None), "tiflash"); + ASSERT_EQ(getProxyLabelByDisaggregatedMode(DisaggregatedMode::None), DEF_PROXY_LABEL); +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index c88e739921f..a8fecced955 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -280,6 +280,7 @@ Context Context::createGlobal(std::shared_ptr runtime res.shared = std::make_shared(runtime_components_factory); res.quota = std::make_shared(); res.timezone_info.init(); + res.disaggregated_mode = DisaggregatedMode::None; return res; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 9f8a440de3f..6488e7e5bce 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -484,6 +485,19 @@ class Context MockMPPServerInfo mockMPPServerInfo() const; void setMockMPPServerInfo(MockMPPServerInfo & info); + void setDisaggregatedMode(DisaggregatedMode mode) + { + disaggregated_mode = mode; + } + bool isDisaggregatedComputeMode() const + { + return disaggregated_mode == DisaggregatedMode::Compute; + } + bool isDisaggregatedStorageMode() const + { + return disaggregated_mode == DisaggregatedMode::Storage; + } + private: /** Check if the current client has access to the specified database. * If access is denied, throw an exception. @@ -501,6 +515,7 @@ class Context void checkIsConfigLoaded() const; bool is_config_loaded = false; /// Is configuration loaded from toml file. + DisaggregatedMode disaggregated_mode = DisaggregatedMode::None; }; using ContextPtr = std::shared_ptr; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 5c729c7084d..fcb22f53fb8 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -242,7 +243,6 @@ struct TiFlashProxyConfig const String engine_store_advertise_address = "advertise-engine-addr"; const String pd_endpoints = "pd-endpoints"; const String engine_label = "engine-label"; - const String engine_label_value = "tiflash"; explicit TiFlashProxyConfig(Poco::Util::LayeredConfiguration & config) { @@ -265,7 +265,9 @@ struct TiFlashProxyConfig args_map[engine_store_address] = config.getString("flash.service_addr"); else args_map[engine_store_advertise_address] = args_map[engine_store_address]; - args_map[engine_label] = engine_label_value; + + auto disaggregated_mode = getDisaggregatedMode(config); + args_map[engine_label] = getProxyLabelByDisaggregatedMode(disaggregated_mode); for (auto && [k, v] : args_map) { @@ -835,6 +837,7 @@ int Server::main(const std::vector & /*args*/) TiFlashErrorRegistry::instance(); // This invocation is for initializing TiFlashProxyConfig proxy_conf(config()); + EngineStoreServerWrap tiflash_instance_wrap{}; auto helper = GetEngineStoreServerHelper( &tiflash_instance_wrap); @@ -901,6 +904,7 @@ int Server::main(const std::vector & /*args*/) global_context = std::make_unique(Context::createGlobal()); global_context->setGlobalContext(*global_context); global_context->setApplicationType(Context::ApplicationType::SERVER); + global_context->setDisaggregatedMode(getDisaggregatedMode(config())); /// Init File Provider if (proxy_conf.is_proxy_runnable) @@ -1189,34 +1193,37 @@ int Server::main(const std::vector & /*args*/) } LOG_DEBUG(log, "Load metadata done."); - /// Then, sync schemas with TiDB, and initialize schema sync service. - for (int i = 0; i < 60; i++) // retry for 3 mins + if (!global_context->isDisaggregatedComputeMode()) { - try - { - global_context->getTMTContext().getSchemaSyncer()->syncSchemas(*global_context); - break; - } - catch (Poco::Exception & e) + /// Then, sync schemas with TiDB, and initialize schema sync service. + for (int i = 0; i < 60; i++) // retry for 3 mins { - const int wait_seconds = 3; - LOG_ERROR( - log, - "Bootstrap failed because sync schema error: {}\nWe will sleep for {}" - " seconds and try again.", - e.displayText(), - wait_seconds); - ::sleep(wait_seconds); + try + { + global_context->getTMTContext().getSchemaSyncer()->syncSchemas(*global_context); + break; + } + catch (Poco::Exception & e) + { + const int wait_seconds = 3; + LOG_ERROR( + log, + "Bootstrap failed because sync schema error: {}\nWe will sleep for {}" + " seconds and try again.", + e.displayText(), + wait_seconds); + ::sleep(wait_seconds); + } } - } - LOG_DEBUG(log, "Sync schemas done."); + LOG_DEBUG(log, "Sync schemas done."); - initStores(*global_context, log, storage_config.lazily_init_store); + initStores(*global_context, log, storage_config.lazily_init_store); - // After schema synced, set current database. - global_context->setCurrentDatabase(default_database); + // After schema synced, set current database. + global_context->setCurrentDatabase(default_database); - global_context->initializeSchemaSyncService(); + global_context->initializeSchemaSyncService(); + } CPUAffinityManager::initCPUAffinityManager(config()); LOG_INFO(log, "CPUAffinity: {}", CPUAffinityManager::getInstance().toString()); SCOPE_EXIT({ diff --git a/dbms/src/Storages/StorageDisaggregated.cpp b/dbms/src/Storages/StorageDisaggregated.cpp new file mode 100644 index 00000000000..6160625df5d --- /dev/null +++ b/dbms/src/Storages/StorageDisaggregated.cpp @@ -0,0 +1,291 @@ +// 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 + +namespace DB +{ +const String StorageDisaggregated::ExecIDPrefixForTiFlashStorageSender = "exec_id_disaggregated_tiflash_storage_sender"; + +BlockInputStreams StorageDisaggregated::read( + const Names &, + const SelectQueryInfo &, + const Context &, + QueryProcessingStage::Enum &, + size_t, + unsigned num_streams) +{ + auto remote_table_ranges = buildRemoteTableRanges(); + + auto batch_cop_tasks = buildBatchCopTasks(remote_table_ranges); + RUNTIME_CHECK(!batch_cop_tasks.empty()); + + std::vector dispatch_reqs; + dispatch_reqs.reserve(batch_cop_tasks.size()); + for (const auto & batch_cop_task : batch_cop_tasks) + dispatch_reqs.emplace_back(buildDispatchMPPTaskRequest(batch_cop_task)); + + DAGPipeline pipeline; + buildReceiverStreams(dispatch_reqs, num_streams, pipeline); + pushDownFilter(pipeline); + + return pipeline.streams; +} + +std::vector StorageDisaggregated::buildRemoteTableRanges() +{ + std::unordered_map all_remote_regions; + for (auto physical_table_id : table_scan.getPhysicalTableIDs()) + { + const auto & table_regions_info = context.getDAGContext()->getTableRegionsInfoByTableID(physical_table_id); + + RUNTIME_CHECK_MSG(table_regions_info.local_regions.empty(), "in disaggregated_compute_mode, local_regions should be empty"); + for (const auto & reg : table_regions_info.remote_regions) + all_remote_regions[physical_table_id].emplace_back(std::cref(reg)); + } + + std::vector remote_table_ranges; + for (auto physical_table_id : table_scan.getPhysicalTableIDs()) + { + const auto & remote_regions = all_remote_regions[physical_table_id]; + if (remote_regions.empty()) + continue; + auto key_ranges = RemoteRequest::buildKeyRanges(remote_regions); + remote_table_ranges.emplace_back(RemoteTableRange{physical_table_id, key_ranges}); + } + return remote_table_ranges; +} + +std::vector StorageDisaggregated::buildBatchCopTasks(const std::vector & remote_table_ranges) +{ + std::vector physical_table_ids; + physical_table_ids.reserve(remote_table_ranges.size()); + std::vector ranges_for_each_physical_table; + ranges_for_each_physical_table.reserve(remote_table_ranges.size()); + for (const auto & remote_table_range : remote_table_ranges) + { + physical_table_ids.emplace_back(remote_table_range.first); + ranges_for_each_physical_table.emplace_back(remote_table_range.second); + } + + pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); + pingcap::kv::Backoffer bo(pingcap::kv::copBuildTaskMaxBackoff); + pingcap::kv::StoreType store_type = pingcap::kv::StoreType::TiFlash; + auto batch_cop_tasks = pingcap::coprocessor::buildBatchCopTasks(bo, cluster, table_scan.isPartitionTableScan(), physical_table_ids, ranges_for_each_physical_table, store_type, &Poco::Logger::get("pingcap/coprocessor")); + LOG_DEBUG(log, "batch cop tasks(nums: {}) build finish for tiflash_storage node", batch_cop_tasks.size()); + return batch_cop_tasks; +} + +StorageDisaggregated::RequestAndRegionIDs StorageDisaggregated::buildDispatchMPPTaskRequest( + const pingcap::coprocessor::BatchCopTask & batch_cop_task) +{ + // For error handling, need to record region_ids and store_id to invalidate cache. + std::vector region_ids; + auto dispatch_req = std::make_shared<::mpp::DispatchTaskRequest>(); + ::mpp::TaskMeta * dispatch_req_meta = dispatch_req->mutable_meta(); + dispatch_req_meta->set_start_ts(sender_target_task_start_ts); + dispatch_req_meta->set_task_id(sender_target_task_task_id); + dispatch_req_meta->set_address(batch_cop_task.store_addr); + const auto & settings = context.getSettings(); + dispatch_req->set_timeout(60); + dispatch_req->set_schema_ver(settings.schema_version); + RUNTIME_CHECK_MSG(batch_cop_task.region_infos.empty() != batch_cop_task.table_regions.empty(), + "region_infos and table_regions should not exist at the same time, single table region info: {}, partition table region info: {}", + batch_cop_task.region_infos.size(), + batch_cop_task.table_regions.size()); + if (!batch_cop_task.region_infos.empty()) + { + // For non-partition table. + for (const auto & region_info : batch_cop_task.region_infos) + { + region_ids.push_back(region_info.region_id); + auto * region = dispatch_req->add_regions(); + region->set_region_id(region_info.region_id.id); + region->mutable_region_epoch()->set_version(region_info.region_id.ver); + region->mutable_region_epoch()->set_conf_ver(region_info.region_id.conf_ver); + for (const auto & key_range : region_info.ranges) + { + key_range.setKeyRange(region->add_ranges()); + } + } + } + else + { + // For partition table. + for (const auto & table_region : batch_cop_task.table_regions) + { + auto * req_table_region = dispatch_req->add_table_regions(); + req_table_region->set_physical_table_id(table_region.physical_table_id); + auto * region = req_table_region->add_regions(); + for (const auto & region_info : table_region.region_infos) + { + region_ids.push_back(region_info.region_id); + region->set_region_id(region_info.region_id.id); + region->mutable_region_epoch()->set_version(region_info.region_id.ver); + region->mutable_region_epoch()->set_conf_ver(region_info.region_id.conf_ver); + for (const auto & key_range : region_info.ranges) + { + key_range.setKeyRange(region->add_ranges()); + } + } + } + } + + const auto & sender_target_task_meta = context.getDAGContext()->getMPPTaskMeta(); + const auto * dag_req = context.getDAGContext()->dag_request; + tipb::DAGRequest sender_dag_req; + sender_dag_req.set_time_zone_name(dag_req->time_zone_name()); + sender_dag_req.set_time_zone_offset(dag_req->time_zone_offset()); + // TODO: We have exec summaries bug for now, remote exec summary will not be merged. + sender_dag_req.set_collect_execution_summaries(false); + sender_dag_req.set_flags(dag_req->flags()); + sender_dag_req.set_encode_type(tipb::EncodeType::TypeCHBlock); + sender_dag_req.set_force_encode_type(true); + const auto & column_infos = table_scan.getColumns(); + for (size_t off = 0; off < column_infos.size(); ++off) + { + sender_dag_req.add_output_offsets(off); + } + + tipb::Executor * executor = sender_dag_req.mutable_root_executor(); + executor->set_tp(tipb::ExecType::TypeExchangeSender); + // Exec summary of ExchangeSender will be merged into TableScan. + executor->set_executor_id(fmt::format("{}_{}_{}", + ExecIDPrefixForTiFlashStorageSender, + sender_target_task_start_ts, + sender_target_task_task_id)); + + tipb::ExchangeSender * sender = executor->mutable_exchange_sender(); + sender->set_tp(tipb::ExchangeType::PassThrough); + sender->add_encoded_task_meta(sender_target_task_meta.SerializeAsString()); + auto * child = sender->mutable_child(); + child->CopyFrom(buildTableScanTiPB()); + for (const auto & column_info : column_infos) + { + auto * field_type = sender->add_all_field_types(); + *field_type = columnInfoToFieldType(column_info); + } + // Ignore sender.PartitionKeys and sender.Types because it's a PassThrough sender. + + dispatch_req->set_encoded_plan(sender_dag_req.SerializeAsString()); + return StorageDisaggregated::RequestAndRegionIDs{dispatch_req, region_ids, batch_cop_task.store_id}; +} + +tipb::Executor StorageDisaggregated::buildTableScanTiPB() +{ + // TODO: For now, to avoid versions of tiflash_compute nodes and tiflash_storage being different, + // disable filter push down to avoid unsupported expression in tiflash_storage. + // Uncomment this when we are sure versions are same. + // executor = push_down_filter.constructSelectionForRemoteRead(dag_req.mutable_root_executor()); + + tipb::Executor ts_exec; + ts_exec.set_tp(tipb::ExecType::TypeTableScan); + ts_exec.set_executor_id(table_scan.getTableScanExecutorID()); + + // In disaggregated mode, use DAGRequest sent from TiDB directly, so no need to rely on SchemaSyncer. + if (table_scan.isPartitionTableScan()) + { + ts_exec.set_tp(tipb::ExecType::TypePartitionTableScan); + auto * mutable_partition_table_scan = ts_exec.mutable_partition_table_scan(); + *mutable_partition_table_scan = table_scan.getTableScanPB()->partition_table_scan(); + } + else + { + ts_exec.set_tp(tipb::ExecType::TypeTableScan); + auto * mutable_table_scan = ts_exec.mutable_tbl_scan(); + *mutable_table_scan = table_scan.getTableScanPB()->tbl_scan(); + } + return ts_exec; +} + +void StorageDisaggregated::buildReceiverStreams(const std::vector & dispatch_reqs, unsigned num_streams, DAGPipeline & pipeline) +{ + tipb::ExchangeReceiver receiver; + for (const auto & dispatch_req : dispatch_reqs) + { + const ::mpp::TaskMeta & sender_task_meta = std::get<0>(dispatch_req)->meta(); + receiver.add_encoded_task_meta(sender_task_meta.SerializeAsString()); + } + + const auto & column_infos = table_scan.getColumns(); + for (const auto & column_info : column_infos) + { + auto * field_type = receiver.add_field_types(); + *field_type = columnInfoToFieldType(column_info); + } + + // ExchangeSender just use TableScan's executor_id, so exec summary will be merged to TableScan. + const auto & sender_target_task_meta = context.getDAGContext()->getMPPTaskMeta(); + const String executor_id = table_scan.getTableScanExecutorID(); + + exchange_receiver = std::make_shared( + std::make_shared( + receiver, + sender_target_task_meta, + context.getTMTContext().getKVCluster(), + context.getTMTContext().getMPPTaskManager(), + context.getSettingsRef().enable_local_tunnel, + context.getSettingsRef().enable_async_grpc_client), + receiver.encoded_task_meta_size(), + num_streams, + log->identifier(), + executor_id, + /*fine_grained_shuffle_stream_count=*/0, + dispatch_reqs); + + // MPPTask::receiver_set will record this ExchangeReceiver, so can cancel it in ReceiverSet::cancel(). + context.getDAGContext()->setDisaggregatedComputeExchangeReceiver(executor_id, exchange_receiver); + + // We can use PhysicalExchange::transform() to build InputStream after + // DAGQueryBlockInterpreter is deprecated to avoid duplicated code here. + const String extra_info = "disaggregated compute node exchange receiver"; + for (size_t i = 0; i < num_streams; ++i) + { + BlockInputStreamPtr stream = std::make_shared(exchange_receiver, + log->identifier(), + executor_id, + /*stream_id=*/0); + stream->setExtraInfo(extra_info); + pipeline.streams.push_back(stream); + } + + auto & table_scan_io_input_streams = context.getDAGContext()->getInBoundIOInputStreamsMap()[table_scan.getTableScanExecutorID()]; + auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[table_scan.getTableScanExecutorID()]; + pipeline.transform([&](auto & stream) { + table_scan_io_input_streams.push_back(stream); + profile_streams.push_back(stream); + }); +} + +void StorageDisaggregated::pushDownFilter(DAGPipeline & pipeline) +{ + NamesAndTypes source_columns = genNamesAndTypesForExchangeReceiver(table_scan); + const auto & receiver_dag_schema = exchange_receiver->getOutputSchema(); + assert(receiver_dag_schema.size() == source_columns.size()); + + analyzer = std::make_unique(std::move(source_columns), context); + + if (push_down_filter.hasValue()) + { + // No need to cast, because already done by tiflash_storage node. + ::DB::executePushedDownFilter(/*remote_read_streams_start_index=*/pipeline.streams.size(), push_down_filter, *analyzer, log, pipeline); + + auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[push_down_filter.executor_id]; + pipeline.transform([&profile_streams](auto & stream) { profile_streams.push_back(stream); }); + } +} +} // namespace DB diff --git a/dbms/src/Storages/StorageDisaggregated.h b/dbms/src/Storages/StorageDisaggregated.h new file mode 100644 index 00000000000..b878ff21695 --- /dev/null +++ b/dbms/src/Storages/StorageDisaggregated.h @@ -0,0 +1,96 @@ +// 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. + +#pragma once + +#include +#include +#include +#include +#include +#include + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#include +#pragma GCC diagnostic pop + +namespace DB +{ + +// Naive implementation of StorageDisaggregated, all region data will be transferred by GRPC, +// rewrite this when local cache is supported. +// Naive StorageDisaggregated will convert TableScan to ExchangeReceiver(executed in tiflash_compute node), +// and ExchangeSender + TableScan(executed in tiflash_storage node). +class StorageDisaggregated : public IStorage +{ +public: + StorageDisaggregated( + Context & context_, + const TiDBTableScan & table_scan_, + const PushDownFilter & push_down_filter_) + : IStorage() + , context(context_) + , table_scan(table_scan_) + , log(Logger::get(context_.getDAGContext()->log ? context_.getDAGContext()->log->identifier() : "")) + , sender_target_task_start_ts(context_.getDAGContext()->getMPPTaskMeta().start_ts()) + , sender_target_task_task_id(context_.getDAGContext()->getMPPTaskMeta().task_id()) + , push_down_filter(push_down_filter_) + {} + + std::string getName() const override + { + return "StorageDisaggregated"; + } + + std::string getTableName() const override + { + return "StorageDisaggregated_" + table_scan.getTableScanExecutorID(); + } + + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + using RequestAndRegionIDs = std::tuple, std::vector<::pingcap::kv::RegionVerID>, uint64_t>; + RequestAndRegionIDs buildDispatchMPPTaskRequest(const pingcap::coprocessor::BatchCopTask & batch_cop_task); + + // To help find exec summary of ExchangeSender in tiflash_storage and merge it into TableScan's exec summary. + static const String ExecIDPrefixForTiFlashStorageSender; + // Members will be transferred to DAGQueryBlockInterpreter after execute + std::unique_ptr analyzer; + +private: + using RemoteTableRange = std::pair; + std::vector buildRemoteTableRanges(); + std::vector buildBatchCopTasks(const std::vector & remote_table_ranges); + void buildReceiverStreams(const std::vector & dispatch_reqs, unsigned num_streams, DAGPipeline & pipeline); + void pushDownFilter(DAGPipeline & pipeline); + tipb::Executor buildTableScanTiPB(); + + Context & context; + const TiDBTableScan & table_scan; + LoggerPtr log; + uint64_t sender_target_task_start_ts; + int64_t sender_target_task_task_id; + const PushDownFilter & push_down_filter; + + std::shared_ptr exchange_receiver; +}; +} // namespace DB diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index de115cd3834..07927956c40 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -40,8 +40,11 @@ const int64_t DEFAULT_WAIT_REGION_READY_TIMEOUT_SEC = 20 * 60; const int64_t DEFAULT_READ_INDEX_WORKER_TICK_MS = 10; -static SchemaSyncerPtr createSchemaSyncer(bool exist_pd_addr, bool for_unit_test, const KVClusterPtr & cluster) +static SchemaSyncerPtr createSchemaSyncer(bool exist_pd_addr, bool for_unit_test, const KVClusterPtr & cluster, bool disaggregated_compute_mode) { + // Doesn't need SchemaSyncer for tiflash_compute mode. + if (disaggregated_compute_mode) + return nullptr; if (exist_pd_addr) { // product env @@ -71,7 +74,7 @@ TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config , cluster(raft_config.pd_addrs.empty() ? std::make_shared() : std::make_shared(raft_config.pd_addrs, cluster_config)) , ignore_databases(raft_config.ignore_databases) - , schema_syncer(createSchemaSyncer(!raft_config.pd_addrs.empty(), raft_config.for_unit_test, cluster)) + , schema_syncer(createSchemaSyncer(!raft_config.pd_addrs.empty(), raft_config.for_unit_test, cluster, context_.isDisaggregatedComputeMode())) , mpp_task_manager(std::make_shared( std::make_unique( context.getSettingsRef().task_scheduler_thread_soft_limit, diff --git a/dbms/src/Storages/Transaction/TMTContext.h b/dbms/src/Storages/Transaction/TMTContext.h index 62e1d574275..535da4329fc 100644 --- a/dbms/src/Storages/Transaction/TMTContext.h +++ b/dbms/src/Storages/Transaction/TMTContext.h @@ -81,7 +81,9 @@ class TMTContext : private boost::noncopyable const Context & getContext() const; - explicit TMTContext(Context & context_, const TiFlashRaftConfig & raft_config, const pingcap::ClusterConfig & cluster_config_); + explicit TMTContext(Context & context_, + const TiFlashRaftConfig & raft_config, + const pingcap::ClusterConfig & cluster_config_); SchemaSyncerPtr getSchemaSyncer() const; diff --git a/etc/config-template.toml b/etc/config-template.toml index f56a6a095d4..fb45d426fb7 100644 --- a/etc/config-template.toml +++ b/etc/config-template.toml @@ -107,6 +107,7 @@ # read_index_runner_count = 1 ## The minimum duration to handle read-index tasks in each worker. # read_index_worker_tick_ms = 10 +# disaggregate_mode = "tiflash_storage" or "tiflash_compute" [flash.proxy] # addr = "0.0.0.0:20170" From 445dc720c46d950a68b3624fd799724d5196b27b Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Fri, 9 Dec 2022 12:34:04 +0800 Subject: [PATCH 13/42] update proxy of master to raftstore-proxy (#6473) ref pingcap/tiflash#4982 --- contrib/tiflash-proxy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 314bab7a5a8..3212b44e3bf 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 314bab7a5a8736c21d1a2cdf5ef66082a8dfcbd9 +Subproject commit 3212b44e3bf34639b6cd2fe1bf08c9b7e38fd0ed From 2805651989029400aa135591d8633f07cf4a243a Mon Sep 17 00:00:00 2001 From: JaySon Date: Fri, 9 Dec 2022 13:50:04 +0800 Subject: [PATCH 14/42] Refine the storage->read in DAGStorageInterpreter (#6472) ref pingcap/tiflash#6233 --- dbms/src/Flash/Coprocessor/DAGContext.h | 1 - .../Coprocessor/DAGStorageInterpreter.cpp | 27 +++----- .../Flash/Coprocessor/DAGStorageInterpreter.h | 4 +- .../Interpreters/InterpreterSelectQuery.cpp | 7 +- .../tests/gtest_dm_storage_delta_merge.cpp | 14 ++-- dbms/src/Storages/Page/V2/PageStorage.h | 2 +- dbms/src/Storages/RegionQueryInfo.h | 11 +++- dbms/src/Storages/StorageDeltaMerge.cpp | 64 +------------------ dbms/src/Storages/StorageDeltaMerge.h | 10 --- dbms/src/Storages/StorageDisaggregated.cpp | 1 + dbms/src/Storages/Transaction/LearnerRead.cpp | 7 +- dbms/src/Storages/Transaction/LearnerRead.h | 1 - 12 files changed, 39 insertions(+), 110 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 48472665578..bcfdefa7cf5 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -172,7 +172,6 @@ class DAGContext // for test explicit DAGContext(UInt64 max_error_count_) : dag_request(nullptr) - , dummy_query_string("") , dummy_ast(makeDummyQuery()) , collect_execution_summaries(false) , is_mpp_task(false) diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 3e36418bef5..1c4bce252c1 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -145,7 +146,6 @@ MakeRegionQueryInfos( mvcc_info.regions_query_info.emplace_back(std::move(info)); } } - mvcc_info.concurrent = mvcc_info.regions_query_info.size() > 1 ? 1.0 : 0.0; if (region_need_retry.empty()) return std::make_tuple(std::nullopt, RegionException::RegionReadStatus::OK); @@ -321,7 +321,9 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) { if (!mvcc_query_info->regions_query_info.empty()) { - dagContext().scan_context_map[table_scan.getTableScanExecutorID()] = std::make_shared(); + auto scan_context = std::make_shared(); + dagContext().scan_context_map[table_scan.getTableScanExecutorID()] = scan_context; + mvcc_query_info->scan_context = scan_context; buildLocalStreams(pipeline, settings.max_block_size); } @@ -552,7 +554,7 @@ LearnerReadSnapshot DAGStorageInterpreter::doCopLearnerRead() if (info_retry) throw RegionException({info_retry->begin()->get().region_id}, status); - return doLearnerRead(logical_table_id, *mvcc_query_info, max_streams, /*for_batch_cop=*/false, context, log); + return doLearnerRead(logical_table_id, *mvcc_query_info, /*for_batch_cop=*/false, context, log); } /// Will assign region_retry_from_local_region @@ -588,7 +590,7 @@ LearnerReadSnapshot DAGStorageInterpreter::doBatchCopLearnerRead() } if (mvcc_query_info->regions_query_info.empty()) return {}; - return doLearnerRead(logical_table_id, *mvcc_query_info, max_streams, /*for_batch_cop=*/true, context, log); + return doLearnerRead(logical_table_id, *mvcc_query_info, /*for_batch_cop=*/true, context, log); } catch (const LockException & e) { @@ -629,23 +631,19 @@ std::unordered_map DAGStorageInterpreter::generateSele query_info.is_fast_scan = table_scan.isFastScan(); return query_info; }; + RUNTIME_CHECK_MSG(mvcc_query_info->scan_context != nullptr, "Unexpected null scan_context"); if (table_scan.isPartitionTableScan()) { for (const auto physical_table_id : table_scan.getPhysicalTableIDs()) { SelectQueryInfo query_info = create_query_info(physical_table_id); - query_info.mvcc_query_info = std::make_unique(mvcc_query_info->resolve_locks, mvcc_query_info->read_tso); + query_info.mvcc_query_info = std::make_unique(mvcc_query_info->resolve_locks, mvcc_query_info->read_tso, mvcc_query_info->scan_context); ret.emplace(physical_table_id, std::move(query_info)); } for (auto & r : mvcc_query_info->regions_query_info) { ret[r.physical_table_id].mvcc_query_info->regions_query_info.push_back(r); } - for (auto & p : ret) - { - // todo mvcc_query_info->concurrent is not used anymore, should remove it later - p.second.mvcc_query_info->concurrent = p.second.mvcc_query_info->regions_query_info.size() > 1 ? 1.0 : 0.0; - } } else { @@ -736,14 +734,7 @@ void DAGStorageInterpreter::buildLocalStreamsForPhysicalTable( try { QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; - const auto & scan_context = dag_context.scan_context_map.at(table_scan.getTableScanExecutorID()); - - // We want to collect performance metrics in storage level, thus we need read with scan_context here. - // while IStorage::read() can't support it, and only StorageDeltaMerge support to read with scan_context to collect the information. - // Thus, storage must cast to StorageDeltaMergePtr here to call the corresponding read() function. - StorageDeltaMergePtr delta_merge_storage = std::dynamic_pointer_cast(storage); - RUNTIME_CHECK_MSG(delta_merge_storage != nullptr, "delta_merge_storage which cast from storage is null"); - pipeline.streams = delta_merge_storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams, scan_context); + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); injectFailPointForLocalRead(query_info); diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index aa026461e31..66e703e7fc6 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -112,12 +112,12 @@ class DAGStorageInterpreter Context & context; const TiDBTableScan & table_scan; const PushDownFilter & push_down_filter; - size_t max_streams; + const size_t max_streams; LoggerPtr log; /// derived from other members, doesn't change during DAGStorageInterpreter's lifetime - TableID logical_table_id; + const TableID logical_table_id; const Settings & settings; TMTContext & tmt; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index fdb00ad7b06..47571a6f860 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -51,6 +51,7 @@ #include #include #include +#include #include #include #include @@ -805,7 +806,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline SelectQueryInfo query_info; query_info.query = query_ptr; query_info.sets = query_analyzer->getPreparedSets(); - query_info.mvcc_query_info = std::make_unique(settings.resolve_locks, settings.read_tso); + auto scan_context = std::make_shared(); + query_info.mvcc_query_info = std::make_unique(settings.resolve_locks, settings.read_tso, scan_context); const String & request_str = settings.regions; @@ -847,7 +849,6 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline if (query_info.mvcc_query_info->regions_query_info.empty()) throw Exception("[InterpreterSelectQuery::executeFetchColumns] no region query", ErrorCodes::LOGICAL_ERROR); - query_info.mvcc_query_info->concurrent = 0.0; } /// PARTITION SELECT only supports MergeTree family now. @@ -878,7 +879,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline if (likely(!select_query->no_kvstore)) { auto table_info = managed_storage->getTableInfo(); - learner_read_snapshot = doLearnerRead(table_info.id, *query_info.mvcc_query_info, max_streams, false, context, log); + learner_read_snapshot = doLearnerRead(table_info.id, *query_info.mvcc_query_info, false, context, log); } } } diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp index 77b629228ec..8808b93c92d 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -115,10 +116,11 @@ try } // get read stream from DeltaMergeStorage + auto scan_context = std::make_shared(); QueryProcessingStage::Enum stage2; SelectQueryInfo query_info; query_info.query = std::make_shared(); - query_info.mvcc_query_info = std::make_unique(ctx.getSettingsRef().resolve_locks, std::numeric_limits::max()); + query_info.mvcc_query_info = std::make_unique(ctx.getSettingsRef().resolve_locks, std::numeric_limits::max(), scan_context); BlockInputStreams ins = storage->read(column_names, query_info, ctx, stage2, 8192, 1); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; @@ -290,7 +292,7 @@ try auto sort_desc = storage->getPrimarySortDescription(); ASSERT_FALSE(storage->storeInited()); - auto & store = storage->getStore(); + const auto & store = storage->getStore(); ASSERT_TRUE(storage->storeInited()); auto pk_type2 = store->getPKDataType(); auto sort_desc2 = store->getPrimarySortDescription(); @@ -619,7 +621,7 @@ try column_names.push_back(name_type.name); } - const String path_name = DB::tests::TiFlashTestEnv::getTemporaryPath("StorageDeltaMerge_ReadWriteCase1"); + const String path_name = DB::tests::TiFlashTestEnv::getTemporaryPath("StorageDeltaMerge_ReadExtraPhysicalTableID"); if (Poco::File path(path_name); path.exists()) path.remove(true); @@ -652,10 +654,11 @@ try } // get read stream from DeltaMergeStorage + auto scan_context = std::make_shared(); QueryProcessingStage::Enum stage2; SelectQueryInfo query_info; query_info.query = std::make_shared(); - query_info.mvcc_query_info = std::make_unique(ctx.getSettingsRef().resolve_locks, std::numeric_limits::max()); + query_info.mvcc_query_info = std::make_unique(ctx.getSettingsRef().resolve_locks, std::numeric_limits::max(), scan_context); Names read_columns = {"col1", EXTRA_TABLE_ID_COLUMN_NAME, "col2"}; BlockInputStreams ins = storage->read(read_columns, query_info, ctx, stage2, 8192, 1); ASSERT_EQ(ins.size(), 1); @@ -755,10 +758,11 @@ try output->writeSuffix(); }; auto read_data = [&]() { + auto scan_context = std::make_shared(); QueryProcessingStage::Enum stage2; SelectQueryInfo query_info; query_info.query = std::make_shared(); - query_info.mvcc_query_info = std::make_unique(ctx.getSettingsRef().resolve_locks, std::numeric_limits::max()); + query_info.mvcc_query_info = std::make_unique(ctx.getSettingsRef().resolve_locks, std::numeric_limits::max(), scan_context); Names read_columns = {"col1", EXTRA_TABLE_ID_COLUMN_NAME, "col2"}; BlockInputStreams ins = storage->read(read_columns, query_info, ctx, stage2, 8192, 1); return getInputStreamNRows(ins[0]); diff --git a/dbms/src/Storages/Page/V2/PageStorage.h b/dbms/src/Storages/Page/V2/PageStorage.h index bf59c53a6f1..0d1dcbb67a7 100644 --- a/dbms/src/Storages/Page/V2/PageStorage.h +++ b/dbms/src/Storages/Page/V2/PageStorage.h @@ -92,7 +92,7 @@ class PageStorage : public DB::PageStorage const FileProviderPtr & file_provider_, BackgroundProcessingPool & ver_compact_pool_, bool no_more_insert_ = false); - ~PageStorage() override = default; + ~PageStorage() override { shutdown(); } void restore() override; diff --git a/dbms/src/Storages/RegionQueryInfo.h b/dbms/src/Storages/RegionQueryInfo.h index 0c280aee2fc..74166e176c0 100644 --- a/dbms/src/Storages/RegionQueryInfo.h +++ b/dbms/src/Storages/RegionQueryInfo.h @@ -20,6 +20,11 @@ namespace DB { using DecodedTiKVKeyPtr = std::shared_ptr; +namespace DM +{ +class ScanContext; +using ScanContextPtr = std::shared_ptr; +} // namespace DM struct RegionQueryInfo { @@ -55,16 +60,16 @@ struct MvccQueryInfo const bool resolve_locks; - Float32 concurrent = 1.0; - using RegionsQueryInfo = std::vector; RegionsQueryInfo regions_query_info; using ReadIndexRes = std::unordered_map; ReadIndexRes read_index_res; + DM::ScanContextPtr scan_context; + public: - explicit MvccQueryInfo(bool resolve_locks_ = false, UInt64 read_tso_ = 0); + explicit MvccQueryInfo(bool resolve_locks_ = false, UInt64 read_tso_ = 0, DM::ScanContextPtr scan_ctx = nullptr); }; } // namespace DB diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 475c4bf8eb6..1a213cb424f 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -774,69 +774,7 @@ BlockInputStreams StorageDeltaMerge::read( auto rs_operator = parseRoughSetFilter(query_info, columns_to_read, context, tracing_logger); - auto streams = store->read( - context, - context.getSettingsRef(), - columns_to_read, - ranges, - num_streams, - /*max_version=*/mvcc_query_info.read_tso, - rs_operator, - query_info.req_id, - query_info.keep_order, - /* is_fast_scan */ query_info.is_fast_scan, - max_block_size, - parseSegmentSet(select_query.segment_expression_list), - extra_table_id_index); - - /// Ensure read_tso info after read. - checkReadTso(mvcc_query_info.read_tso, context.getTMTContext(), context, global_context); - - LOG_TRACE(tracing_logger, "[ranges: {}] [streams: {}]", ranges.size(), streams.size()); - - return streams; -} - -BlockInputStreams StorageDeltaMerge::read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & /*processed_stage*/, - size_t max_block_size, - unsigned num_streams, - const ScanContextPtr & scan_context) -{ - auto & store = getAndMaybeInitStore(); - // Note that `columns_to_read` should keep the same sequence as ColumnRef - // in `Coprocessor.TableScan.columns`, or rough set filter could be - // failed to parsed. - ColumnDefines columns_to_read; - size_t extra_table_id_index = InvalidColumnID; - setColumnsToRead(store, columns_to_read, extra_table_id_index, column_names); - - const ASTSelectQuery & select_query = typeid_cast(*query_info.query); - if (select_query.raw_for_mutable) // for selraw - { - // Read without MVCC filtering and del_mark = 1 filtering - return store->readRaw( - context, - context.getSettingsRef(), - columns_to_read, - num_streams, - query_info.keep_order, - parseSegmentSet(select_query.segment_expression_list), - extra_table_id_index); - } - - auto tracing_logger = log->getChild(query_info.req_id); - - // Read with MVCC filtering - RUNTIME_CHECK(query_info.mvcc_query_info != nullptr); - const auto & mvcc_query_info = *query_info.mvcc_query_info; - - auto ranges = parseMvccQueryInfo(mvcc_query_info, num_streams, context, tracing_logger); - - auto rs_operator = parseRoughSetFilter(query_info, columns_to_read, context, tracing_logger); + const auto & scan_context = mvcc_query_info.scan_context; auto streams = store->read( context, diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 91b6bf19982..e4db7207541 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -66,16 +66,6 @@ class StorageDeltaMerge size_t max_block_size, unsigned num_streams) override; - /// use scan_context to record the performance metrics during read. - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams, - const DM::ScanContextPtr & scan_context); - BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override; /// Write from raft layer. diff --git a/dbms/src/Storages/StorageDisaggregated.cpp b/dbms/src/Storages/StorageDisaggregated.cpp index 6160625df5d..8a447637b31 100644 --- a/dbms/src/Storages/StorageDisaggregated.cpp +++ b/dbms/src/Storages/StorageDisaggregated.cpp @@ -276,6 +276,7 @@ void StorageDisaggregated::pushDownFilter(DAGPipeline & pipeline) NamesAndTypes source_columns = genNamesAndTypesForExchangeReceiver(table_scan); const auto & receiver_dag_schema = exchange_receiver->getOutputSchema(); assert(receiver_dag_schema.size() == source_columns.size()); + UNUSED(receiver_dag_schema); analyzer = std::make_unique(std::move(source_columns), context); diff --git a/dbms/src/Storages/Transaction/LearnerRead.cpp b/dbms/src/Storages/Transaction/LearnerRead.cpp index 0a52118eed4..a6f4bf18cea 100644 --- a/dbms/src/Storages/Transaction/LearnerRead.cpp +++ b/dbms/src/Storages/Transaction/LearnerRead.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -159,7 +160,6 @@ class MvccQueryInfoWrap LearnerReadSnapshot doLearnerRead( const TiDB::TableID logical_table_id, MvccQueryInfo & mvcc_query_info_, - size_t num_streams, bool for_batch_cop, Context & context, const LoggerPtr & log) @@ -172,7 +172,7 @@ LearnerReadSnapshot doLearnerRead( const auto & regions_info = mvcc_query_info.getRegionsInfo(); // adjust concurrency by num of regions or num of streams * mvcc_query_info.concurrent - size_t concurrent_num = std::max(1, std::min(static_cast(num_streams * mvcc_query_info->concurrent), regions_info.size())); + size_t concurrent_num = std::max(1, regions_info.size()); // use single thread to do replica read by default because there is some overhead from thread pool itself. concurrent_num = std::min(tmt.replicaReadMaxThread(), concurrent_num); @@ -488,9 +488,10 @@ void validateQueryInfo( } } -MvccQueryInfo::MvccQueryInfo(bool resolve_locks_, UInt64 read_tso_) +MvccQueryInfo::MvccQueryInfo(bool resolve_locks_, UInt64 read_tso_, DM::ScanContextPtr scan_ctx) : read_tso(read_tso_) , resolve_locks(read_tso_ == std::numeric_limits::max() ? false : resolve_locks_) + , scan_context(std::move(scan_ctx)) { // using `std::numeric_limits::max()` to resolve lock may break basic logic. } diff --git a/dbms/src/Storages/Transaction/LearnerRead.h b/dbms/src/Storages/Transaction/LearnerRead.h index ab7da31935c..7d4cc5192c0 100644 --- a/dbms/src/Storages/Transaction/LearnerRead.h +++ b/dbms/src/Storages/Transaction/LearnerRead.h @@ -42,7 +42,6 @@ using LearnerReadSnapshot = std::unordered_map Date: Sat, 10 Dec 2022 14:37:41 +0800 Subject: [PATCH 15/42] Optimize delta compact strategy to reduce iops and write amplification (#6461) close pingcap/tiflash#6460 --- .../DeltaMerge/ColumnFile/ColumnFileTiny.cpp | 10 +------- .../Delta/ColumnFilePersistedSet.cpp | 7 ++++-- .../DeltaMerge/Delta/DeltaValueSpace.cpp | 6 +++-- .../DeltaMerge/Delta/MinorCompaction.h | 14 +++++++++-- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 2 +- .../tests/gtest_dm_delta_value_space.cpp | 7 ++++++ dbms/src/Storages/Page/V2/PageFile.cpp | 10 +++++++- .../Page/V2/tests/gtest_page_storage.cpp | 11 ++++++++ dbms/src/Storages/Page/V3/BlobStore.cpp | 25 +++++++++++++++---- .../Page/V3/tests/gtest_page_storage.cpp | 22 ++++++++++++++++ 10 files changed, 92 insertions(+), 22 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.cpp index 519bec134bc..3dcdabf736a 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFileTiny.cpp @@ -165,15 +165,7 @@ Block ColumnFileTiny::readBlockForMinorCompaction(const PageReader & page_reader else { const auto & schema_ref = *schema; - - PageStorage::PageReadFields fields; - fields.first = data_page_id; - for (size_t i = 0; i < schema_ref.columns(); ++i) - fields.second.push_back(i); - - auto page_map = page_reader.read({fields}); - auto page = page_map[data_page_id]; - + auto page = page_reader.read(data_page_id); auto columns = schema_ref.cloneEmptyColumns(); if (unlikely(columns.size() != page.fieldSize())) diff --git a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp index 54cd00096a0..d3a6845f80d 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/ColumnFilePersistedSet.cpp @@ -263,6 +263,7 @@ MinorCompactionPtr ColumnFilePersistedSet::pickUpMinorCompaction(DMContext & con is_all_trivial_move = is_all_trivial_move && is_trivial_move; cur_task = {}; }; + size_t index = 0; for (auto & file : persisted_files) { if (auto * t_file = file->tryToTinyFile(); t_file) @@ -280,13 +281,15 @@ MinorCompactionPtr ColumnFilePersistedSet::pickUpMinorCompaction(DMContext & con if (cur_task_full || !small_column_file || !schema_ok) pack_up_cur_task(); - cur_task.addColumnFile(file); + cur_task.addColumnFile(file, index); } else { pack_up_cur_task(); - cur_task.addColumnFile(file); + cur_task.addColumnFile(file, index); } + + ++index; } pack_up_cur_task(); diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp index 687044e5183..9a917cdc5f3 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp @@ -404,10 +404,12 @@ bool DeltaValueSpace::compact(DMContext & context) LOG_DEBUG(log, "Compact stop because structure got updated, delta={}", simpleInfo()); return false; } - // Reset to 0 if the minor compaction succeed, + // Reset to the index of first file that can be compacted if the minor compaction succeed, // and it may trigger another minor compaction if there is still too many column files. // This process will stop when there is no more minor compaction to be done. - last_try_compact_column_files.store(0); + auto first_compact_index = compaction_task->getFirsCompactIndex(); + RUNTIME_ASSERT(first_compact_index != std::numeric_limits::max()); + last_try_compact_column_files.store(first_compact_index); LOG_DEBUG(log, "{} delta={}", compaction_task->info(), info()); } wbs.writeRemoves(); diff --git a/dbms/src/Storages/DeltaMerge/Delta/MinorCompaction.h b/dbms/src/Storages/DeltaMerge/Delta/MinorCompaction.h index 3a9890b770c..0112c94a849 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/MinorCompaction.h +++ b/dbms/src/Storages/DeltaMerge/Delta/MinorCompaction.h @@ -36,17 +36,19 @@ class MinorCompaction : public std::enable_shared_from_this Task() = default; ColumnFilePersisteds to_compact; + size_t first_file_index = std::numeric_limits::max(); size_t total_rows = 0; size_t total_bytes = 0; bool is_trivial_move = false; ColumnFilePersistedPtr result; - void addColumnFile(const ColumnFilePersistedPtr & column_file) + void addColumnFile(const ColumnFilePersistedPtr & column_file, size_t index) { + to_compact.push_back(column_file); + first_file_index = std::min(first_file_index, index); total_rows += column_file->getRows(); total_bytes += column_file->getBytes(); - to_compact.push_back(column_file); } }; using Tasks = std::vector; @@ -54,6 +56,9 @@ class MinorCompaction : public std::enable_shared_from_this private: Tasks tasks; + // The index of the first cftiny which can be compacted + size_t first_compact_index = std::numeric_limits::max(); + size_t current_compaction_version; size_t total_compact_files = 0; @@ -82,13 +87,18 @@ class MinorCompaction : public std::enable_shared_from_this } is_trivial_move = true; } + task.is_trivial_move = is_trivial_move; tasks.push_back(std::move(task)); + if (!is_trivial_move) + first_compact_index = std::min(first_compact_index, task.first_file_index); return is_trivial_move; } const Tasks & getTasks() const { return tasks; } + size_t getFirsCompactIndex() const { return first_compact_index; } + size_t getCompactionVersion() const { return current_compaction_version; } /// Create new column file by combining several small `ColumnFileTiny`s diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 7b1049225d1..51f47cd3faa 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1179,7 +1179,7 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const || (segment_rows >= segment_limit_rows * 3 || segment_bytes >= segment_limit_bytes * 3); // Don't do compact on starting up. - bool should_compact = (thread_type != ThreadType::Init) && std::max(static_cast(column_file_count) - delta_last_try_compact_column_files, 0) >= 10; + bool should_compact = (thread_type != ThreadType::Init) && std::max(static_cast(column_file_count) - delta_last_try_compact_column_files, 0) >= 15; // Don't do background place index if we limit DeltaIndex cache. bool should_place_delta_index = !dm_context->db_context.isDeltaIndexLimited() diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp index d6c2ed3c99f..17b5d79febf 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp @@ -26,6 +26,7 @@ #include #include +#include #include namespace CurrentMetrics @@ -350,6 +351,7 @@ TEST_F(DeltaValueSpaceTest, MinorCompaction) // The second task is a trivial move for a ColumnFileDeleteRange. // The third task is a trivial move for and a ColumnFileTiny. const auto & tasks = compaction_task->getTasks(); + ASSERT_EQ(compaction_task->getFirsCompactIndex(), 0); ASSERT_EQ(tasks.size(), 3); ASSERT_EQ(tasks[0].to_compact.size(), 3); ASSERT_EQ(tasks[0].is_trivial_move, false); @@ -379,9 +381,13 @@ TEST_F(DeltaValueSpaceTest, MinorCompaction) } // now the column files in persisted_file_set should be: T_300, D_0_100, T_100, T_100 { + // generate but not commit compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); + EXPECT_EQ(compaction_task->getFirsCompactIndex(), 2); + // generate and commit PageReader reader = dmContext().storage_pool.newLogReader(dmContext().getReadLimiter(), true, ""); compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); + EXPECT_EQ(compaction_task->getFirsCompactIndex(), 2); compaction_task->prepare(dmContext(), wbs, reader); ASSERT_TRUE(compaction_task->commit(persisted_file_set, wbs)); ASSERT_EQ(persisted_file_set->getRows(), total_rows_write); @@ -407,6 +413,7 @@ TEST_F(DeltaValueSpaceTest, MinorCompaction) auto minor_compaction_task = persisted_file_set->pickUpMinorCompaction(dmContext()); if (!minor_compaction_task) break; + ASSERT_NE(minor_compaction_task->getFirsCompactIndex(), std::numeric_limits::max()); minor_compaction_task->prepare(dmContext(), wbs, reader); minor_compaction_task->commit(persisted_file_set, wbs); } diff --git a/dbms/src/Storages/Page/V2/PageFile.cpp b/dbms/src/Storages/Page/V2/PageFile.cpp index a0c501f2bb8..d4642270822 100644 --- a/dbms/src/Storages/Page/V2/PageFile.cpp +++ b/dbms/src/Storages/Page/V2/PageFile.cpp @@ -922,7 +922,15 @@ PageMap PageFile::Reader::read(PageIdAndEntries & to_read, const ReadLimiterPtr page.page_id = page_id; page.data = ByteBuffer(pos, pos + entry.size); page.mem_holder = mem_holder; - page_map.emplace(page_id, page); + + // Calculate the field_offsets from page entry + for (size_t index = 0; index < entry.field_offsets.size(); index++) + { + const auto offset = entry.field_offsets[index].first; + page.field_offsets.emplace(index, offset); + } + + page_map.emplace(page_id, std::move(page)); pos += entry.size; } diff --git a/dbms/src/Storages/Page/V2/tests/gtest_page_storage.cpp b/dbms/src/Storages/Page/V2/tests/gtest_page_storage.cpp index ebea32b69f1..49060f9fdfe 100644 --- a/dbms/src/Storages/Page/V2/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V2/tests/gtest_page_storage.cpp @@ -754,11 +754,22 @@ try ASSERT_EQ(page0.page_id, 0UL); for (size_t i = 0; i < buf_sz; ++i) EXPECT_EQ(*(page0.data.begin() + i), static_cast(i % 0xff)); + ASSERT_EQ(page0.fieldSize(), page0_fields.size()); + for (const auto & [idx, sz] : page0_fields) + { + ASSERT_EQ(page0.getFieldData(idx).size(), sz); + } + DB::Page page1 = storage->read(1); ASSERT_EQ(page1.data.size(), buf_sz); ASSERT_EQ(page1.page_id, 1UL); for (size_t i = 0; i < buf_sz; ++i) EXPECT_EQ(*(page1.data.begin() + i), static_cast(i % 0xff)); + ASSERT_EQ(page1.fieldSize(), page1_fields.size()); + for (const auto & [idx, sz] : page1_fields) + { + ASSERT_EQ(page1.getFieldData(idx).size(), sz); + } } } CATCH diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index 7d1d451f999..799d0c2cfe8 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -720,7 +720,15 @@ PageMap BlobStore::read(PageIDAndEntriesV3 & entries, const ReadLimiterPtr & rea Page page(page_id_v3); page.data = ByteBuffer(pos, pos + entry.size); page.mem_holder = mem_holder; - page_map.emplace(page_id_v3.low, page); + + // Calculate the field_offsets from page entry + for (size_t index = 0; index < entry.field_offsets.size(); index++) + { + const auto offset = entry.field_offsets[index].first; + page.field_offsets.emplace(index, offset); + } + + page_map.emplace(page_id_v3.low, std::move(page)); pos += entry.size; } @@ -736,15 +744,15 @@ PageMap BlobStore::read(PageIDAndEntriesV3 & entries, const ReadLimiterPtr & rea Page BlobStore::read(const PageIDAndEntryV3 & id_entry, const ReadLimiterPtr & read_limiter) { - if (!id_entry.second.isValid()) + const auto & [page_id_v3, entry] = id_entry; + const size_t buf_size = entry.size; + + if (!entry.isValid()) { Page page_not_found(buildV3Id(id_entry.first.high, INVALID_PAGE_ID)); return page_not_found; } - const auto & [page_id_v3, entry] = id_entry; - const size_t buf_size = entry.size; - // When we read `WriteBatch` which is `WriteType::PUT_EXTERNAL`. // The `buf_size` will be 0, we need avoid calling malloc/free with size 0. if (buf_size == 0) @@ -782,6 +790,13 @@ Page BlobStore::read(const PageIDAndEntryV3 & id_entry, const ReadLimiterPtr & r page.data = ByteBuffer(data_buf, data_buf + buf_size); page.mem_holder = mem_holder; + // Calculate the field_offsets from page entry + for (size_t index = 0; index < entry.field_offsets.size(); index++) + { + const auto offset = entry.field_offsets[index].first; + page.field_offsets.emplace(index, offset); + } + return page; } diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp index cd6d527c229..d699701b56d 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp @@ -426,6 +426,28 @@ try ASSERT_GT(page_maps.count(5), 0); ASSERT_EQ(page_maps[5].isValid(), false); } + { + // Read with id can also fetch the fieldOffsets + auto page_4 = page_storage->readImpl(TEST_NAMESPACE_ID, 4, nullptr, nullptr, false); + ASSERT_EQ(page_4.fieldSize(), 4); + ASSERT_EQ(page_4.getFieldData(0).size(), 20); + ASSERT_EQ(page_4.getFieldData(1).size(), 20); + ASSERT_EQ(page_4.getFieldData(2).size(), 30); + ASSERT_EQ(page_4.getFieldData(3).size(), 30); + } + { + // Read with ids can also fetch the fieldOffsets + PageIds page_ids{4}; + auto pages = page_storage->readImpl(TEST_NAMESPACE_ID, page_ids, nullptr, nullptr, false); + ASSERT_EQ(pages.size(), 1); + ASSERT_GT(pages.count(4), 0); + auto page_4 = pages[4]; + ASSERT_EQ(page_4.fieldSize(), 4); + ASSERT_EQ(page_4.getFieldData(0).size(), 20); + ASSERT_EQ(page_4.getFieldData(1).size(), 20); + ASSERT_EQ(page_4.getFieldData(2).size(), 30); + ASSERT_EQ(page_4.getFieldData(3).size(), 30); + } } CATCH From fa67119a8a5f5cce8679b1c6a4c572cbb01e0d25 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Wed, 14 Dec 2022 14:52:52 +0800 Subject: [PATCH 16/42] Add metrics about the status of exchange sender/receiver queue (#6483) close pingcap/tiflash#6225 --- dbms/src/Common/TiFlashMetrics.h | 3 + dbms/src/Flash/EstablishCall.cpp | 1 + dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 38 ++++++++- dbms/src/Flash/Mpp/ExchangeReceiver.h | 2 + dbms/src/Flash/Mpp/MPPTunnel.cpp | 18 +++-- dbms/src/Flash/Mpp/MPPTunnel.h | 45 +++++++++-- metrics/grafana/tiflash_summary.json | 102 +++++++++++++++++++++++- 7 files changed, 190 insertions(+), 19 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 4c27a0f3025..8accc59b155 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -262,6 +262,9 @@ namespace DB F(type_merged_task, {{"type", "merged_task"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_mpp_task_manager, "The gauge of mpp task manager", Gauge, \ F(type_mpp_query_count, {"type", "mpp_query_count"})) \ + M(tiflash_exchange_queueing_data_bytes, "Total bytes of data contained in the queue", Gauge, \ + F(type_send, {{"type", "send_queue"}}), \ + F(type_receive, {{"type", "recv_queue"}})) \ // clang-format on /// Buckets with boundaries [start * base^0, start * base^1, ..., start * base^(size-1)] diff --git a/dbms/src/Flash/EstablishCall.cpp b/dbms/src/Flash/EstablishCall.cpp index c9b1bbe95c6..df7b4e41a33 100644 --- a/dbms/src/Flash/EstablishCall.cpp +++ b/dbms/src/Flash/EstablishCall.cpp @@ -233,6 +233,7 @@ void EstablishCallData::trySendOneMsg() switch (async_tunnel_sender->pop(res, this)) { case GRPCSendQueueRes::OK: + async_tunnel_sender->subDataSizeMetric(res->getPacket().ByteSizeLong()); /// Note: has to switch the memory tracker before `write` /// because after `write`, `async_tunnel_sender` can be destroyed at any time /// so there is a risk that `res` is destructed after `aysnc_tunnel_sender` diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index adbbad0e48b..26e2ccda376 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -155,6 +155,26 @@ enum class AsyncRequestStage FINISHED, }; +namespace ExchangeReceiverMetric +{ +inline void addDataSizeMetric(std::atomic & data_size_in_queue, size_t size) +{ + data_size_in_queue.fetch_add(size); + GET_METRIC(tiflash_exchange_queueing_data_bytes, type_receive).Increment(size); +} + +inline void subDataSizeMetric(std::atomic & data_size_in_queue, size_t size) +{ + data_size_in_queue.fetch_sub(size); + GET_METRIC(tiflash_exchange_queueing_data_bytes, type_receive).Decrement(size); +} + +inline void clearDataSizeMetric(std::atomic & data_size_in_queue) +{ + GET_METRIC(tiflash_exchange_queueing_data_bytes, type_receive).Decrement(data_size_in_queue.load()); +} +} // namespace ExchangeReceiverMetric + using Clock = std::chrono::system_clock; using TimePoint = Clock::time_point; @@ -176,12 +196,14 @@ class AsyncRequestHandler : public UnaryCallback std::vector * msg_channels_, const std::shared_ptr & context, const Request & req, - const String & req_id) + const String & req_id, + std::atomic * data_size_in_queue_) : rpc_context(context) , cq(&(GRPCCompletionQueuePool::global_instance->pickQueue())) , request(&req) , notify_queue(queue) , msg_channels(msg_channels_) + , data_size_in_queue(data_size_in_queue_) , req_info(fmt::format("tunnel{}+{}", req.send_task_id, req.recv_task_id)) , log(Logger::get(req_id, req_info)) { @@ -369,6 +391,9 @@ class AsyncRequestHandler : public UnaryCallback *msg_channels, log)) return false; + + ExchangeReceiverMetric::addDataSizeMetric(*data_size_in_queue, packet->getPacket().ByteSizeLong()); + // can't reuse packet since it is sent to readers. packet = std::make_shared(); } @@ -387,6 +412,7 @@ class AsyncRequestHandler : public UnaryCallback const Request * request; // won't be null MPMCQueue * notify_queue; // won't be null std::vector * msg_channels; // won't be null + std::atomic * data_size_in_queue; // won't be null String req_info; bool meet_error = false; @@ -423,6 +449,7 @@ ExchangeReceiverBase::ExchangeReceiverBase( , state(ExchangeReceiverState::NORMAL) , exc_log(Logger::get(req_id, executor_id)) , collected(false) + , data_size_in_queue(0) , disaggregated_dispatch_reqs(disaggregated_dispatch_reqs_) { try @@ -430,9 +457,7 @@ ExchangeReceiverBase::ExchangeReceiverBase( if (enable_fine_grained_shuffle_flag) { for (size_t i = 0; i < output_stream_count; ++i) - { msg_channels.push_back(std::make_unique>>(max_buffer_size)); - } } else { @@ -465,6 +490,7 @@ ExchangeReceiverBase::~ExchangeReceiverBase() { close(); thread_manager->wait(); + ExchangeReceiverMetric::clearDataSizeMetric(data_size_in_queue); } catch (...) { @@ -545,7 +571,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_channels, rpc_context, req, exc_log->identifier())); + handlers.emplace_back(std::make_unique(&ready_requests, &msg_channels, rpc_context, req, exc_log->identifier(), &data_size_in_queue)); while (alive_async_connections > 0) { @@ -617,6 +643,8 @@ void ExchangeReceiverBase::readLoop(const Request & req) local_err_msg = fmt::format("Push mpp packet failed. {}", getStatusString()); break; } + + ExchangeReceiverMetric::addDataSizeMetric(data_size_in_queue, packet->getPacket().ByteSizeLong()); } // if meet error, such as decode packet fails, it will not retry. if (meet_error) @@ -713,6 +741,8 @@ ExchangeReceiverResult ExchangeReceiverBase::nextResult( assert(recv_msg != nullptr); if (unlikely(recv_msg->error_ptr != nullptr)) return ExchangeReceiverResult::newError(recv_msg->source_index, recv_msg->req_info, recv_msg->error_ptr->msg()); + + ExchangeReceiverMetric::subDataSizeMetric(data_size_in_queue, recv_msg->packet->getPacket().ByteSizeLong()); return toDecodeResult(block_queue, header, recv_msg, decoder_ptr); } } diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.h b/dbms/src/Flash/Mpp/ExchangeReceiver.h index a05bff8a0cc..b5cc4877903 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.h +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.h @@ -221,6 +221,8 @@ class ExchangeReceiverBase bool collected = false; int thread_count = 0; + std::atomic data_size_in_queue; + // For tiflash_compute node, need to send MPPTask to tiflash_storage node. std::vector disaggregated_dispatch_reqs; }; diff --git a/dbms/src/Flash/Mpp/MPPTunnel.cpp b/dbms/src/Flash/Mpp/MPPTunnel.cpp index 18791132675..431b473f4e8 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnel.cpp @@ -45,7 +45,7 @@ String tunnelSenderModeToString(TunnelSenderMode mode) } // Update metric for tunnel's response bytes -inline void updateMetric(size_t pushed_data_size, TunnelSenderMode mode) +void updateMetric(std::atomic & data_size_in_queue, size_t pushed_data_size, TunnelSenderMode mode) { switch (mode) { @@ -59,6 +59,7 @@ inline void updateMetric(size_t pushed_data_size, TunnelSenderMode mode) default: throw DB::Exception("Illegal TunnelSenderMode"); } + MPPTunnelMetric::addDataSizeMetric(data_size_in_queue, pushed_data_size); } } // namespace @@ -86,6 +87,7 @@ MPPTunnel::MPPTunnel( , mem_tracker(current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr) , queue_size(std::max(5, input_steams_num_ * 5)) // MPMCQueue can benefit from a slightly larger queue size , log(Logger::get(req_id, tunnel_id)) + , data_size_in_queue(0) { RUNTIME_ASSERT(!(is_local_ && is_async_), log, "is_local: {}, is_async: {}.", is_local_, is_async_); if (is_local_) @@ -105,6 +107,7 @@ MPPTunnel::~MPPTunnel() try { close("", true); + MPPTunnelMetric::clearDataSizeMetric(data_size_in_queue); } catch (...) { @@ -160,7 +163,7 @@ void MPPTunnel::write(TrackedMppDataPacketPtr && data) auto pushed_data_size = data->getPacket().ByteSizeLong(); if (tunnel_sender->push(std::move(data))) { - updateMetric(pushed_data_size, mode); + updateMetric(data_size_in_queue, pushed_data_size, mode); connection_profile_info.bytes += pushed_data_size; connection_profile_info.packets += 1; return; @@ -196,14 +199,14 @@ void MPPTunnel::connect(PacketWriter * writer) case TunnelSenderMode::LOCAL: { RUNTIME_ASSERT(writer == nullptr, log); - local_tunnel_sender = std::make_shared(queue_size, mem_tracker, log, tunnel_id); + local_tunnel_sender = std::make_shared(queue_size, mem_tracker, log, tunnel_id, &data_size_in_queue); tunnel_sender = local_tunnel_sender; break; } case TunnelSenderMode::SYNC_GRPC: { RUNTIME_ASSERT(writer != nullptr, log, "Sync writer shouldn't be null"); - sync_tunnel_sender = std::make_shared(queue_size, mem_tracker, log, tunnel_id); + sync_tunnel_sender = std::make_shared(queue_size, mem_tracker, log, tunnel_id, &data_size_in_queue); sync_tunnel_sender->startSendThread(writer); tunnel_sender = sync_tunnel_sender; break; @@ -231,11 +234,11 @@ void MPPTunnel::connectAsync(IAsyncCallData * call_data) auto kick_func_for_test = call_data->getKickFuncForTest(); if (unlikely(kick_func_for_test.has_value())) { - async_tunnel_sender = std::make_shared(queue_size, mem_tracker, log, tunnel_id, kick_func_for_test.value()); + async_tunnel_sender = std::make_shared(queue_size, mem_tracker, log, tunnel_id, kick_func_for_test.value(), &data_size_in_queue); } else { - async_tunnel_sender = std::make_shared(queue_size, mem_tracker, log, tunnel_id, call_data->grpcCall()); + async_tunnel_sender = std::make_shared(queue_size, mem_tracker, log, tunnel_id, call_data->grpcCall(), &data_size_in_queue); } call_data->attachAsyncTunnelSender(async_tunnel_sender); tunnel_sender = async_tunnel_sender; @@ -345,6 +348,7 @@ void SyncTunnelSender::sendJob(PacketWriter * writer) TrackedMppDataPacketPtr res; while (send_queue.pop(res) == MPMCQueueResult::OK) { + MPPTunnelMetric::subDataSizeMetric(*data_size_in_queue, res->getPacket().ByteSizeLong()); if (!writer->write(res->packet)) { err_msg = "grpc writes failed."; @@ -389,6 +393,8 @@ std::shared_ptr LocalTunnelSender::readForLocal() auto result = send_queue.pop(res); if (result == MPMCQueueResult::OK) { + MPPTunnelMetric::subDataSizeMetric(*data_size_in_queue, res->getPacket().ByteSizeLong()); + // switch tunnel's memory tracker into receiver's res->switchMemTracker(current_memory_tracker); return res; diff --git a/dbms/src/Flash/Mpp/MPPTunnel.h b/dbms/src/Flash/Mpp/MPPTunnel.h index db1326b3c61..953d40be3c2 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.h +++ b/dbms/src/Flash/Mpp/MPPTunnel.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -49,6 +50,26 @@ namespace tests class TestMPPTunnel; } // namespace tests +namespace MPPTunnelMetric +{ +inline void addDataSizeMetric(std::atomic & data_size_in_queue, size_t size) +{ + data_size_in_queue.fetch_add(size); + GET_METRIC(tiflash_exchange_queueing_data_bytes, type_send).Increment(size); +} + +inline void subDataSizeMetric(std::atomic & data_size_in_queue, size_t size) +{ + data_size_in_queue.fetch_sub(size); + GET_METRIC(tiflash_exchange_queueing_data_bytes, type_send).Decrement(size); +} + +inline void clearDataSizeMetric(std::atomic & data_size_in_queue) +{ + GET_METRIC(tiflash_exchange_queueing_data_bytes, type_send).Decrement(data_size_in_queue.load()); +} +} // namespace MPPTunnelMetric + class IAsyncCallData; enum class TunnelSenderMode @@ -64,11 +85,12 @@ class TunnelSender : private boost::noncopyable { public: virtual ~TunnelSender() = default; - TunnelSender(size_t queue_size, MemoryTrackerPtr & memory_tracker_, const LoggerPtr & log_, const String & tunnel_id_) + TunnelSender(size_t queue_size, MemoryTrackerPtr & memory_tracker_, const LoggerPtr & log_, const String & tunnel_id_, std::atomic * data_size_in_queue_) : memory_tracker(memory_tracker_) - , send_queue(MPMCQueue(queue_size)) , log(log_) , tunnel_id(tunnel_id_) + , data_size_in_queue(data_size_in_queue_) + , send_queue(MPMCQueue(queue_size)) { } @@ -137,11 +159,14 @@ class TunnelSender : private boost::noncopyable std::shared_future future; std::atomic msg_has_set{false}; }; + MemoryTrackerPtr memory_tracker; - MPMCQueue send_queue; ConsumerState consumer_state; const LoggerPtr log; const String tunnel_id; + + std::atomic * data_size_in_queue; // From MppTunnel + MPMCQueue send_queue; }; /// SyncTunnelSender maintains a new thread itself to consume and send data @@ -163,14 +188,14 @@ class SyncTunnelSender : public TunnelSender class AsyncTunnelSender : public TunnelSender { public: - AsyncTunnelSender(size_t queue_size, MemoryTrackerPtr & memory_tracker, const LoggerPtr & log_, const String & tunnel_id_, grpc_call * call_) - : TunnelSender(0, memory_tracker, log_, tunnel_id_) + AsyncTunnelSender(size_t queue_size, MemoryTrackerPtr & memory_tracker, const LoggerPtr & log_, const String & tunnel_id_, grpc_call * call_, std::atomic * data_size_in_queue) + : TunnelSender(0, memory_tracker, log_, tunnel_id_, data_size_in_queue) , queue(queue_size, call_, log_) {} /// For gtest usage. - AsyncTunnelSender(size_t queue_size, MemoryTrackerPtr & memoryTracker, const LoggerPtr & log_, const String & tunnel_id_, GRPCKickFunc func) - : TunnelSender(0, memoryTracker, log_, tunnel_id_) + AsyncTunnelSender(size_t queue_size, MemoryTrackerPtr & memoryTracker, const LoggerPtr & log_, const String & tunnel_id_, GRPCKickFunc func, std::atomic * data_size_in_queue) + : TunnelSender(0, memoryTracker, log_, tunnel_id_, data_size_in_queue) , queue(queue_size, func) {} @@ -199,6 +224,11 @@ class AsyncTunnelSender : public TunnelSender return queue.pop(data, new_tag); } + void subDataSizeMetric(size_t size) + { + ::DB::MPPTunnelMetric::subDataSizeMetric(*data_size_in_queue, size); + } + private: GRPCSendQueue queue; }; @@ -347,6 +377,7 @@ class MPPTunnel : private boost::noncopyable SyncTunnelSenderPtr sync_tunnel_sender; AsyncTunnelSenderPtr async_tunnel_sender; LocalTunnelSenderPtr local_tunnel_sender; + std::atomic data_size_in_queue; }; using MPPTunnelPtr = std::shared_ptr; diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 9d69c8ec6d8..87c5c9ad035 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -51,8 +51,7 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": null, - "iteration": 1667534599787, + "iteration": 1670904884485, "links": [], "panels": [ { @@ -3961,6 +3960,105 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 45 + }, + "hiddenSeries": false, + "id": 166, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tiflash_exchange_queueing_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Data size in send and receive queue", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, From c51206f82112e015c7aaa9faf382bb4114e3934c Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 15 Dec 2022 11:00:52 +0800 Subject: [PATCH 17/42] Set max_threads according to the number of physical cpu cores (#6481) close pingcap/tiflash#6480 --- dbms/src/Common/getNumberOfCPUCores.cpp | 62 +++++++++++++++++++ ...ogicalCPUCores.h => getNumberOfCPUCores.h} | 5 +- .../src/Common/getNumberOfLogicalCPUCores.cpp | 34 ---------- dbms/src/Flash/FlashService.cpp | 2 +- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 2 +- dbms/src/Interpreters/SettingsCommon.h | 4 +- dbms/src/Server/Server.cpp | 4 +- 7 files changed, 73 insertions(+), 40 deletions(-) create mode 100644 dbms/src/Common/getNumberOfCPUCores.cpp rename dbms/src/Common/{getNumberOfLogicalCPUCores.h => getNumberOfCPUCores.h} (78%) delete mode 100644 dbms/src/Common/getNumberOfLogicalCPUCores.cpp diff --git a/dbms/src/Common/getNumberOfCPUCores.cpp b/dbms/src/Common/getNumberOfCPUCores.cpp new file mode 100644 index 00000000000..c3d3264c627 --- /dev/null +++ b/dbms/src/Common/getNumberOfCPUCores.cpp @@ -0,0 +1,62 @@ +// 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 CPUCores +{ +UInt16 number_of_logical_cpu_cores = std::thread::hardware_concurrency(); +UInt16 number_of_physical_cpu_cores = std::thread::hardware_concurrency() / 2; +} // namespace CPUCores + + +UInt16 getNumberOfLogicalCPUCores() +{ + return CPUCores::number_of_logical_cpu_cores; +} + +UInt16 getNumberOfPhysicalCPUCores() +{ + return CPUCores::number_of_physical_cpu_cores; +} + +// We should call this function before Context has been created, +// which will call `getNumberOfLogicalCPUCores`, or we can not +// set cpu cores any more. +void setNumberOfLogicalCPUCores(UInt16 number_of_logical_cpu_cores_) +{ + CPUCores::number_of_logical_cpu_cores = number_of_logical_cpu_cores_; +} + +void computeAndSetNumberOfPhysicalCPUCores(UInt16 number_of_logical_cpu_cores_, UInt16 number_of_hardware_physical_cores) +{ + // First of all, we need to take consideration of two situation: + // 1. tiflash on physical machine. + // In old codes, tiflash needs to set max_threads which is equal to + // physical cpu cores, so we need to ensure this behavior is not broken. + // 2. tiflash on virtual environment. + // In virtual environment, when setting max_threads, we can't directly + // get physical cpu cores to set this variable because only host machine's + // physical cpu core can be reached. So, number of physical cpus cores can + // only be assigned by calculated with logical cpu cores. + // + // - `number_of_logical_cpu_cores_` which represents how many logical cpu cores a tiflash could use(no matter in physical or virtual environment) is assigned from ServerInfo. + // - `hardware_logical_cpu_cores` represents how many logical cpu cores the host physical machine has. + // - `number_of_hardware_physical_cores` represents how many physical cpu cores the host physical machine has. + // - `(hardware_logical_cpu_cores / number_of_hardware_physical_cores)` means how many logical cpu core a physical cpu core has. + // - `number_of_logical_cpu_cores_ / (hardware_logical_cpu_cores / number_of_hardware_physical_cores)` means how many physical cpu cores the tiflash process could use. (Actually, it's needless to get physical cpu cores in virtual environment, but we must ensure the behavior `1` is not broken) + auto hardware_logical_cpu_cores = std::thread::hardware_concurrency(); + CPUCores::number_of_physical_cpu_cores = number_of_logical_cpu_cores_ / (hardware_logical_cpu_cores / number_of_hardware_physical_cores); +} diff --git a/dbms/src/Common/getNumberOfLogicalCPUCores.h b/dbms/src/Common/getNumberOfCPUCores.h similarity index 78% rename from dbms/src/Common/getNumberOfLogicalCPUCores.h rename to dbms/src/Common/getNumberOfCPUCores.h index d50d13c2596..e6b0f5b84ca 100644 --- a/dbms/src/Common/getNumberOfLogicalCPUCores.h +++ b/dbms/src/Common/getNumberOfCPUCores.h @@ -19,8 +19,11 @@ #include UInt16 getNumberOfLogicalCPUCores(); +UInt16 getNumberOfPhysicalCPUCores(); // We should call this function before Context has been created, // which will call `getNumberOfLogicalCPUCores`, or we can not // set cpu cores any more. -void setNumberOfLogicalCPUCores(UInt16 max_logical_cpu_cores); +void setNumberOfLogicalCPUCores(UInt16 number_of_logical_cpu_cores_); + +void computeAndSetNumberOfPhysicalCPUCores(UInt16 number_of_logical_cpu_cores, UInt16 number_of_hardware_physical_cores); diff --git a/dbms/src/Common/getNumberOfLogicalCPUCores.cpp b/dbms/src/Common/getNumberOfLogicalCPUCores.cpp deleted file mode 100644 index 16854909636..00000000000 --- a/dbms/src/Common/getNumberOfLogicalCPUCores.cpp +++ /dev/null @@ -1,34 +0,0 @@ -// 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 - -namespace CPUCores -{ -UInt16 number_of_logical_cpu_cores = std::thread::hardware_concurrency(); -} // namespace CPUCores - - -UInt16 getNumberOfLogicalCPUCores() -{ - return CPUCores::number_of_logical_cpu_cores; -} - -// We should call this function before Context has been created, -// which will call `getNumberOfLogicalCPUCores`, or we can not -// set cpu cores any more. -void setNumberOfLogicalCPUCores(UInt16 max_logical_cpu_cores) -{ - CPUCores::number_of_logical_cpu_cores = max_logical_cpu_cores; -} diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index ed13fcfe1c9..c1e647bc160 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index 8107a9e3712..b7deae93311 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index 4510159da57..fe84f1f1790 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -16,7 +16,7 @@ #include #include -#include +#include #include #include #include @@ -167,7 +167,7 @@ struct SettingMaxThreads static UInt64 getAutoValue() { - static auto res = getNumberOfLogicalCPUCores(); + static auto res = getNumberOfPhysicalCPUCores(); return res; } diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index fcb22f53fb8..c3194c840df 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -32,7 +32,7 @@ #include #include #include -#include +#include #include #include #include @@ -886,11 +886,13 @@ int Server::main(const std::vector & /*args*/) helper->fn_server_info(helper->proxy_ptr, strIntoView(&req), &response); server_info.parseSysInfo(response); setNumberOfLogicalCPUCores(server_info.cpu_info.logical_cores); + computeAndSetNumberOfPhysicalCPUCores(server_info.cpu_info.logical_cores, server_info.cpu_info.physical_cores); LOG_INFO(log, "ServerInfo: {}", server_info.debugString()); } else { setNumberOfLogicalCPUCores(std::thread::hardware_concurrency()); + computeAndSetNumberOfPhysicalCPUCores(std::thread::hardware_concurrency(), std::thread::hardware_concurrency() / 2); LOG_INFO(log, "TiFlashRaftProxyHelper is null, failed to get server info"); } From 4bdafd96f0f384cabb9e260007333732b7574a0d Mon Sep 17 00:00:00 2001 From: Ning Xu Date: Thu, 15 Dec 2022 11:40:52 +0800 Subject: [PATCH 18/42] Fix compilation error on macOS m1 due to sprintf (#6487) close pingcap/tiflash#6488 --- dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp b/dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp index 5cedd24266d..972df6e80e9 100644 --- a/dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp +++ b/dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp @@ -15,8 +15,14 @@ #include #include #include + +#pragma GCC diagnostic push +#pragma GCC diagnostic warning "-Wdeprecated-declarations" + #include +#pragma GCC diagnostic pop + #include namespace DB { From acb6d9e1e9fd8212a0ed16d269ebbcb8641ebb7d Mon Sep 17 00:00:00 2001 From: Meng Xin Date: Thu, 15 Dec 2022 15:22:52 +0800 Subject: [PATCH 19/42] split hash join return block if its rows is more than max_block_size (#6280) close pingcap/tiflash#6384 --- dbms/src/Columns/ColumnAggregateFunction.cpp | 9 +- dbms/src/Columns/ColumnAggregateFunction.h | 2 +- dbms/src/Columns/ColumnArray.cpp | 33 +-- dbms/src/Columns/ColumnArray.h | 2 +- dbms/src/Columns/ColumnConst.cpp | 6 +- dbms/src/Columns/ColumnConst.h | 2 +- dbms/src/Columns/ColumnDecimal.cpp | 10 +- dbms/src/Columns/ColumnDecimal.h | 3 +- dbms/src/Columns/ColumnFixedString.cpp | 15 +- dbms/src/Columns/ColumnFixedString.h | 2 +- dbms/src/Columns/ColumnFunction.cpp | 19 +- dbms/src/Columns/ColumnFunction.h | 4 +- dbms/src/Columns/ColumnNullable.cpp | 7 +- dbms/src/Columns/ColumnNullable.h | 2 +- dbms/src/Columns/ColumnString.cpp | 20 +- dbms/src/Columns/ColumnString.h | 27 ++- dbms/src/Columns/ColumnTuple.cpp | 4 +- dbms/src/Columns/ColumnTuple.h | 2 +- dbms/src/Columns/ColumnVector.cpp | 13 +- dbms/src/Columns/ColumnVector.h | 2 +- dbms/src/Columns/IColumn.h | 8 +- dbms/src/Columns/IColumnDummy.h | 7 +- .../HashJoinProbeBlockInputStream.cpp | 30 ++- .../HashJoinProbeBlockInputStream.h | 6 +- .../MockExchangeReceiverInputStream.h | 2 +- .../MockTableScanBlockInputStream.h | 2 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 2 +- dbms/src/Flash/Planner/plans/PhysicalJoin.cpp | 2 +- dbms/src/Flash/tests/gtest_join_executor.cpp | 175 ++++++++++++++ dbms/src/Interpreters/ExpressionActions.cpp | 5 +- dbms/src/Interpreters/Join.cpp | 214 +++++++++++++----- dbms/src/Interpreters/Join.h | 28 ++- dbms/src/TestUtils/ExecutorTestUtils.cpp | 12 + dbms/src/TestUtils/ExecutorTestUtils.h | 4 + 34 files changed, 513 insertions(+), 168 deletions(-) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index 8702a2b241a..3b72b127081 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -372,21 +372,24 @@ void ColumnAggregateFunction::popBack(size_t n) data.resize_assume_reserved(new_size); } -ColumnPtr ColumnAggregateFunction::replicate(const IColumn::Offsets & offsets) const +ColumnPtr ColumnAggregateFunction::replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const { size_t size = data.size(); if (size != offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + assert(start_row < end_row); + assert(end_row <= size); + if (size == 0) return cloneEmpty(); auto res = createView(); auto & res_data = res->getData(); - res_data.reserve(offsets.back()); + res_data.reserve(offsets[end_row - 1]); IColumn::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = start_row; i < end_row; ++i) { size_t size_to_replicate = offsets[i] - prev_offset; prev_offset = offsets[i]; diff --git a/dbms/src/Columns/ColumnAggregateFunction.h b/dbms/src/Columns/ColumnAggregateFunction.h index ac0205528c7..4dc76f44543 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.h +++ b/dbms/src/Columns/ColumnAggregateFunction.h @@ -173,7 +173,7 @@ class ColumnAggregateFunction final : public COWPtrHelper(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) - return replicateString(replicate_offsets); - if (typeid_cast(data.get())) - return replicateConst(replicate_offsets); - if (typeid_cast(data.get())) - return replicateNullable(replicate_offsets); - if (typeid_cast(data.get())) - return replicateTuple(replicate_offsets); - return replicateGeneric(replicate_offsets); + throw Exception("not implement.", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index 6493dbecab5..72327995403 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -95,7 +95,7 @@ class ColumnArray final : public COWPtrHelper size_t byteSize() const override; size_t byteSize(size_t offset, size_t limit) const override; size_t allocatedBytes() const override; - ColumnPtr replicate(const Offsets & replicate_offsets) const override; + ColumnPtr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & replicate_offsets) const override; ColumnPtr convertToFullColumnIfConst() const override; void getExtremes(Field & min, Field & max) const override; diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index b049fbd3f38..67a688c68ba 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -55,17 +55,19 @@ ColumnPtr ColumnConst::filter(const Filter & filt, ssize_t /*result_size_hint*/) return ColumnConst::create(data, countBytesInFilter(filt)); } -ColumnPtr ColumnConst::replicate(const Offsets & offsets) const +ColumnPtr ColumnConst::replicateRange(size_t /*start_row*/, size_t end_row, const IColumn::Offsets & offsets) const { if (s != offsets.size()) throw Exception( fmt::format("Size of offsets ({}) doesn't match size of column ({})", offsets.size(), s), ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - size_t replicated_size = 0 == s ? 0 : offsets.back(); + assert(end_row <= s); + size_t replicated_size = 0 == s ? 0 : (offsets[end_row - 1]); return ColumnConst::create(data, replicated_size); } + ColumnPtr ColumnConst::permute(const Permutation & perm, size_t limit) const { if (limit == 0) diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 39c7a90838c..0a37e641d25 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -171,7 +171,7 @@ class ColumnConst final : public COWPtrHelper void updateWeakHash32(WeakHash32 & hash, const TiDB::TiDBCollatorPtr &, String &) const override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; - ColumnPtr replicate(const Offsets & offsets) const override; + ColumnPtr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; diff --git a/dbms/src/Columns/ColumnDecimal.cpp b/dbms/src/Columns/ColumnDecimal.cpp index 8af6cd9b6e7..b2775b46086 100644 --- a/dbms/src/Columns/ColumnDecimal.cpp +++ b/dbms/src/Columns/ColumnDecimal.cpp @@ -325,21 +325,24 @@ ColumnPtr ColumnDecimal::filter(const IColumn::Filter & filt, ssize_t result_ } template -ColumnPtr ColumnDecimal::replicate(const IColumn::Offsets & offsets) const +ColumnPtr ColumnDecimal::replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const { size_t size = data.size(); if (size != offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + assert(start_row < end_row); + assert(end_row <= size); + auto res = this->create(0, scale); if (0 == size) return res; typename Self::Container & res_data = res->getData(); - res_data.reserve(offsets.back()); + res_data.reserve(offsets[end_row - 1]); IColumn::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) + for (size_t i = start_row; i < end_row; ++i) { size_t size_to_replicate = offsets[i] - prev_offset; prev_offset = offsets[i]; @@ -351,6 +354,7 @@ ColumnPtr ColumnDecimal::replicate(const IColumn::Offsets & offsets) const return res; } + template void ColumnDecimal::gather(ColumnGathererStream & gatherer) { diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index e06e53e143f..85ddfa4f627 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -167,7 +167,8 @@ class ColumnDecimal final : public COWPtrHelper ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; - ColumnPtr replicate(const IColumn::Offsets & offsets) const override; + ColumnPtr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const override; + void getExtremes(Field & min, Field & max) const override; MutableColumns scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const override diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index ef4a7b287c4..1d3070f128b 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -301,22 +301,25 @@ ColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t limit) con return res; } -ColumnPtr ColumnFixedString::replicate(const Offsets & offsets) const +ColumnPtr ColumnFixedString::replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const { - size_t col_size = size(); - if (col_size != offsets.size()) + size_t col_rows = size(); + if (col_rows != offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + assert(start_row < end_row); + assert(end_row <= col_rows); + auto res = ColumnFixedString::create(n); - if (0 == col_size) + if (0 == col_rows) return res; Chars_t & res_chars = res->chars; - res_chars.resize(n * offsets.back()); + res_chars.resize(n * (offsets[end_row - 1])); Offset curr_offset = 0; - for (size_t i = 0; i < col_size; ++i) + for (size_t i = start_row; i < end_row; ++i) for (size_t next_offset = offsets[i]; curr_offset < next_offset; ++curr_offset) memcpySmallAllowReadWriteOverflow15(&res->chars[curr_offset * n], &chars[i * n], n); diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index afb2379af05..9b211f620a5 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -134,7 +134,7 @@ class ColumnFixedString final : public COWPtrHelper ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr replicate(const Offsets & offsets) const override; + ColumnPtr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override { diff --git a/dbms/src/Columns/ColumnFunction.cpp b/dbms/src/Columns/ColumnFunction.cpp index 3e9a0cf5fc4..fa9725727d3 100644 --- a/dbms/src/Columns/ColumnFunction.cpp +++ b/dbms/src/Columns/ColumnFunction.cpp @@ -44,18 +44,21 @@ MutableColumnPtr ColumnFunction::cloneResized(size_t size) const return ColumnFunction::create(size, function, capture); } -ColumnPtr ColumnFunction::replicate(const Offsets & offsets) const +ColumnPtr ColumnFunction::replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const { if (column_size != offsets.size()) throw Exception( fmt::format("Size of offsets ({}) doesn't match size of column ({})", offsets.size(), column_size), ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + assert(start_row < end_row); + assert(end_row <= column_size); + ColumnsWithTypeAndName capture = captured_columns; for (auto & column : capture) - column.column = column.column->replicate(offsets); + column.column = column.column->replicateRange(start_row, end_row, offsets); - size_t replicated_size = 0 == column_size ? 0 : offsets.back(); + size_t replicated_size = 0 == column_size ? 0 : (offsets[end_row - 1]); return ColumnFunction::create(replicated_size, function, capture); } @@ -68,20 +71,20 @@ ColumnPtr ColumnFunction::cut(size_t start, size_t length) const return ColumnFunction::create(length, function, capture); } -ColumnPtr ColumnFunction::filter(const Filter & filt, ssize_t result_size_hint) const +ColumnPtr ColumnFunction::filter(const Filter & filter, ssize_t result_size_hint) const { - if (column_size != filt.size()) + if (column_size != filter.size()) throw Exception( - fmt::format("Size of filter ({}) doesn't match size of column ({})", filt.size(), column_size), + fmt::format("Size of filter ({}) doesn't match size of column ({})", filter.size(), column_size), ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); ColumnsWithTypeAndName capture = captured_columns; for (auto & column : capture) - column.column = column.column->filter(filt, result_size_hint); + column.column = column.column->filter(filter, result_size_hint); size_t filtered_size = 0; if (capture.empty()) - filtered_size = countBytesInFilter(filt); + filtered_size = countBytesInFilter(filter); else filtered_size = capture.front().column->size(); diff --git a/dbms/src/Columns/ColumnFunction.h b/dbms/src/Columns/ColumnFunction.h index 11c42ec4645..d03aa26941c 100644 --- a/dbms/src/Columns/ColumnFunction.h +++ b/dbms/src/Columns/ColumnFunction.h @@ -41,8 +41,8 @@ class ColumnFunction final : public COWPtrHelper size_t size() const override { return column_size; } ColumnPtr cut(size_t start, size_t length) const override; - ColumnPtr replicate(const Offsets & offsets) const override; - ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; + ColumnPtr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const override; + ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; void insertDefault() override; void popBack(size_t n) override; diff --git a/dbms/src/Columns/ColumnNullable.cpp b/dbms/src/Columns/ColumnNullable.cpp index 558b5574675..1f22825607f 100644 --- a/dbms/src/Columns/ColumnNullable.cpp +++ b/dbms/src/Columns/ColumnNullable.cpp @@ -524,14 +524,13 @@ void ColumnNullable::getExtremes(Field & min, Field & max) const }); } -ColumnPtr ColumnNullable::replicate(const Offsets & offsets) const +ColumnPtr ColumnNullable::replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const { - ColumnPtr replicated_data = getNestedColumn().replicate(offsets); - ColumnPtr replicated_null_map = getNullMapColumn().replicate(offsets); + ColumnPtr replicated_data = getNestedColumn().replicateRange(start_row, end_row, offsets); + ColumnPtr replicated_null_map = getNullMapColumn().replicateRange(start_row, end_row, offsets); return ColumnNullable::create(replicated_data, replicated_null_map); } - template void ColumnNullable::applyNullMapImpl(const ColumnUInt8 & map) { diff --git a/dbms/src/Columns/ColumnNullable.h b/dbms/src/Columns/ColumnNullable.h index f35274a2831..d9b7bba7968 100644 --- a/dbms/src/Columns/ColumnNullable.h +++ b/dbms/src/Columns/ColumnNullable.h @@ -92,7 +92,7 @@ class ColumnNullable final : public COWPtrHelper size_t byteSize() const override; size_t byteSize(size_t offset, size_t limit) const override; size_t allocatedBytes() const override; - ColumnPtr replicate(const Offsets & replicate_offsets) const override; + ColumnPtr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & replicate_offsets) const override; void updateHashWithValue(size_t n, SipHash & hash, const TiDB::TiDBCollatorPtr &, String &) const override; void updateHashWithValues(IColumn::HashValues & hash_values, const TiDB::TiDBCollatorPtr &, String &) const override; void updateWeakHash32(WeakHash32 & hash, const TiDB::TiDBCollatorPtr &, String &) const override; diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index 32673b9e14f..fe6716d2822 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -230,28 +230,30 @@ void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_directio } } - -ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const +ColumnPtr ColumnString::replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & replicate_offsets) const { - size_t col_size = size(); - if (col_size != replicate_offsets.size()) + size_t col_rows = size(); + if (col_rows != replicate_offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + assert(start_row < end_row); + assert(end_row <= col_rows); + auto res = ColumnString::create(); - if (0 == col_size) + if (0 == col_rows) return res; Chars_t & res_chars = res->chars; Offsets & res_offsets = res->offsets; - res_chars.reserve(chars.size() / col_size * replicate_offsets.back()); - res_offsets.reserve(replicate_offsets.back()); + res_chars.reserve(chars.size() / col_rows * (replicate_offsets[end_row - 1])); + res_offsets.reserve(replicate_offsets[end_row - 1]); Offset prev_replicate_offset = 0; - Offset prev_string_offset = 0; + Offset prev_string_offset = start_row == 0 ? 0 : offsets[start_row - 1]; Offset current_new_offset = 0; - for (size_t i = 0; i < col_size; ++i) + for (size_t i = start_row; i < end_row; ++i) { size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset; size_t string_size = offsets[i] - prev_string_offset; diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index c5e88ea4c4b..7cb16bbe685 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -301,7 +301,7 @@ class ColumnString final : public COWPtrHelper /// Sorting with respect of collation. void getPermutationWithCollationImpl(const ICollator & collator, bool reverse, size_t limit, Permutation & res) const; - ColumnPtr replicate(const Offsets & replicate_offsets) const override; + ColumnPtr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & replicate_offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override { @@ -320,14 +320,29 @@ class ColumnString final : public COWPtrHelper void getExtremes(Field & min, Field & max) const override; - bool canBeInsideNullable() const override { return true; } + bool canBeInsideNullable() const override + { + return true; + } - Chars_t & getChars() { return chars; } - const Chars_t & getChars() const { return chars; } + Chars_t & getChars() + { + return chars; + } + const Chars_t & getChars() const + { + return chars; + } - Offsets & getOffsets() { return offsets; } - const Offsets & getOffsets() const { return offsets; } + Offsets & getOffsets() + { + return offsets; + } + const Offsets & getOffsets() const + { + return offsets; + } }; diff --git a/dbms/src/Columns/ColumnTuple.cpp b/dbms/src/Columns/ColumnTuple.cpp index 80f652219ad..333506fde42 100644 --- a/dbms/src/Columns/ColumnTuple.cpp +++ b/dbms/src/Columns/ColumnTuple.cpp @@ -211,13 +211,13 @@ ColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const return ColumnTuple::create(new_columns); } -ColumnPtr ColumnTuple::replicate(const Offsets & offsets) const +ColumnPtr ColumnTuple::replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const { const size_t tuple_size = columns.size(); Columns new_columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) - new_columns[i] = columns[i]->replicate(offsets); + new_columns[i] = columns[i]->replicateRange(start_row, end_row, offsets); return ColumnTuple::create(new_columns); } diff --git a/dbms/src/Columns/ColumnTuple.h b/dbms/src/Columns/ColumnTuple.h index 8f69b2f6ca4..16fe3d86dd2 100644 --- a/dbms/src/Columns/ColumnTuple.h +++ b/dbms/src/Columns/ColumnTuple.h @@ -78,7 +78,7 @@ class ColumnTuple final : public COWPtrHelper void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr replicate(const Offsets & offsets) const override; + ColumnPtr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; void scatterTo(ScatterColumns & scatterColumns, const Selector & selector) const override; void gather(ColumnGathererStream & gatherer_stream) override; diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 3ea8af02309..7b7785940d3 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -288,27 +288,34 @@ ColumnPtr ColumnVector::permute(const IColumn::Permutation & perm, size_t lim } template -ColumnPtr ColumnVector::replicate(const IColumn::Offsets & offsets) const +ColumnPtr ColumnVector::replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const { size_t size = data.size(); if (size != offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + assert(start_row < end_row); + assert(end_row <= size); + if (0 == size) return this->create(); auto res = this->create(); typename Self::Container & res_data = res->getData(); - res_data.reserve(offsets.back()); + + res_data.reserve(offsets[end_row - 1]); IColumn::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) + + for (size_t i = start_row; i < end_row; ++i) { size_t size_to_replicate = offsets[i] - prev_offset; prev_offset = offsets[i]; for (size_t j = 0; j < size_to_replicate; ++j) + { res_data.push_back(data[i]); + } } return res; diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 95f33588161..60cc0063108 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -366,7 +366,7 @@ class ColumnVector final : public COWPtrHelper */ using Offset = UInt64; using Offsets = PaddedPODArray; - virtual Ptr replicate(const Offsets & offsets) const = 0; + + virtual Ptr replicateRange(size_t start_row, size_t end_row, const IColumn::Offsets & offsets) const = 0; + + Ptr replicate(const Offsets & offsets) const + { + return replicateRange(0, offsets.size(), offsets); + } /** Split column to smaller columns. Each value goes to column index, selected by corresponding element of 'selector'. * Selector must contain values from 0 to num_columns - 1. diff --git a/dbms/src/Columns/IColumnDummy.h b/dbms/src/Columns/IColumnDummy.h index 854c00dd06e..f00554c1d01 100644 --- a/dbms/src/Columns/IColumnDummy.h +++ b/dbms/src/Columns/IColumnDummy.h @@ -120,14 +120,15 @@ class IColumnDummy : public IColumn res[i] = i; } - ColumnPtr replicate(const Offsets & offsets) const override + ColumnPtr replicateRange(size_t /*start_row*/, size_t end_row, const IColumn::Offsets & offsets) const override { if (s != offsets.size()) throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - return cloneDummy(s == 0 ? 0 : offsets.back()); + assert(end_row <= s); + return cloneDummy(s == 0 ? 0 : offsets[end_row - 1]); } + MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override { if (s != selector.size()) diff --git a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp index a5a293f3e43..b7ae64cfafc 100644 --- a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp +++ b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp @@ -20,9 +20,11 @@ namespace DB HashJoinProbeBlockInputStream::HashJoinProbeBlockInputStream( const BlockInputStreamPtr & input, const JoinPtr & join_, - const String & req_id) + const String & req_id, + UInt64 max_block_size) : log(Logger::get(req_id)) , join(join_) + , probe_process_info(max_block_size) { children.push_back(input); @@ -50,28 +52,32 @@ Block HashJoinProbeBlockInputStream::getTotals() } join->joinTotals(totals); } + return totals; } Block HashJoinProbeBlockInputStream::getHeader() const { Block res = children.back()->getHeader(); - join->joinBlock(res); - return res; + assert(res.rows() == 0); + ProbeProcessInfo header_probe_process_info(0); + header_probe_process_info.resetBlock(std::move(res)); + return join->joinBlock(header_probe_process_info); } Block HashJoinProbeBlockInputStream::readImpl() { - Block res = children.back()->read(); - if (!res) - return res; - - join->joinBlock(res); - - // TODO split block if block.size() > settings.max_block_size - // https://github.com/pingcap/tiflash/issues/3436 + if (probe_process_info.all_rows_joined_finish) + { + Block block = children.back()->read(); + if (!block) + return block; + join->checkTypes(block); + probe_process_info.resetBlock(std::move(block)); + } - return res; + return join->joinBlock(probe_process_info); } + } // namespace DB diff --git a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h index df68d041994..3cc6fc4af6b 100644 --- a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h +++ b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h @@ -19,7 +19,7 @@ namespace DB { -class ExpressionActions; + /** Executes a certain expression over the block. * Basically the same as ExpressionBlockInputStream, @@ -38,7 +38,8 @@ class HashJoinProbeBlockInputStream : public IProfilingBlockInputStream HashJoinProbeBlockInputStream( const BlockInputStreamPtr & input, const JoinPtr & join_, - const String & req_id); + const String & req_id, + UInt64 max_block_size); String getName() const override { return name; } Block getTotals() override; @@ -50,6 +51,7 @@ class HashJoinProbeBlockInputStream : public IProfilingBlockInputStream private: const LoggerPtr log; JoinPtr join; + ProbeProcessInfo probe_process_info; }; } // namespace DB diff --git a/dbms/src/DataStreams/MockExchangeReceiverInputStream.h b/dbms/src/DataStreams/MockExchangeReceiverInputStream.h index c87f7d9b89d..c61cdb453cf 100644 --- a/dbms/src/DataStreams/MockExchangeReceiverInputStream.h +++ b/dbms/src/DataStreams/MockExchangeReceiverInputStream.h @@ -29,7 +29,7 @@ class MockExchangeReceiverInputStream : public IProfilingBlockInputStream MockExchangeReceiverInputStream(ColumnsWithTypeAndName columns, size_t max_block_size); Block getHeader() const override { - return Block(columns); + return Block(columns).cloneEmpty(); } String getName() const override { return "MockExchangeReceiver"; } size_t getSourceNum() const { return source_num; } diff --git a/dbms/src/DataStreams/MockTableScanBlockInputStream.h b/dbms/src/DataStreams/MockTableScanBlockInputStream.h index c3721485c5e..fa5a9c0ef7a 100644 --- a/dbms/src/DataStreams/MockTableScanBlockInputStream.h +++ b/dbms/src/DataStreams/MockTableScanBlockInputStream.h @@ -24,7 +24,7 @@ class MockTableScanBlockInputStream : public IProfilingBlockInputStream MockTableScanBlockInputStream(ColumnsWithTypeAndName columns, size_t max_block_size, bool is_infinite_ = false); Block getHeader() const override { - return Block(columns); + return Block(columns).cloneEmpty(); } String getName() const override { return "MockTableScan"; } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index db7f2a0f9ae..2902d66b57a 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -331,7 +331,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & } for (auto & stream : pipeline.streams) { - stream = std::make_shared(stream, join_ptr, log->identifier()); + stream = std::make_shared(stream, join_ptr, log->identifier(), settings.max_block_size); stream->setExtraInfo(fmt::format("join probe, join_executor_id = {}", query_block.source_name)); } diff --git a/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp b/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp index 91d4e489c65..6cf94d9beae 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp @@ -206,7 +206,7 @@ void PhysicalJoin::probeSideTransform(DAGPipeline & probe_pipeline, Context & co String join_probe_extra_info = fmt::format("join probe, join_executor_id = {}", execId()); for (auto & stream : probe_pipeline.streams) { - stream = std::make_shared(stream, join_ptr, log->identifier()); + stream = std::make_shared(stream, join_ptr, log->identifier(), settings.max_block_size); stream->setExtraInfo(join_probe_extra_info); } } diff --git a/dbms/src/Flash/tests/gtest_join_executor.cpp b/dbms/src/Flash/tests/gtest_join_executor.cpp index de98df0c997..b7e3ff58683 100644 --- a/dbms/src/Flash/tests/gtest_join_executor.cpp +++ b/dbms/src/Flash/tests/gtest_join_executor.cpp @@ -537,6 +537,155 @@ try } CATCH +TEST_F(JoinExecutorTestRunner, JoinWithNullTable) +try +{ + context.addMockTable("null_test", "t", {{"a", TiDB::TP::TypeLong}, {"b", TiDB::TP::TypeLong}, {"c", TiDB::TP::TypeLong}}, {toVec("a", {1, 2, 3, 4, 5, 6, 7, 8, 9, 0}), toVec("b", {1, 1, 1, 1, 1, 1, 1, 2, 2, 2}), toVec("c", {1, 1, 1, 1, 1, 2, 2, 2, 2, 2})}); + context.addMockTable("null_test", "null_table", {{"a", TiDB::TP::TypeLong}, {"b", TiDB::TP::TypeLong}, {"c", TiDB::TP::TypeLong}}, {toVec("a", {}), toVec("b", {}), toVec("c", {})}); + + std::shared_ptr request; + + // inner join + { + // null table join non-null table + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "t"), tipb::JoinType::TypeInnerJoin, {col("a")}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + // non-null table join null table + request = context.scan("null_test", "t") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeInnerJoin, {col("a")}) + .build(context); + executeAndAssertColumnsEqual(request, {toNullableVec({}), toNullableVec({}), toNullableVec({}), toNullableVec({}), toNullableVec({}), toNullableVec({})}); + + // null table join null table + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeInnerJoin, {col("a")}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + } + + // cross join + const auto cond = gt(col("a"), lit(Field(static_cast(5)))); + // non-null table join null table + { + request = context.scan("null_test", "t") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeInnerJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {toNullableVec({}), toNullableVec({}), toNullableVec({}), toNullableVec({}), toNullableVec({}), toNullableVec({})}); + + request = context.scan("null_test", "t") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeLeftOuterJoin, {}, {cond}, {}, {}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {toNullableVec({1, 2, 3, 4, 5, 6, 7, 8, 9, 0}), toNullableVec({1, 1, 1, 1, 1, 1, 1, 2, 2, 2}), toNullableVec({1, 1, 1, 1, 1, 2, 2, 2, 2, 2}), toNullableVec({{}, {}, {}, {}, {}, {}, {}, {}, {}, {}}), toNullableVec({{}, {}, {}, {}, {}, {}, {}, {}, {}, {}}), toNullableVec({{}, {}, {}, {}, {}, {}, {}, {}, {}, {}})}); + + request = context.scan("null_test", "t") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeRightOuterJoin, {}, {}, {cond}, {}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "t") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {toNullableVec({}), toNullableVec({}), toNullableVec({})}); + + request = context.scan("null_test", "t") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeAntiSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {toNullableVec({1, 2, 3, 4, 5, 6, 7, 8, 9, 0}), toNullableVec({1, 1, 1, 1, 1, 1, 1, 2, 2, 2}), toNullableVec({1, 1, 1, 1, 1, 2, 2, 2, 2, 2})}); + + request = context.scan("null_test", "t") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeLeftOuterSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + // the 4th col is left semi helper col. + executeAndAssertColumnsEqual(request, {toNullableVec({1, 2, 3, 4, 5, 6, 7, 8, 9, 0}), toNullableVec({1, 1, 1, 1, 1, 1, 1, 2, 2, 2}), toNullableVec({1, 1, 1, 1, 1, 2, 2, 2, 2, 2}), toNullableVec({0, 0, 0, 0, 0, 0, 0, 0, 0, 0})}); + + request = context.scan("null_test", "t") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeAntiLeftOuterSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + // the 4th col is left semi helper col. + executeAndAssertColumnsEqual(request, {toNullableVec({1, 2, 3, 4, 5, 6, 7, 8, 9, 0}), toNullableVec({1, 1, 1, 1, 1, 1, 1, 2, 2, 2}), toNullableVec({1, 1, 1, 1, 1, 2, 2, 2, 2, 2}), toNullableVec({1, 1, 1, 1, 1, 1, 1, 1, 1, 1})}); + } + + // null table join non-null table + { + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "t"), tipb::JoinType::TypeInnerJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "t"), tipb::JoinType::TypeLeftOuterJoin, {}, {cond}, {}, {}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "t"), tipb::JoinType::TypeRightOuterJoin, {}, {}, {cond}, {}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {toNullableVec({{}, {}, {}, {}, {}, {}, {}, {}, {}, {}}), toNullableVec({{}, {}, {}, {}, {}, {}, {}, {}, {}, {}}), toNullableVec({{}, {}, {}, {}, {}, {}, {}, {}, {}, {}}), toNullableVec({1, 2, 3, 4, 5, 6, 7, 8, 9, 0}), toNullableVec({1, 1, 1, 1, 1, 1, 1, 2, 2, 2}), toNullableVec({1, 1, 1, 1, 1, 2, 2, 2, 2, 2})}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "t"), tipb::JoinType::TypeSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "t"), tipb::JoinType::TypeAntiSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "t"), tipb::JoinType::TypeLeftOuterSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "t"), tipb::JoinType::TypeAntiLeftOuterSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + } + + // null table join null table + { + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeInnerJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeLeftOuterJoin, {}, {cond}, {}, {}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeRightOuterJoin, {}, {}, {cond}, {}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeAntiSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeLeftOuterSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + + request = context.scan("null_test", "null_table") + .join(context.scan("null_test", "null_table"), tipb::JoinType::TypeAntiLeftOuterSemiJoin, {}, {}, {}, {cond}, {}) + .build(context); + executeAndAssertColumnsEqual(request, {}); + } +} +CATCH + // Currently only support join with `using` TEST_F(JoinExecutorTestRunner, RawQuery) @@ -548,5 +697,31 @@ try } CATCH +TEST_F(JoinExecutorTestRunner, SplitJoinResult) +try +{ + context.addMockTable("split_test", "t1", {{"a", TiDB::TP::TypeLong}}, {toVec("a", {1, 1, 1, 1, 1, 1, 1, 1, 1, 1})}); + context.addMockTable("split_test", "t2", {{"a", TiDB::TP::TypeLong}}, {toVec("a", {1, 1, 1, 1, 1})}); + + auto request = context + .scan("split_test", "t1") + .join(context.scan("split_test", "t2"), tipb::JoinType::TypeInnerJoin, {col("a")}) + .build(context); + + std::vector block_sizes{1, 2, 7, 25, 49, 50, 51, DEFAULT_BLOCK_SIZE}; + std::vector> expect{{5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, {5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, {5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, {25, 25}, {45, 5}, {50}, {50}, {50}}; + for (size_t i = 0; i < block_sizes.size(); ++i) + { + context.context.setSetting("max_block_size", Field(static_cast(block_sizes[i]))); + auto blocks = getExecuteStreamsReturnBlocks(request); + ASSERT_EQ(expect[i].size(), blocks.size()); + for (size_t j = 0; j < blocks.size(); ++j) + { + ASSERT_EQ(expect[i][j], blocks[j].rows()); + } + } +} +CATCH + } // namespace tests } // namespace DB diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 099303442c4..784a095ccdc 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -306,9 +306,12 @@ void ExpressionAction::execute(Block & block) const break; } + //TODO: Clean up all Join logic in ExpressionAction case JOIN: { - join->joinBlock(block); + ProbeProcessInfo probe_process_info(0); + probe_process_info.block = block; + join->joinBlock(probe_process_info); break; } diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 9044c160db9..fc4043d5006 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -968,74 +968,83 @@ struct Adder; template struct Adder { - static void addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t /*i*/, IColumn::Filter * /*filter*/, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & right_indexes) + static bool addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t /*i*/, IColumn::Filter * /*filter*/, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & right_indexes, ProbeProcessInfo & /*probe_process_info*/) { for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertFrom(*it->getMapped().block->getByPosition(right_indexes[j]).column.get(), it->getMapped().row_num); + return false; } - static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t /*i*/, IColumn::Filter * /*filter*/, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/) + static bool addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t /*i*/, IColumn::Filter * /*filter*/, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, ProbeProcessInfo & /*probe_process_info*/) { for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertDefault(); + return false; } }; template struct Adder { - static void addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & right_indexes) + static bool addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & right_indexes, ProbeProcessInfo & /*probe_process_info*/) { (*filter)[i] = 1; for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertFrom(*it->getMapped().block->getByPosition(right_indexes[j]).column.get(), it->getMapped().row_num); + + return false; } - static void addNotFound(size_t /*num_columns_to_add*/, MutableColumns & /*added_columns*/, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/) + static bool addNotFound(size_t /*num_columns_to_add*/, MutableColumns & /*added_columns*/, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, ProbeProcessInfo & /*probe_process_info*/) { (*filter)[i] = 0; + return false; } }; template struct Adder { - static void addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & /*it*/, size_t /*num_columns_to_add*/, MutableColumns & /*added_columns*/, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & /*right_indexes*/) + static bool addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & /*it*/, size_t /*num_columns_to_add*/, MutableColumns & /*added_columns*/, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & /*right_indexes*/, ProbeProcessInfo & /*probe_process_info*/) { (*filter)[i] = 0; + return false; } - static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/) + static bool addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, ProbeProcessInfo & /*probe_process_info*/) { (*filter)[i] = 1; for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertDefault(); + return false; } }; template struct Adder { - static void addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & /*it*/, size_t num_columns_to_add, MutableColumns & added_columns, size_t /*i*/, IColumn::Filter * /*filter*/, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & /*right_indexes*/) + static bool addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & /*it*/, size_t num_columns_to_add, MutableColumns & added_columns, size_t /*i*/, IColumn::Filter * /*filter*/, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, const std::vector & /*right_indexes*/, ProbeProcessInfo & /*probe_process_info*/) { for (size_t j = 0; j < num_columns_to_add - 1; ++j) added_columns[j]->insertDefault(); added_columns[num_columns_to_add - 1]->insert(FIELD_INT8_1); + return false; } - static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t /*i*/, IColumn::Filter * /*filter*/, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/) + static bool addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t /*i*/, IColumn::Filter * /*filter*/, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, ProbeProcessInfo & /*probe_process_info*/) { for (size_t j = 0; j < num_columns_to_add - 1; ++j) added_columns[j]->insertDefault(); added_columns[num_columns_to_add - 1]->insert(FIELD_INT8_0); + return false; } }; template struct Adder { - static void addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * /*filter*/, IColumn::Offset & current_offset, IColumn::Offsets * offsets, const std::vector & right_indexes) + static bool addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * /*filter*/, IColumn::Offset & current_offset, IColumn::Offsets * offsets, const std::vector & right_indexes, ProbeProcessInfo & /*probe_process_info*/) { for (auto current = &static_cast(it->getMapped()); current != nullptr; current = current->next) { @@ -1047,9 +1056,10 @@ struct Adder /// we insert only one row to `match-helper` for each row of left block /// so before the execution of `HandleOtherConditions`, column sizes of temporary block may be different. added_columns[num_columns_to_add - 1]->insert(FIELD_INT8_1); + return false; } - static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * /*filter*/, IColumn::Offset & current_offset, IColumn::Offsets * offsets) + static bool addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * /*filter*/, IColumn::Offset & current_offset, IColumn::Offsets * offsets, ProbeProcessInfo & /*probe_process_info*/) { ++current_offset; (*offsets)[i] = current_offset; @@ -1057,21 +1067,31 @@ struct Adder for (size_t j = 0; j < num_columns_to_add - 1; ++j) added_columns[j]->insertDefault(); added_columns[num_columns_to_add - 1]->insert(FIELD_INT8_0); + return false; } }; template struct Adder { - static void addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets, const std::vector & right_indexes) + static bool addFound(const typename Map::SegmentType::HashTable::ConstLookupResult & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets, const std::vector & right_indexes, ProbeProcessInfo & probe_process_info) { size_t rows_joined = 0; + // If there are too many rows in the column to split, record the number of rows that have been expanded for next read. + // and it means the rows in this block are not joined finish. + + for (auto current = &static_cast(it->getMapped()); current != nullptr; current = current->next) + ++rows_joined; + + if (current_offset && current_offset + rows_joined > probe_process_info.max_block_size) + { + return true; + } + for (auto current = &static_cast(it->getMapped()); current != nullptr; current = current->next) { for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertFrom(*current->block->getByPosition(right_indexes[j]).column.get(), current->row_num); - - ++rows_joined; } current_offset += rows_joined; @@ -1081,9 +1101,11 @@ struct Adder /// away because it might failed in other condition, so we add the matched rows to the result, but set (*filter)[i] = 0 /// to indicate that the row is matched during probe stage, this will be used in handleOtherConditions (*filter)[i] = 0; + + return false; } - static void addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets) + static bool addNotFound(size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets, ProbeProcessInfo & probe_process_info) { if (KIND == ASTTableJoin::Kind::Inner) { @@ -1091,6 +1113,10 @@ struct Adder } else { + if (current_offset && current_offset + 1 > probe_process_info.max_block_size) + { + return true; + } if (KIND == ASTTableJoin::Kind::Anti) (*filter)[i] = 1; ++current_offset; @@ -1099,6 +1125,7 @@ struct Adder for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertDefault(); } + return false; } }; @@ -1116,8 +1143,17 @@ void NO_INLINE joinBlockImplTypeCase( const std::vector & right_indexes, const TiDB::TiDBCollators & collators, bool enable_fine_grained_shuffle, - size_t fine_grained_shuffle_count) + size_t fine_grained_shuffle_count, + ProbeProcessInfo & probe_process_info) { + if (rows == 0) + { + probe_process_info.all_rows_joined_finish = true; + return; + } + + assert(probe_process_info.start_row < rows); + size_t num_columns_to_add = right_indexes.size(); KeyGetter key_getter(key_columns, key_sizes, collators); @@ -1140,17 +1176,21 @@ void NO_INLINE joinBlockImplTypeCase( size_t segment_size = map.getSegmentSize(); const auto & shuffle_hash_data = shuffle_hash.getData(); - for (size_t i = 0; i < rows; ++i) + assert(probe_process_info.start_row < rows); + size_t i; + bool block_full = false; + for (i = probe_process_info.start_row; i < rows; ++i) { if (has_null_map && (*null_map)[i]) { - Adder::addNotFound( + block_full = Adder::addNotFound( num_columns_to_add, added_columns, i, filter.get(), current_offset, - offsets_to_replicate.get()); + offsets_to_replicate.get(), + probe_process_info); } else { @@ -1190,7 +1230,7 @@ void NO_INLINE joinBlockImplTypeCase( if (it != internal_map.end()) { it->getMapped().setUsed(); - Adder::addFound( + block_full = Adder::addFound( it, num_columns_to_add, added_columns, @@ -1198,19 +1238,31 @@ void NO_INLINE joinBlockImplTypeCase( filter.get(), current_offset, offsets_to_replicate.get(), - right_indexes); + right_indexes, + probe_process_info); } else - Adder::addNotFound( + block_full = Adder::addNotFound( num_columns_to_add, added_columns, i, filter.get(), current_offset, - offsets_to_replicate.get()); + offsets_to_replicate.get(), + probe_process_info); keyHolderDiscardKey(key_holder); } + + // if block_full is true means that the current offset is greater than max_block_size, we need break the loop. + if (block_full) + { + break; + } } + + probe_process_info.end_row = i; + // if i == rows, it means that all probe rows have been joined finish. + probe_process_info.all_rows_joined_finish = (i == rows); } template @@ -1227,7 +1279,8 @@ void joinBlockImplType( const std::vector & right_indexes, const TiDB::TiDBCollators & collators, bool enable_fine_grained_shuffle, - size_t fine_grained_shuffle_count) + size_t fine_grained_shuffle_count, + ProbeProcessInfo & probe_process_info) { if (null_map) joinBlockImplTypeCase( @@ -1243,7 +1296,8 @@ void joinBlockImplType( right_indexes, collators, enable_fine_grained_shuffle, - fine_grained_shuffle_count); + fine_grained_shuffle_count, + probe_process_info); else joinBlockImplTypeCase( map, @@ -1258,7 +1312,8 @@ void joinBlockImplType( right_indexes, collators, enable_fine_grained_shuffle, - fine_grained_shuffle_count); + fine_grained_shuffle_count, + probe_process_info); } } // namespace @@ -1485,7 +1540,7 @@ void Join::handleOtherConditions(Block & block, std::unique_ptr } template -void Join::joinBlockImpl(Block & block, const Maps & maps) const +void Join::joinBlockImpl(Block & block, const Maps & maps, ProbeProcessInfo & probe_process_info) const { size_t keys_size = key_names_left.size(); ColumnRawPtrs key_columns(keys_size); @@ -1606,7 +1661,8 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const right_indexes, \ collators, \ enable_fine_grained_shuffle, \ - fine_grained_shuffle_count); \ + fine_grained_shuffle_count, \ + probe_process_info); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -1621,15 +1677,40 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const block.insert(ColumnWithTypeAndName(std::move(added_columns[i]), sample_col.type, sample_col.name)); } - /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. - if (filter && !(kind == ASTTableJoin::Kind::Anti && strictness == ASTTableJoin::Strictness::All)) - for (size_t i = 0; i < existing_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(*filter, -1); + size_t process_rows = probe_process_info.end_row - probe_process_info.start_row; - /// If ALL ... JOIN - we replicate all the columns except the new ones. - if (offsets_to_replicate) - for (size_t i = 0; i < existing_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate); + // if rows equal 0, we could ignore filter and offsets_to_replicate, and do not need to update start row. + if (likely(rows != 0)) + { + /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. + if (filter && !(kind == ASTTableJoin::Kind::Anti && strictness == ASTTableJoin::Strictness::All)) + { + // If ANY INNER | RIGHT JOIN, the result will not be spilt, so the block rows must equal process_rows. + RUNTIME_CHECK(rows == process_rows); + for (size_t i = 0; i < existing_columns; ++i) + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(*filter, -1); + } + + + /// If ALL ... JOIN - we replicate all the columns except the new ones. + if (offsets_to_replicate) + { + for (size_t i = 0; i < existing_columns; ++i) + { + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicateRange(probe_process_info.start_row, probe_process_info.end_row, *offsets_to_replicate); + } + + if (rows != process_rows) + { + if (isLeftSemiFamily(kind)) + { + auto helper_col = block.getByName(match_helper_name).column; + helper_col = helper_col->cut(probe_process_info.start_row, probe_process_info.end_row); + } + offsets_to_replicate->assign(offsets_to_replicate->begin() + probe_process_info.start_row, offsets_to_replicate->begin() + probe_process_info.end_row); + } + } + } /// handle other conditions if (!other_filter_column.empty() || !other_eq_filter_from_in_column.empty()) @@ -1891,6 +1972,10 @@ void Join::joinBlockImplCross(Block & block) const joinBlockImplCrossInternal(block, nullptr); } +void Join::checkTypes(const Block & block) const +{ + checkTypesOfKeys(block, sample_block_with_keys); +} void Join::checkTypesOfKeys(const Block & block_left, const Block & block_right) const { @@ -1911,10 +1996,8 @@ void Join::checkTypesOfKeys(const Block & block_left, const Block & block_right) } } -void Join::joinBlock(Block & block) const +Block Join::joinBlock(ProbeProcessInfo & probe_process_info) const { - // std::cerr << "joinBlock: " << block.dumpStructure() << "\n"; - // ck will use this function to generate header, that's why here is a check. { std::unique_lock lk(build_table_mutex); @@ -1926,40 +2009,42 @@ void Join::joinBlock(Block & block) const std::shared_lock lock(rwlock); - checkTypesOfKeys(block, sample_block_with_keys); + probe_process_info.updateStartRow(); + + Block block = probe_process_info.block; /// TODO: after we bumping to C++20, use `using enum` to simplify code here. /// using enum ASTTableJoin::Strictness; /// using enum ASTTableJoin::Kind; if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl(block, maps_any); + joinBlockImpl(block, maps_any, probe_process_info); else if (kind == ASTTableJoin::Kind::Inner && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl(block, maps_any); + joinBlockImpl(block, maps_any, probe_process_info); else if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl(block, maps_all); + joinBlockImpl(block, maps_all, probe_process_info); else if (kind == ASTTableJoin::Kind::Inner && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl(block, maps_all); + joinBlockImpl(block, maps_all, probe_process_info); else if (kind == ASTTableJoin::Kind::Full && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl(block, maps_any_full); + joinBlockImpl(block, maps_any_full, probe_process_info); else if (kind == ASTTableJoin::Kind::Right && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl(block, maps_any_full); + joinBlockImpl(block, maps_any_full, probe_process_info); else if (kind == ASTTableJoin::Kind::Full && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl(block, maps_all_full); + joinBlockImpl(block, maps_all_full, probe_process_info); else if (kind == ASTTableJoin::Kind::Right && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl(block, maps_all_full); + joinBlockImpl(block, maps_all_full, probe_process_info); else if (kind == ASTTableJoin::Kind::Anti && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl(block, maps_any); + joinBlockImpl(block, maps_any, probe_process_info); else if (kind == ASTTableJoin::Kind::Anti && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl(block, maps_all); + joinBlockImpl(block, maps_all, probe_process_info); else if (kind == ASTTableJoin::Kind::LeftSemi && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl(block, maps_any); + joinBlockImpl(block, maps_any, probe_process_info); else if (kind == ASTTableJoin::Kind::LeftSemi && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl(block, maps_all); + joinBlockImpl(block, maps_all, probe_process_info); else if (kind == ASTTableJoin::Kind::LeftAnti && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl(block, maps_any); + joinBlockImpl(block, maps_any, probe_process_info); else if (kind == ASTTableJoin::Kind::LeftAnti && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl(block, maps_all); + joinBlockImpl(block, maps_all, probe_process_info); else if (kind == ASTTableJoin::Kind::Cross && strictness == ASTTableJoin::Strictness::All) joinBlockImplCross(block); else if (kind == ASTTableJoin::Kind::Cross && strictness == ASTTableJoin::Strictness::Any) @@ -1997,6 +2082,13 @@ void Join::joinBlock(Block & block) const block.getByName(match_helper_name).column = ColumnNullable::create(std::move(col_non_matched), std::move(nullable_column->getNullMapColumnPtr())); } + + if (isCrossJoin(kind)) + { + probe_process_info.all_rows_joined_finish = true; + } + + return block; } void Join::joinTotals(Block & block) const @@ -2025,7 +2117,6 @@ void Join::joinTotals(Block & block) const } } - template struct AdderNonJoined; @@ -2301,4 +2392,19 @@ BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sampl return std::make_shared(*this, left_sample_block, index, step, max_block_size); } +void ProbeProcessInfo::resetBlock(Block && block_) +{ + block = std::move(block_); + start_row = 0; + end_row = 0; + all_rows_joined_finish = false; + // If the probe block size is greater than max_block_size, we will set max_block_size to the probe block size to avoid some unnecessary split. + max_block_size = std::max(max_block_size, block.rows()); +} + +void ProbeProcessInfo::updateStartRow() +{ + assert(start_row <= end_row); + start_row = end_row; +} } // namespace DB diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 8c2332e4903..0565e0fe79e 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -29,9 +29,9 @@ #include - namespace DB { +struct ProbeProcessInfo; /** Data structure for implementation of JOIN. * It is just a hash table: keys -> rows of joined ("right") table. * Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys. @@ -120,7 +120,9 @@ class Join /** Join data from the map (that was previously built by calls to insertFromBlock) to the block with data from "left" table. * Could be called from different threads in parallel. */ - void joinBlock(Block & block) const; + Block joinBlock(ProbeProcessInfo & probe_process_info) const; + + void checkTypes(const Block & block) const; /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. */ @@ -191,7 +193,6 @@ class Join {} }; - /** Depending on template parameter, adds or doesn't add a flag, that element was used (row was joined). * For implementation of RIGHT and FULL JOINs. * NOTE: It is possible to store the flag in one bit of pointer to block or row_num. It seems not reasonable, because memory saving is minimal. @@ -273,6 +274,7 @@ class Join // only use for left semi joins. const String match_helper_name; + private: friend class NonJoinedBlockInputStream; @@ -288,6 +290,8 @@ class Join bool use_nulls; size_t build_concurrency; + +private: /// collators for the join key const TiDB::TiDBCollators collators; @@ -319,6 +323,7 @@ class Join /// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows. Arenas pools; + private: Type type = Type::EMPTY; @@ -383,7 +388,7 @@ class Join void insertFromBlockInternal(Block * stored_block, size_t stream_index); template - void joinBlockImpl(Block & block, const Maps & maps) const; + void joinBlockImpl(Block & block, const Maps & maps, ProbeProcessInfo & probe_process_info) const; /** Handle non-equal join conditions * @@ -402,5 +407,20 @@ class Join using JoinPtr = std::shared_ptr; using Joins = std::vector; +struct ProbeProcessInfo +{ + Block block; + UInt64 max_block_size; + size_t start_row; + size_t end_row; + bool all_rows_joined_finish; + + ProbeProcessInfo(UInt64 max_block_size_) + : max_block_size(max_block_size_) + , all_rows_joined_finish(true){}; + + void resetBlock(Block && block_); + void updateStartRow(); +}; } // namespace DB diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 7887a260a9f..90fa363fe4d 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -234,6 +234,18 @@ DB::ColumnsWithTypeAndName ExecutorTest::executeStreams(const std::shared_ptr & request, size_t concurrency) +{ + DAGContext dag_context(*request, "executor_test", concurrency); + context.context.setExecutorTest(); + context.context.setMockStorage(context.mockStorage()); + context.context.setDAGContext(&dag_context); + // Currently, don't care about regions information in tests. + Blocks blocks; + queryExecute(context.context, /*internal=*/true)->execute([&blocks](const Block & block) { blocks.push_back(block); }).verify(); + return blocks; +} + DB::ColumnsWithTypeAndName ExecutorTest::executeRawQuery(const String & query, size_t concurrency) { DAGProperties properties; diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index 641f8e4646c..acb239a96df 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -98,6 +98,10 @@ class ExecutorTest : public ::testing::Test const std::shared_ptr & request, size_t concurrency = 1); + Blocks getExecuteStreamsReturnBlocks( + const std::shared_ptr & request, + size_t concurrency = 1); + private: void executeExecutor( const std::shared_ptr & request, From 91a14e520740b58a7712389190c092a0142a8767 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Thu, 15 Dec 2022 16:22:52 +0800 Subject: [PATCH 20/42] feat: auto reload ssl cert (#6346) close pingcap/tiflash#5503 --- dbms/src/Common/Config/ConfigObject.h | 30 ++ dbms/src/Common/Config/ConfigReloader.cpp | 61 ++-- dbms/src/Common/Config/ConfigReloader.h | 25 +- dbms/src/Common/FileChangesTracker.h | 58 ++++ dbms/src/Common/TiFlashSecurity.h | 261 ++++++++++++++---- .../Common/tests/gtest_config_reloader.cpp | 116 ++++++++ .../Common/tests/gtest_tiflash_security.cpp | 174 ++++++++++-- dbms/src/Debug/MockComputeServerManager.cpp | 6 +- dbms/src/Flash/FlashService.cpp | 5 +- dbms/src/Flash/FlashService.h | 3 +- dbms/src/Interpreters/Context.cpp | 17 ++ dbms/src/Interpreters/Context.h | 8 + dbms/src/Server/CMakeLists.txt | 5 +- dbms/src/Server/CertificateReloader.cpp | 70 +++++ dbms/src/Server/CertificateReloader.h | 45 +++ dbms/src/Server/FlashGrpcServerHolder.cpp | 59 +++- dbms/src/Server/FlashGrpcServerHolder.h | 4 +- dbms/src/Server/IServer.h | 2 - dbms/src/Server/MetricsPrometheus.cpp | 38 +-- dbms/src/Server/MetricsPrometheus.h | 2 +- dbms/src/Server/RaftConfigParser.cpp | 2 +- dbms/src/Server/RaftConfigParser.h | 5 +- dbms/src/Server/Server.cpp | 84 +++--- dbms/src/Server/Server.h | 4 - dbms/src/Server/UserConfigParser.cpp | 6 +- dbms/src/Storages/Transaction/TMTContext.cpp | 6 + dbms/src/Storages/Transaction/TMTContext.h | 2 + libs/libdaemon/CMakeLists.txt | 2 +- 28 files changed, 881 insertions(+), 219 deletions(-) create mode 100644 dbms/src/Common/Config/ConfigObject.h create mode 100644 dbms/src/Common/FileChangesTracker.h create mode 100644 dbms/src/Common/tests/gtest_config_reloader.cpp create mode 100644 dbms/src/Server/CertificateReloader.cpp create mode 100644 dbms/src/Server/CertificateReloader.h diff --git a/dbms/src/Common/Config/ConfigObject.h b/dbms/src/Common/Config/ConfigObject.h new file mode 100644 index 00000000000..3e461cc96ea --- /dev/null +++ b/dbms/src/Common/Config/ConfigObject.h @@ -0,0 +1,30 @@ +// 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. + +#pragma once + +namespace DB +{ +/** Base class for Configuration component that keep track of a set of files + * + * If the set of files are changed, the ConfigReloader will call reloadIfNewer + * to reload all the config. + */ +class ConfigObject +{ +public: + virtual bool fileUpdated() = 0; + virtual ~ConfigObject() = default; +}; +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Common/Config/ConfigReloader.cpp b/dbms/src/Common/Config/ConfigReloader.cpp index 49308968676..baf716b44cd 100644 --- a/dbms/src/Common/Config/ConfigReloader.cpp +++ b/dbms/src/Common/Config/ConfigReloader.cpp @@ -12,16 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "ConfigReloader.h" - +#include +#include #include #include #include #include #include -#include "ConfigProcessor.h" - namespace DB { @@ -42,7 +40,6 @@ void ConfigReloader::start() thread = std::thread(&ConfigReloader::run, this); } - ConfigReloader::~ConfigReloader() { try @@ -78,14 +75,23 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error) std::lock_guard lock(reload_mutex); FilesChangesTracker new_files = getNewFileList(); - if (force || new_files.isDifferOrNewerThan(files)) + bool config_object_updated = false; + for (const auto & conf : config_objects) + { + if (conf->fileUpdated()) + { + config_object_updated = true; + break; + } + } + + if (force || new_files.isDifferOrNewerThan(files) || config_object_updated) { ConfigProcessor config_processor(path); ConfigProcessor::LoadedConfig loaded_config; try { - LOG_DEBUG(log, "Loading config `{}`", path); - + LOG_DEBUG(log, "Loading config from `{}`", path); loaded_config = config_processor.loadConfig(); } catch (...) @@ -93,9 +99,10 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error) if (throw_on_error) throw; - tryLogCurrentException(log, "Error loading config from `" + path + "'"); + tryLogCurrentException(log, fmt::format("Error loading config from `{}`", path)); return; } + config_processor.savePreprocessedConfig(loaded_config); /** We should remember last modification time if and only if config was sucessfully loaded @@ -114,44 +121,12 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error) { if (throw_on_error) throw; - tryLogCurrentException(log, "Error updating configuration from `" + path + "' config."); + tryLogCurrentException(log, fmt::format("Error updating configuration from `{}` config.", path)); } } } -struct ConfigReloader::FileWithTimestamp -{ - std::string path; - time_t modification_time; - - FileWithTimestamp(const std::string & path_, time_t modification_time_) - : path(path_) - , modification_time(modification_time_) - {} - - bool operator<(const FileWithTimestamp & rhs) const { return path < rhs.path; } - - static bool isTheSame(const FileWithTimestamp & lhs, const FileWithTimestamp & rhs) - { - return (lhs.modification_time == rhs.modification_time) && (lhs.path == rhs.path); - } -}; - - -void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path) -{ - if (!path.empty() && Poco::File(path).exists()) - { - files.emplace(path, Poco::File(path).getLastModified().epochTime()); - } -} - -bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs) const -{ - return (files.size() != rhs.files.size()) || !std::equal(files.begin(), files.end(), rhs.files.begin(), FileWithTimestamp::isTheSame); -} - -ConfigReloader::FilesChangesTracker ConfigReloader::getNewFileList() const +FilesChangesTracker ConfigReloader::getNewFileList() const { FilesChangesTracker file_list; diff --git a/dbms/src/Common/Config/ConfigReloader.h b/dbms/src/Common/Config/ConfigReloader.h index a8a725b2aea..7a5a8aabfb6 100644 --- a/dbms/src/Common/Config/ConfigReloader.h +++ b/dbms/src/Common/Config/ConfigReloader.h @@ -14,18 +14,20 @@ #pragma once +#include +#include +#include #include #include #include #include +#include #include #include #include #include -#include "ConfigProcessor.h" - namespace Poco { @@ -58,6 +60,9 @@ class ConfigReloader /// Reload immediately. For SYSTEM RELOAD CONFIG query. void reload() { reloadIfNewer(/* force */ true, /* throw_on_error */ true); } + /// Add ConfigObject to keep tracker of files that will be changed in config + void addConfigObject(std::shared_ptr object) { config_objects.push_back(object); } + protected: void reloadIfNewer(bool force, bool throw_on_error); Updater & getUpdater() { return updater; } @@ -65,16 +70,6 @@ class ConfigReloader private: void run(); - struct FileWithTimestamp; - - struct FilesChangesTracker - { - std::set files; - - void addIfExists(const std::string & path); - bool isDifferOrNewerThan(const FilesChangesTracker & rhs) const; - }; - FilesChangesTracker getNewFileList() const; protected: @@ -87,9 +82,13 @@ class ConfigReloader std::string path; FilesChangesTracker files; - Updater updater; + // ConfigObject contains a set of files that are used in config file. + // We can check if the files in ConfigObject are updated. + // If they are updated, the reloadIfNewer will be called. + std::vector> config_objects; + std::atomic quit{false}; std::thread thread; diff --git a/dbms/src/Common/FileChangesTracker.h b/dbms/src/Common/FileChangesTracker.h new file mode 100644 index 00000000000..10af6c6f6be --- /dev/null +++ b/dbms/src/Common/FileChangesTracker.h @@ -0,0 +1,58 @@ +// 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. + +#pragma once + +#include + +#include +#include + +namespace DB +{ +struct FileWithTimestamp +{ + std::string path; + time_t modification_time; + + FileWithTimestamp(const std::string & path_, time_t modification_time_) + : path(path_) + , modification_time(modification_time_) + {} + + bool operator<(const FileWithTimestamp & rhs) const { return path < rhs.path; } + + static bool isTheSame(const FileWithTimestamp & lhs, const FileWithTimestamp & rhs) + { + return (lhs.modification_time == rhs.modification_time) && (lhs.path == rhs.path); + } +}; + +struct FilesChangesTracker +{ + std::set files; + + void addIfExists(const std::string & path) + { + if (!path.empty() && Poco::File(path).exists()) + { + files.emplace(path, Poco::File(path).getLastModified().epochTime()); + } + } + bool isDifferOrNewerThan(const FilesChangesTracker & rhs) const + { + return (files.size() != rhs.files.size()) || !std::equal(files.begin(), files.end(), rhs.files.begin(), FileWithTimestamp::isTheSame); + } +}; +} // namespace DB diff --git a/dbms/src/Common/TiFlashSecurity.h b/dbms/src/Common/TiFlashSecurity.h index e3e34cadbc5..3a0f3fc4f96 100644 --- a/dbms/src/Common/TiFlashSecurity.h +++ b/dbms/src/Common/TiFlashSecurity.h @@ -13,15 +13,21 @@ // limitations under the License. #pragma once +#include +#include #include #include #include +#include #include #include #include #include +#include +#include #include +#include namespace DB { @@ -30,63 +36,75 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; } -struct TiFlashSecurityConfig +class TiFlashSecurityConfig : public ConfigObject { - String ca_path; - String cert_path; - String key_path; +public: + TiFlashSecurityConfig() = default; - bool redact_info_log = false; + explicit TiFlashSecurityConfig(const LoggerPtr & log_) + : log(log_) + { + } - std::set allowed_common_names; + void init(Poco::Util::AbstractConfiguration & config) + { + if (!inited) + { + update(config); + inited = true; + } + } - bool inited = false; - bool has_tls_config = false; - grpc::SslCredentialsOptions options; + void setLog(const LoggerPtr & log_) + { + std::unique_lock lock(mu); + log = log_; + } -public: - TiFlashSecurityConfig() = default; + bool hasTlsConfig() + { + std::unique_lock lock(mu); + return has_tls_config; + } + + bool redactInfoLog() + { + std::unique_lock lock(mu); + return redact_info_log; + } - TiFlashSecurityConfig(Poco::Util::LayeredConfiguration & config, const LoggerPtr & log) + std::tuple getPaths() { + std::unique_lock lock(mu); + return {ca_path, cert_path, key_path}; + } + + std::pair getCertAndKeyPath() + { + std::unique_lock lock(mu); + return {cert_path, key_path}; + } + + std::set allowedCommonNames() + { + std::unique_lock lock(mu); + return allowed_common_names; + } + + // return value indicate whether the Ssl certificate path is changed. + bool update(Poco::Util::AbstractConfiguration & config) + { + std::unique_lock lock(mu); if (config.has("security")) { - bool miss_ca_path = true; - bool miss_cert_path = true; - bool miss_key_path = true; - if (config.has("security.ca_path")) - { - ca_path = config.getString("security.ca_path"); - miss_ca_path = false; - } - if (config.has("security.cert_path")) - { - cert_path = config.getString("security.cert_path"); - miss_cert_path = false; - } - if (config.has("security.key_path")) - { - key_path = config.getString("security.key_path"); - miss_key_path = false; - } - if (miss_ca_path && miss_cert_path && miss_key_path) + if (inited && !has_security) { - LOG_INFO(log, "No security config is set."); - } - else if (miss_ca_path || miss_cert_path || miss_key_path) - { - throw Exception("ca_path, cert_path, key_path must be set at the same time.", ErrorCodes::INVALID_CONFIG_PARAMETER); - } - else - { - has_tls_config = true; - LOG_INFO( - log, - "security config is set: ca path is {} cert path is {} key path is {}", - ca_path, - cert_path, - key_path); + LOG_WARNING(log, "Can't add security config online"); + return false; } + has_security = true; + + bool cert_file_updated = updateCertPath(config); if (config.has("security.cert_allowed_cn") && has_tls_config) { @@ -94,13 +112,25 @@ struct TiFlashSecurityConfig parseAllowedCN(verify_cns); } - // redact_info_log = config.getBool("security.redact-info-log", false); // Mostly options name are combined with "_", keep this style if (config.has("security.redact_info_log")) { redact_info_log = config.getBool("security.redact_info_log"); } + return cert_file_updated; } + else + { + if (inited && has_security) + { + LOG_WARNING(log, "Can't remove security config online"); + } + else + { + LOG_INFO(log, "security config is not set"); + } + } + return false; } void parseAllowedCN(String verify_cns) @@ -121,9 +151,9 @@ struct TiFlashSecurityConfig } } - bool checkGrpcContext(const grpc::ServerContext * grpc_context) const { + std::unique_lock lock(mu); if (allowed_common_names.empty() || grpc_context == nullptr) { return true; @@ -137,19 +167,40 @@ struct TiFlashSecurityConfig return false; } - grpc::SslCredentialsOptions readAndCacheSecurityInfo() + bool checkCommonName(const Poco::Crypto::X509Certificate & cert) { - if (inited) + std::unique_lock lock(mu); + if (allowed_common_names.empty()) { - return options; + return true; } + return allowed_common_names.count(cert.commonName()) > 0; + } + + grpc::SslCredentialsOptions readAndCacheSslCredentialOptions() + { + std::unique_lock lock(mu); + if (ssl_cerd_options_cached) + return options; options.pem_root_certs = readFile(ca_path); options.pem_cert_chain = readFile(cert_path); options.pem_private_key = readFile(key_path); - inited = true; + ssl_cerd_options_cached = true; + LOG_INFO(log, "read new SslCredentialOptions: ca_path: {}, cert_path: {}, key_path: {}", ca_path, cert_path, key_path); return options; } + bool fileUpdated() override + { + FilesChangesTracker new_files; + for (const auto & file : cert_files.files) + new_files.addIfExists(file.path); + + bool updated = new_files.isDifferOrNewerThan(cert_files); + LOG_DEBUG(log, "Ssl certificate files updated: {}", updated); + return updated; + } + private: static String readFile(const String & filename) { @@ -167,6 +218,112 @@ struct TiFlashSecurityConfig } return result; } + + bool updateCertPath(Poco::Util::AbstractConfiguration & config) + { + bool miss_ca_path = true; + bool miss_cert_path = true; + bool miss_key_path = true; + String new_ca_path; + String new_cert_path; + String new_key_path; + bool updated = false; + if (config.has("security.ca_path")) + { + new_ca_path = config.getString("security.ca_path"); + miss_ca_path = false; + } + if (config.has("security.cert_path")) + { + new_cert_path = config.getString("security.cert_path"); + miss_cert_path = false; + } + if (config.has("security.key_path")) + { + new_key_path = config.getString("security.key_path"); + miss_key_path = false; + } + if (miss_ca_path && miss_cert_path && miss_key_path) + { + if (inited && has_tls_config) + { + LOG_WARNING(log, "Can't remove tls config online"); + } + else + { + LOG_INFO(log, "No TLS config is set."); + } + } + else if (miss_ca_path || miss_cert_path || miss_key_path) + { + throw Exception("ca_path, cert_path, key_path must be set at the same time.", ErrorCodes::INVALID_CONFIG_PARAMETER); + } + else + { + if (inited && !has_tls_config) + { + LOG_WARNING(log, "Can't add TLS config online"); + return false; + } + else + { + has_tls_config = true; + if (new_ca_path != ca_path || new_cert_path != cert_path || new_key_path != key_path) + { + ca_path = new_ca_path; + cert_path = new_cert_path; + key_path = new_key_path; + cert_files.files.clear(); + cert_files.addIfExists(ca_path); + cert_files.addIfExists(cert_path); + cert_files.addIfExists(key_path); + updated = true; + ssl_cerd_options_cached = false; + LOG_INFO( + log, + "Ssl certificate config path is updated: ca path is {} cert path is {} key path is {}", + ca_path, + cert_path, + key_path); + } + else + { + // whether the cert file content is updated + updated = fileUpdated(); + // update cert files + if (updated) + { + FilesChangesTracker new_files; + for (const auto & file : cert_files.files) + { + new_files.addIfExists(file.path); + } + cert_files = std::move(new_files); + ssl_cerd_options_cached = false; + } + } + } + } + return updated; + } + +private: + mutable std::mutex mu; + String ca_path; + String cert_path; + String key_path; + + FilesChangesTracker cert_files; + bool redact_info_log = false; + std::set allowed_common_names; + bool has_tls_config = false; + bool has_security = false; + bool inited = false; + + bool ssl_cerd_options_cached = false; + grpc::SslCredentialsOptions options; + + LoggerPtr log; }; } // namespace DB diff --git a/dbms/src/Common/tests/gtest_config_reloader.cpp b/dbms/src/Common/tests/gtest_config_reloader.cpp new file mode 100644 index 00000000000..d0f77883610 --- /dev/null +++ b/dbms/src/Common/tests/gtest_config_reloader.cpp @@ -0,0 +1,116 @@ +// 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 + +namespace DB +{ +namespace tests +{ +class ConfigReloaderTest : public ext::Singleton +{ +}; + +TEST(ConfigReloaderTest, Basic) +{ + auto path = DB::tests::TiFlashTestEnv::getTemporaryPath("ConfigReloaderTest_Basic"); + Poco::File file(path); + if (file.exists()) + file.remove(); + file.createFile(); + Poco::FileOutputStream stream(path); + stream << R"( +[security] +ca_path="security/ca.pem" +cert_path="security/cert.pem" +key_path="security/key.pem" +cert_allowed_cn="tidb" + )"; + + int call_times = 0; + auto main_config_reloader = std::make_unique( + path, + [&](ConfigurationPtr config [[maybe_unused]]) { + call_times++; + }, + /* already_loaded = */ false); + + auto other_config_reloader = std::make_unique( + path, + [&](ConfigurationPtr config [[maybe_unused]]) { + call_times++; + }, + /* already_loaded = */ false, + "otherCfgLoader"); + + main_config_reloader->start(); + other_config_reloader->start(); + std::this_thread::sleep_for(std::chrono::seconds(3)); + ASSERT_EQ(call_times, 2); +} + +class TestConfigObject : public ConfigObject +{ +public: + bool fileUpdated() override + { + if (!inited) + { + inited = true; + return true; + } + return false; + } + +private: + bool inited = false; +}; + +TEST(ConfigReloaderTest, WithConfigObject) +{ + auto path = DB::tests::TiFlashTestEnv::getTemporaryPath("ConfigReloaderTest_WithConfigObject"); + Poco::File file(path); + if (file.exists()) + file.remove(); + file.createFile(); + Poco::FileOutputStream stream(path); + stream << R"( + [profiles] +[profiles.default] +max_memory_usage = 0 + )"; + + int call_times = 0; + auto main_config_reloader = std::make_unique( + path, + [&](ConfigurationPtr config [[maybe_unused]]) { + call_times++; + }, + /* already_loaded = */ false); + main_config_reloader->addConfigObject(std::make_shared()); + main_config_reloader->start(); + std::this_thread::sleep_for(std::chrono::seconds(3)); + ASSERT_EQ(call_times, 2); +} +} // namespace tests +} // namespace DB diff --git a/dbms/src/Common/tests/gtest_tiflash_security.cpp b/dbms/src/Common/tests/gtest_tiflash_security.cpp index 81f67d6363a..e11e3a51ea3 100644 --- a/dbms/src/Common/tests/gtest_tiflash_security.cpp +++ b/dbms/src/Common/tests/gtest_tiflash_security.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include @@ -21,34 +22,171 @@ namespace DB { namespace tests { -class TestTiFlashSecurity : public ext::Singleton +class TiFlashSecurityTest : public ext::Singleton { }; -TEST(TestTiFlashSecurity, Config) +TEST(TiFlashSecurityTest, Config) { - TiFlashSecurityConfig config; - config.parseAllowedCN(String("[abc,efg]")); - ASSERT_EQ((int)config.allowed_common_names.count("abc"), 1); - ASSERT_EQ((int)config.allowed_common_names.count("efg"), 1); + TiFlashSecurityConfig tiflash_config; + const auto log = Logger::get(); + tiflash_config.setLog(log); - config.allowed_common_names.clear(); + tiflash_config.parseAllowedCN(String("[abc,efg]")); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("abc"), 1); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("efg"), 1); - config.parseAllowedCN(String("[\"abc\",\"efg\"]")); - ASSERT_EQ((int)config.allowed_common_names.count("abc"), 1); - ASSERT_EQ((int)config.allowed_common_names.count("efg"), 1); + tiflash_config.allowedCommonNames().clear(); - config.allowed_common_names.clear(); + tiflash_config.parseAllowedCN(String(R"(["abc","efg"])")); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("abc"), 1); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("efg"), 1); - config.parseAllowedCN(String("[ abc , efg ]")); - ASSERT_EQ((int)config.allowed_common_names.count("abc"), 1); - ASSERT_EQ((int)config.allowed_common_names.count("efg"), 1); + tiflash_config.allowedCommonNames().clear(); - config.allowed_common_names.clear(); + tiflash_config.parseAllowedCN(String("[ abc , efg ]")); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("abc"), 1); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("efg"), 1); - config.parseAllowedCN(String("[ \"abc\", \"efg\" ]")); - ASSERT_EQ((int)config.allowed_common_names.count("abc"), 1); - ASSERT_EQ((int)config.allowed_common_names.count("efg"), 1); + tiflash_config.allowedCommonNames().clear(); + + tiflash_config.parseAllowedCN(String(R"([ "abc", "efg" ])")); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("abc"), 1); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("efg"), 1); + + String test = + R"( +[security] +ca_path="security/ca.pem" +cert_path="security/cert.pem" +key_path="security/key.pem" +cert_allowed_cn="tidb" + )"; + auto new_config = loadConfigFromString(test); + tiflash_config.update(*new_config); + ASSERT_EQ((int)tiflash_config.allowedCommonNames().count("tidb"), 1); + + test = + R"( +[security] +cert_allowed_cn="tidb" + )"; + new_config = loadConfigFromString(test); + auto new_tiflash_config = TiFlashSecurityConfig(log); + new_tiflash_config.init(*new_config); + ASSERT_EQ((int)new_tiflash_config.allowedCommonNames().count("tidb"), 0); +} + +TEST(TiFlashSecurityTest, Update) +{ + String test = + R"( +[security] +cert_allowed_cn="tidb" + )"; + + auto config = loadConfigFromString(test); + const auto log = Logger::get(); + + TiFlashSecurityConfig tiflash_config(log); // no TLS config is set + tiflash_config.init(*config); + test = + R"( +[security] +ca_path="security/ca.pem" +cert_path="security/cert.pem" +key_path="security/key.pem" +cert_allowed_cn="tidb" + )"; + config = loadConfigFromString(test); + ASSERT_FALSE(tiflash_config.update(*config)); // Can't add tls config online + ASSERT_FALSE(tiflash_config.hasTlsConfig()); + config = loadConfigFromString(test); + TiFlashSecurityConfig tiflash_config_1(log); + tiflash_config_1.init(*config); + test = + R"( + )"; + config = loadConfigFromString(test); + ASSERT_FALSE(tiflash_config_1.update(*config)); // Can't remove security config online + ASSERT_TRUE(tiflash_config_1.hasTlsConfig()); + + test = + R"( +[security] +cert_allowed_cn="tidb" + )"; + config = loadConfigFromString(test); + ASSERT_FALSE(tiflash_config_1.update(*config)); // Can't remove tls config online + ASSERT_TRUE(tiflash_config_1.hasTlsConfig()); + + test = + R"( +[security] +ca_path="security/ca_new.pem" +cert_path="security/cert_new.pem" +key_path="security/key_new.pem" +cert_allowed_cn="tidb" + )"; + config = loadConfigFromString(test); + ASSERT_TRUE(tiflash_config_1.update(*config)); + auto paths = tiflash_config_1.getPaths(); + ASSERT_EQ(std::get<0>(paths), "security/ca_new.pem"); + ASSERT_EQ(std::get<1>(paths), "security/cert_new.pem"); + ASSERT_EQ(std::get<2>(paths), "security/key_new.pem"); + ASSERT_EQ((int)tiflash_config_1.allowedCommonNames().count("tidb"), 1); + ASSERT_EQ((int)tiflash_config_1.allowedCommonNames().count("tiflash"), 0); + + // add cert allowed cn + test = + R"( +[security] +ca_path="security/ca_new.pem" +cert_path="security/cert_new.pem" +key_path="security/key_new.pem" +cert_allowed_cn="[tidb, tiflash]" + )"; + config = loadConfigFromString(test); + ASSERT_FALSE(tiflash_config_1.update(*config)); + paths = tiflash_config_1.getPaths(); + ASSERT_EQ(std::get<0>(paths), "security/ca_new.pem"); + ASSERT_EQ(std::get<1>(paths), "security/cert_new.pem"); + ASSERT_EQ(std::get<2>(paths), "security/key_new.pem"); + ASSERT_EQ((int)tiflash_config_1.allowedCommonNames().count("tidb"), 1); + ASSERT_EQ((int)tiflash_config_1.allowedCommonNames().count("tiflash"), 1); + + // Without security config + test = + R"( + )"; + config = loadConfigFromString(test); + TiFlashSecurityConfig tiflash_config_2(log); + tiflash_config_2.init(*config); + + test = + R"( +[security] +cert_allowed_cn="[tidb, tiflash]" + )"; + + config = loadConfigFromString(test); + ASSERT_FALSE(tiflash_config_2.update(*config)); //Can't add security config online + ASSERT_TRUE(tiflash_config_2.allowedCommonNames().empty()); + ASSERT_FALSE(tiflash_config_2.hasTlsConfig()); + + test = + R"( +[security] +ca_path="security/ca_new.pem" +cert_path="security/cert_new.pem" +key_path="security/key_new.pem" +cert_allowed_cn="[tidb, tiflash]" +redact_info_log=false + )"; + config = loadConfigFromString(test); + ASSERT_FALSE(tiflash_config_2.update(*config)); // Can't add security config online + ASSERT_TRUE(tiflash_config_2.allowedCommonNames().empty()); + ASSERT_FALSE(tiflash_config_2.hasTlsConfig()); } } // namespace tests } // namespace DB diff --git a/dbms/src/Debug/MockComputeServerManager.cpp b/dbms/src/Debug/MockComputeServerManager.cpp index 118316bf1cf..41bf549fc61 100644 --- a/dbms/src/Debug/MockComputeServerManager.cpp +++ b/dbms/src/Debug/MockComputeServerManager.cpp @@ -45,11 +45,10 @@ void MockComputeServerManager::startServers(const LoggerPtr & log_ptr, Context & global_context.setMPPTest(); for (const auto & server_config : server_config_map) { - TiFlashSecurityConfig security_config; TiFlashRaftConfig raft_config; raft_config.flash_server_addr = server_config.second.addr; Poco::AutoPtr config = new Poco::Util::LayeredConfiguration; - addServer(server_config.first, std::make_unique(global_context, *config, security_config, raft_config, log_ptr)); + addServer(server_config.first, std::make_unique(global_context, *config, raft_config, log_ptr)); } prepareMockMPPServerInfo(); @@ -59,13 +58,12 @@ void MockComputeServerManager::startServers(const LoggerPtr & log_ptr, int start { for (const auto & server_config : server_config_map) { - TiFlashSecurityConfig security_config; TiFlashRaftConfig raft_config; raft_config.flash_server_addr = server_config.second.addr; Poco::AutoPtr config = new Poco::Util::LayeredConfiguration; auto & context = TiFlashTestEnv::getGlobalContext(start_idx++); context.setMPPTest(); - addServer(server_config.first, std::make_unique(context, *config, security_config, raft_config, log_ptr)); + addServer(server_config.first, std::make_unique(context, *config, raft_config, log_ptr)); } prepareMockMPPServerInfo(); diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index c1e647bc160..abd921a1119 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -63,9 +63,8 @@ constexpr char tls_err_msg[] = "common name check is failed"; FlashService::FlashService() = default; -void FlashService::init(const TiFlashSecurityConfig & security_config_, Context & context_) +void FlashService::init(Context & context_) { - security_config = &security_config_; context = &context_; log = &Poco::Logger::get("FlashService"); manual_compact_manager = std::make_unique( @@ -417,7 +416,7 @@ grpc::Status FlashService::checkGrpcContext(const grpc::ServerContext * grpc_con // For coprocessor/mpp test, we don't care about security config. if likely (!context->isMPPTest() && !context->isCopTest()) { - if (!security_config->checkGrpcContext(grpc_context)) + if (!context->getSecurityConfig()->checkGrpcContext(grpc_context)) { return grpc::Status(grpc::PERMISSION_DENIED, tls_err_msg); } diff --git a/dbms/src/Flash/FlashService.h b/dbms/src/Flash/FlashService.h index 7d0e8d05298..99e1ff66c53 100644 --- a/dbms/src/Flash/FlashService.h +++ b/dbms/src/Flash/FlashService.h @@ -50,7 +50,7 @@ class FlashService : public tikvpb::Tikv::Service { public: FlashService(); - void init(const TiFlashSecurityConfig & security_config_, Context & context_); + void init(Context & context_); ~FlashService() override; @@ -89,7 +89,6 @@ class FlashService : public tikvpb::Tikv::Service std::tuple createDBContext(const grpc::ServerContext * grpc_context) const; grpc::Status checkGrpcContext(const grpc::ServerContext * grpc_context) const; - const TiFlashSecurityConfig * security_config = nullptr; Context * context = nullptr; Poco::Logger * log = nullptr; bool is_async = false; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index a8fecced955..93ddd06b931 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -48,6 +48,7 @@ #include #include #include +#include #include #include #include @@ -158,6 +159,8 @@ struct ContextShared IORateLimiter io_rate_limiter; PageStorageRunMode storage_run_mode = PageStorageRunMode::ONLY_V3; DM::GlobalStoragePoolPtr global_storage_pool; + TiFlashSecurityConfigPtr security_config; + /// Named sessions. The user could specify session identifier to reuse settings and temporary tables in subsequent requests. class SessionKeyHash @@ -571,6 +574,20 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } +void Context::setSecurityConfig(Poco::Util::AbstractConfiguration & config, const LoggerPtr & log) +{ + LOG_INFO(log, "Setting secuirty config."); + auto lock = getLock(); + shared->security_config = std::make_shared(log); + shared->security_config->init(config); +} + +TiFlashSecurityConfigPtr Context::getSecurityConfig() +{ + auto lock = getLock(); + return shared->security_config; +} + void Context::reloadDeltaTreeConfig(const Poco::Util::AbstractConfiguration & config) { auto default_profile_name = config.getString("default_profile", "default"); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 6488e7e5bce..65fb2b91908 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -100,6 +101,8 @@ class ReadLimiter; using ReadLimiterPtr = std::shared_ptr; using MockMPPServerInfo = DB::tests::MockMPPServerInfo; using MockStorage = DB::tests::MockStorage; +class TiFlashSecurityConfig; +using TiFlashSecurityConfigPtr = std::shared_ptr; enum class PageStorageRunMode : UInt8; namespace DM @@ -217,6 +220,11 @@ class Context void setUsersConfig(const ConfigurationPtr & config); ConfigurationPtr getUsersConfig(); + /// Security configuration settings. + void setSecurityConfig(Poco::Util::AbstractConfiguration & config, const LoggerPtr & log); + + TiFlashSecurityConfigPtr getSecurityConfig(); + /// Must be called before getClientInfo. void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key); /// Compute and set actual user settings, client_info.current_user should be set diff --git a/dbms/src/Server/CMakeLists.txt b/dbms/src/Server/CMakeLists.txt index 7d25716660d..3020ee8cf55 100644 --- a/dbms/src/Server/CMakeLists.txt +++ b/dbms/src/Server/CMakeLists.txt @@ -32,10 +32,11 @@ add_library(server_for_test ) target_link_libraries (server_for_test PUBLIC clickhouse_common_io clickhouse_storages_system) +target_include_directories(server_for_test PRIVATE ${TiFlash_SOURCE_DIR}/contrib/grpc/) add_library (clickhouse-server-lib HTTPHandler.cpp - FlashGrpcServerHolder.cpp + CertificateReloader.cpp MetricsTransmitter.cpp MetricsPrometheus.cpp NotFoundHandler.cpp @@ -51,7 +52,7 @@ add_library (clickhouse-server-lib target_link_libraries(clickhouse-server-lib PRIVATE ${TIFLASH_PROXY_LIBRARY}) -target_link_libraries (clickhouse-server-lib PUBLIC clickhouse_common_io daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions) +target_link_libraries (clickhouse-server-lib PUBLIC server_for_test clickhouse_common_io daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions) target_include_directories (clickhouse-server-lib PUBLIC ${TiFlash_SOURCE_DIR}/libs/libdaemon/include) add_library (clickhouse-client-lib Client.cpp) diff --git a/dbms/src/Server/CertificateReloader.cpp b/dbms/src/Server/CertificateReloader.cpp new file mode 100644 index 00000000000..1ebb14f21a9 --- /dev/null +++ b/dbms/src/Server/CertificateReloader.cpp @@ -0,0 +1,70 @@ +// 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 + +#if Poco_NetSSL_FOUND + +#include + +namespace DB +{ +namespace +{ +/// Call set process for certificate. +int callSetCertificate(SSL * ssl, void * arg) +{ + return CertificateReloader::instance().setCertificate(ssl, arg); +} +} // namespace + +/// This is callback for OpenSSL. It will be called on every connection to obtain a certificate and private key. +int CertificateReloader::setCertificate(SSL * ssl, void * arg) +{ + RUNTIME_ASSERT(ssl); + auto * context = static_cast(arg); + auto security_config = context->getSecurityConfig(); + auto [cert_path, key_path] = security_config->getCertAndKeyPath(); + LOG_DEBUG(log, "setCertificate callback called, cert_path: {}, key_path: {}", cert_path, key_path); + Poco::Crypto::X509Certificate cert(cert_path); + Poco::Crypto::EVPPKey key("", key_path); + int err = SSL_use_certificate(ssl, const_cast(cert.certificate())); + if (err != 1) + { + LOG_ERROR(log, "Unusable ssl certificate {}", Poco::Net::Utility::getLastError()); + } + + err = SSL_use_PrivateKey(ssl, const_cast(static_cast(key))); + if (err != 1) + { + LOG_ERROR(log, "Unusable ssl key {}", Poco::Net::Utility::getLastError()); + } + + err = SSL_check_private_key(ssl); + if (err != 1) + { + LOG_ERROR(log, "Unusable ssl certificate key-pair {}", Poco::Net::Utility::getLastError()); + return -1; + } + + return 1; +} + +void CertificateReloader::initSSLCallback(Poco::Net::Context::Ptr context, Context * global_context) +{ + SSL_CTX_set_cert_cb(context->sslContext(), callSetCertificate, reinterpret_cast(global_context)); +} +} // namespace DB + +#endif diff --git a/dbms/src/Server/CertificateReloader.h b/dbms/src/Server/CertificateReloader.h new file mode 100644 index 00000000000..3c6f3d2714e --- /dev/null +++ b/dbms/src/Server/CertificateReloader.h @@ -0,0 +1,45 @@ +// 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. + +#pragma once + +#include + +#if Poco_NetSSL_FOUND + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +class CertificateReloader : public ext::Singleton +{ +public: + static void initSSLCallback(Poco::Net::Context::Ptr context, Context * global_context); + /// A callback for OpenSSL + int setCertificate(SSL * ssl, void * arg); + +private: + LoggerPtr log = Logger::get("CertificateReloader"); +}; +} // namespace DB + +#endif diff --git a/dbms/src/Server/FlashGrpcServerHolder.cpp b/dbms/src/Server/FlashGrpcServerHolder.cpp index 05dcf3f6816..1aeff0a49c6 100644 --- a/dbms/src/Server/FlashGrpcServerHolder.cpp +++ b/dbms/src/Server/FlashGrpcServerHolder.cpp @@ -14,6 +14,11 @@ #include #include +// In order to include grpc::SecureServerCredentials which used in +// sslServerCredentialsWithFetcher() +// We implement sslServerCredentialsWithFetcher() to set config fetcher +// to auto reload sslServerCredentials +#include "../../contrib/grpc/src/cpp/server/secure_server_credentials.h" namespace DB { @@ -79,20 +84,54 @@ void handleRpcs(grpc::ServerCompletionQueue * curcq, const LoggerPtr & log) } } // namespace -FlashGrpcServerHolder::FlashGrpcServerHolder(Context & context, Poco::Util::LayeredConfiguration & config_, TiFlashSecurityConfig & security_config, const TiFlashRaftConfig & raft_config, const LoggerPtr & log_) +static grpc_ssl_certificate_config_reload_status +sslServerCertificateConfigCallback( + void * arg, + grpc_ssl_server_certificate_config ** config) +{ + if (config == nullptr) + { + return GRPC_SSL_CERTIFICATE_CONFIG_RELOAD_FAIL; + } + auto * context = static_cast(arg); + auto options = context->getSecurityConfig()->readAndCacheSslCredentialOptions(); + grpc_ssl_pem_key_cert_pair pem_key_cert_pair = {options.pem_private_key.c_str(), options.pem_cert_chain.c_str()}; + *config = grpc_ssl_server_certificate_config_create(options.pem_root_certs.c_str(), + &pem_key_cert_pair, + 1); + return GRPC_SSL_CERTIFICATE_CONFIG_RELOAD_NEW; +} + +grpc_server_credentials * grpcSslServerCredentialsCreateWithFetcher( + grpc_ssl_client_certificate_request_type client_certificate_request, + Context * context) +{ + grpc_ssl_server_credentials_options * options = grpc_ssl_server_credentials_create_options_using_config_fetcher( + client_certificate_request, + sslServerCertificateConfigCallback, + reinterpret_cast(context)); + return grpc_ssl_server_credentials_create_with_options(options); +} + +std::shared_ptr sslServerCredentialsWithFetcher(Context & context) +{ + grpc_server_credentials * c_creds = grpcSslServerCredentialsCreateWithFetcher( + GRPC_SSL_REQUEST_AND_REQUIRE_CLIENT_CERTIFICATE_AND_VERIFY, + &context); + return std::shared_ptr( + new grpc::SecureServerCredentials(c_creds)); +} + +FlashGrpcServerHolder::FlashGrpcServerHolder(Context & context, Poco::Util::LayeredConfiguration & config_, const TiFlashRaftConfig & raft_config, const LoggerPtr & log_) : log(log_) , is_shutdown(std::make_shared>(false)) { background_task.begin(); grpc::ServerBuilder builder; - if (security_config.has_tls_config) + + if (!context.isTest() && context.getSecurityConfig()->hasTlsConfig()) { - grpc::SslServerCredentialsOptions server_cred(GRPC_SSL_REQUEST_AND_REQUIRE_CLIENT_CERTIFICATE_AND_VERIFY); - auto options = security_config.readAndCacheSecurityInfo(); - server_cred.pem_root_certs = options.pem_root_certs; - server_cred.pem_key_cert_pairs.push_back( - grpc::SslServerCredentialsOptions::PemKeyCertPair{options.pem_private_key, options.pem_cert_chain}); - builder.AddListeningPort(raft_config.flash_server_addr, grpc::SslServerCredentials(server_cred)); + builder.AddListeningPort(raft_config.flash_server_addr, sslServerCredentialsWithFetcher(context)); } else { @@ -105,7 +144,7 @@ FlashGrpcServerHolder::FlashGrpcServerHolder(Context & context, Poco::Util::Laye flash_service = std::make_unique(); else flash_service = std::make_unique(); - flash_service->init(security_config, context); + flash_service->init(context); diagnostics_service = std::make_unique(context, config_); builder.SetOption(grpc::MakeChannelArgumentOption(GRPC_ARG_HTTP2_MIN_RECV_PING_INTERVAL_WITHOUT_DATA_MS, 5 * 1000)); @@ -219,4 +258,4 @@ std::unique_ptr & FlashGrpcServerHolder::flashService() { return flash_service; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Server/FlashGrpcServerHolder.h b/dbms/src/Server/FlashGrpcServerHolder.h index f9a807ffb7f..09eea8cab97 100644 --- a/dbms/src/Server/FlashGrpcServerHolder.h +++ b/dbms/src/Server/FlashGrpcServerHolder.h @@ -22,6 +22,7 @@ #include #include #include +#include namespace DB @@ -35,7 +36,6 @@ class FlashGrpcServerHolder FlashGrpcServerHolder( Context & context, Poco::Util::LayeredConfiguration & config_, - TiFlashSecurityConfig & security_config, const TiFlashRaftConfig & raft_config, const LoggerPtr & log_); ~FlashGrpcServerHolder(); @@ -59,4 +59,4 @@ class FlashGrpcServerHolder CollectProcInfoBackgroundTask background_task; }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Server/IServer.h b/dbms/src/Server/IServer.h index 72934f60a80..761daf016ea 100644 --- a/dbms/src/Server/IServer.h +++ b/dbms/src/Server/IServer.h @@ -35,8 +35,6 @@ class IServer /// Returns global application's context. virtual Context & context() const = 0; - virtual const TiFlashSecurityConfig & securityConfig() const = 0; - /// Returns true if shutdown signaled. virtual bool isCancelled() const = 0; diff --git a/dbms/src/Server/MetricsPrometheus.cpp b/dbms/src/Server/MetricsPrometheus.cpp index c195722e2fd..abe99333e42 100644 --- a/dbms/src/Server/MetricsPrometheus.cpp +++ b/dbms/src/Server/MetricsPrometheus.cpp @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "MetricsPrometheus.h" - #include #include #include @@ -28,6 +26,8 @@ #include #include #include +#include +#include #include #include #include @@ -101,27 +101,27 @@ class MetricHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory }; std::shared_ptr getHTTPServer( - const TiFlashSecurityConfig & security_config, + Context & global_context, const std::weak_ptr & collectable, const String & metrics_port) { + auto security_config = global_context.getSecurityConfig(); + auto [ca_path, cert_path, key_path] = security_config->getPaths(); Poco::Net::Context::Ptr context = new Poco::Net::Context( Poco::Net::Context::TLSV1_2_SERVER_USE, - security_config.key_path, - security_config.cert_path, - security_config.ca_path, + key_path, + cert_path, + ca_path, Poco::Net::Context::VerificationMode::VERIFY_STRICT); - std::function check_common_name = [&](const Poco::Crypto::X509Certificate & cert) { - if (security_config.allowed_common_names.empty()) - { - return true; - } - return security_config.allowed_common_names.count(cert.commonName()) > 0; + auto check_common_name = [&](const Poco::Crypto::X509Certificate & cert) { + return global_context.getSecurityConfig()->checkCommonName(cert); }; context->setAdhocVerification(check_common_name); - +#if Poco_NetSSL_FOUND + CertificateReloader::initSSLCallback(context, &global_context); +#endif Poco::Net::SecureServerSocket socket(context); Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; @@ -138,8 +138,7 @@ constexpr Int64 INIT_DELAY = 5; MetricsPrometheus::MetricsPrometheus( Context & context, - const AsynchronousMetrics & async_metrics_, - const TiFlashSecurityConfig & security_config) + const AsynchronousMetrics & async_metrics_) : timer("Prometheus") , async_metrics(async_metrics_) , log(&Poco::Logger::get("Prometheus")) @@ -202,9 +201,10 @@ MetricsPrometheus::MetricsPrometheus( if (conf.hasOption(status_metrics_port) || !conf.hasOption(status_metrics_addr)) { auto metrics_port = conf.getString(status_metrics_port, DB::toString(DEFAULT_METRICS_PORT)); - if (security_config.has_tls_config) + + if (context.getSecurityConfig()->hasTlsConfig()) { - server = getHTTPServer(security_config, tiflash_metrics.registry, metrics_port); + server = getHTTPServer(context, tiflash_metrics.registry, metrics_port); server->start(); LOG_INFO(log, "Enable prometheus secure pull mode; Metrics Port = {}", metrics_port); } @@ -234,13 +234,13 @@ MetricsPrometheus::~MetricsPrometheus() void MetricsPrometheus::run() { auto & tiflash_metrics = TiFlashMetrics::instance(); - for (ProfileEvents::Event event = 0; event < ProfileEvents::end(); event++) + for (ProfileEvents::Event event = 0; event < ProfileEvents::end(); ++event) { const auto value = ProfileEvents::counters[event].load(std::memory_order_relaxed); tiflash_metrics.registered_profile_events[event]->Set(value); } - for (CurrentMetrics::Metric metric = 0; metric < CurrentMetrics::end(); metric++) + for (CurrentMetrics::Metric metric = 0; metric < CurrentMetrics::end(); ++metric) { const auto value = CurrentMetrics::values[metric].load(std::memory_order_relaxed); tiflash_metrics.registered_current_metrics[metric]->Set(value); diff --git a/dbms/src/Server/MetricsPrometheus.h b/dbms/src/Server/MetricsPrometheus.h index 0c68270881e..4ef84f6f142 100644 --- a/dbms/src/Server/MetricsPrometheus.h +++ b/dbms/src/Server/MetricsPrometheus.h @@ -35,7 +35,7 @@ class Context; class MetricsPrometheus { public: - MetricsPrometheus(Context & context, const AsynchronousMetrics & async_metrics_, const TiFlashSecurityConfig & config); + MetricsPrometheus(Context & context, const AsynchronousMetrics & async_metrics_); ~MetricsPrometheus(); private: diff --git a/dbms/src/Server/RaftConfigParser.cpp b/dbms/src/Server/RaftConfigParser.cpp index 38472163d2d..b8cfe1ab3bf 100644 --- a/dbms/src/Server/RaftConfigParser.cpp +++ b/dbms/src/Server/RaftConfigParser.cpp @@ -30,7 +30,7 @@ extern const int INVALID_CONFIG_PARAMETER; } // namespace ErrorCodes /// Load raft related configs. -TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::LayeredConfiguration & config, const LoggerPtr & log) +TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::AbstractConfiguration & config, const LoggerPtr & log) { TiFlashRaftConfig res; res.flash_server_addr = config.getString("flash.service_addr", "0.0.0.0:3930"); diff --git a/dbms/src/Server/RaftConfigParser.h b/dbms/src/Server/RaftConfigParser.h index 659a42d76fb..604a2476c44 100644 --- a/dbms/src/Server/RaftConfigParser.h +++ b/dbms/src/Server/RaftConfigParser.h @@ -13,6 +13,7 @@ // limitations under the License. #pragma once +#include #include #include @@ -23,7 +24,7 @@ namespace Poco class Logger; namespace Util { -class LayeredConfiguration; +class AbstractConfiguration; } } // namespace Poco @@ -51,7 +52,7 @@ struct TiFlashRaftConfig public: TiFlashRaftConfig() = default; - static TiFlashRaftConfig parseSettings(Poco::Util::LayeredConfiguration & config, const LoggerPtr & log); + static TiFlashRaftConfig parseSettings(Poco::Util::AbstractConfiguration & config, const LoggerPtr & log); }; } // namespace DB diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index c3194c840df..e4893518e23 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -55,6 +55,7 @@ #include #include #include +#include #include #include #include @@ -287,14 +288,16 @@ struct TiFlashProxyConfig const std::string TiFlashProxyConfig::config_prefix = "flash.proxy"; -pingcap::ClusterConfig getClusterConfig(const TiFlashSecurityConfig & security_config, const TiFlashRaftConfig & raft_config) +pingcap::ClusterConfig getClusterConfig(TiFlashSecurityConfigPtr security_config, const TiFlashRaftConfig & raft_config, const LoggerPtr & log) { pingcap::ClusterConfig config; config.tiflash_engine_key = raft_config.engine_key; config.tiflash_engine_value = raft_config.engine_value; - config.ca_path = security_config.ca_path; - config.cert_path = security_config.cert_path; - config.key_path = security_config.key_path; + auto [ca_path, cert_path, key_path] = security_config->getPaths(); + config.ca_path = ca_path; + config.cert_path = cert_path; + config.key_path = key_path; + LOG_INFO(log, "update cluster config, ca_path: {}, cert_path: {}, key_path: {}", ca_path, cert_path, key_path); return config; } @@ -528,7 +531,7 @@ class Server::TcpHttpServersHolder , server_pool(1, server.config().getUInt("max_connections", 1024)) { auto & config = server.config(); - auto & security_config = server.security_config; + auto security_config = server.global_context->getSecurityConfig(); Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; // NOLINT @@ -604,27 +607,24 @@ class Server::TcpHttpServersHolder if (config.has("https_port")) { #if Poco_NetSSL_FOUND - if (!security_config.has_tls_config) + if (!security_config->hasTlsConfig()) { LOG_ERROR(log, "https_port is set but tls config is not set"); } + auto [ca_path, cert_path, key_path] = security_config->getPaths(); Poco::Net::Context::Ptr context = new Poco::Net::Context(Poco::Net::Context::TLSV1_2_SERVER_USE, - security_config.key_path, - security_config.cert_path, - security_config.ca_path, + key_path, + cert_path, + ca_path, Poco::Net::Context::VerificationMode::VERIFY_STRICT); - std::function check_common_name - = [&](const Poco::Crypto::X509Certificate & cert) { - if (security_config.allowed_common_names.empty()) - { - return true; - } - return security_config.allowed_common_names.count(cert.commonName()) > 0; - }; + auto check_common_name = [&](const Poco::Crypto::X509Certificate & cert) { + return server.global_context->getSecurityConfig()->checkCommonName(cert); + }; context->setAdhocVerification(check_common_name); std::call_once(ssl_init_once, SSLInit); Poco::Net::SecureServerSocket socket(context); + CertificateReloader::initSSLCallback(context, server.global_context.get()); auto address = socket_bind_listen(socket, listen_host, config.getInt("https_port"), /* secure = */ true); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); @@ -640,7 +640,7 @@ class Server::TcpHttpServersHolder else { /// HTTP - if (security_config.has_tls_config) + if (security_config->hasTlsConfig()) { throw Exception("tls config is set but https_port is not set ", ErrorCodes::INVALID_CONFIG_PARAMETER); } @@ -658,7 +658,7 @@ class Server::TcpHttpServersHolder /// TCP if (config.has("tcp_port")) { - if (security_config.has_tls_config) + if (security_config->hasTlsConfig()) { LOG_ERROR(log, "tls config is set but tcp_port_secure is not set."); } @@ -671,19 +671,21 @@ class Server::TcpHttpServersHolder LOG_INFO(log, "Listening tcp: {}", address.toString()); } - else if (security_config.has_tls_config) + else if (security_config->hasTlsConfig()) { LOG_INFO(log, "tcp_port is closed because tls config is set"); } /// TCP with SSL - if (config.has("tcp_port_secure") && !security_config.has_tls_config) + if (config.has("tcp_port_secure") && !security_config->hasTlsConfig()) { #if Poco_NetSSL_FOUND + auto [ca_path, cert_path, key_path] = security_config->getPaths(); Poco::Net::Context::Ptr context = new Poco::Net::Context(Poco::Net::Context::TLSV1_2_SERVER_USE, - security_config.key_path, - security_config.cert_path, - security_config.ca_path); + key_path, + cert_path, + ca_path); + CertificateReloader::initSSLCallback(context, server.global_context.get()); Poco::Net::SecureServerSocket socket(context); auto address = socket_bind_listen(socket, listen_host, config.getInt("tcp_port_secure"), /* secure = */ true); socket.setReceiveTimeout(settings.receive_timeout); @@ -699,9 +701,9 @@ class Server::TcpHttpServersHolder ErrorCodes::SUPPORT_IS_DISABLED}; #endif } - else if (security_config.has_tls_config) + else if (security_config->hasTlsConfig()) { - LOG_INFO(log, "tcp_port is closed because tls config is set"); + LOG_INFO(log, "tcp_port_secure is closed because tls config is set"); } /// At least one of TCP and HTTP servers must be created. @@ -789,11 +791,6 @@ class Server::TcpHttpServersHolder LOG_DEBUG(log, debug_msg); } - const std::vector> & getServers() const - { - return servers; - } - private: Server & server; const LoggerPtr & log; @@ -976,9 +973,8 @@ int Server::main(const std::vector & /*args*/) global_context->setPath(path); /// ===== Paths related configuration initialized end ===== /// - - security_config = TiFlashSecurityConfig(config(), log); - Redact::setRedactLog(security_config.redact_info_log); + global_context->setSecurityConfig(config(), log); + Redact::setRedactLog(global_context->getSecurityConfig()->redactInfoLog()); // Create directories for 'path' and for default database, if not exist. for (const String & candidate_path : all_normal_path) @@ -1103,11 +1099,24 @@ int Server::main(const std::vector & /*args*/) auto main_config_reloader = std::make_unique( config_path, [&](ConfigurationPtr config) { + LOG_DEBUG(log, "run main config reloader"); buildLoggers(*config); global_context->setMacros(std::make_unique(*config, "macros")); global_context->getTMTContext().reloadConfig(*config); global_context->getIORateLimiter().updateConfig(*config); global_context->reloadDeltaTreeConfig(*config); + + { + // update TiFlashSecurity and related config in client for ssl certificate reload. + bool updated = global_context->getSecurityConfig()->update(*config); // Whether the cert path or file is updated. + if (updated) + { + auto raft_config = TiFlashRaftConfig::parseSettings(*config, log); + auto cluster_config = getClusterConfig(global_context->getSecurityConfig(), raft_config, log); + global_context->getTMTContext().updateSecurityConfig(std::move(raft_config), std::move(cluster_config)); + LOG_DEBUG(log, "TMTContext updated security config"); + } + } }, /* already_loaded = */ true); @@ -1160,7 +1169,7 @@ int Server::main(const std::vector & /*args*/) { /// create TMTContext - auto cluster_config = getClusterConfig(security_config, raft_config); + auto cluster_config = getClusterConfig(global_context->getSecurityConfig(), raft_config, log); global_context->createTMTContext(raft_config, std::move(cluster_config)); global_context->getTMTContext().reloadConfig(config()); } @@ -1266,11 +1275,12 @@ int Server::main(const std::vector & /*args*/) } /// Then, startup grpc server to serve raft and/or flash services. - FlashGrpcServerHolder flash_grpc_server_holder(this->context(), this->config(), this->security_config, raft_config, log); + FlashGrpcServerHolder flash_grpc_server_holder(this->context(), this->config(), raft_config, log); { TcpHttpServersHolder tcpHttpServersHolder(*this, settings, log); + main_config_reloader->addConfigObject(global_context->getSecurityConfig()); main_config_reloader->start(); users_config_reloader->start(); @@ -1308,7 +1318,7 @@ int Server::main(const std::vector & /*args*/) metrics_transmitters.emplace_back(std::make_unique(*global_context, async_metrics, graphite_key)); } - auto metrics_prometheus = std::make_unique(*global_context, async_metrics, security_config); + auto metrics_prometheus = std::make_unique(*global_context, async_metrics); SessionCleaner session_cleaner(*global_context); diff --git a/dbms/src/Server/Server.h b/dbms/src/Server/Server.h index 9f083d298cf..c4567a09e81 100644 --- a/dbms/src/Server/Server.h +++ b/dbms/src/Server/Server.h @@ -40,8 +40,6 @@ class Server : public BaseDaemon return BaseDaemon::config(); } - const TiFlashSecurityConfig & securityConfig() const override { return security_config; }; - Poco::Logger & logger() const override { return BaseDaemon::logger(); @@ -69,8 +67,6 @@ class Server : public BaseDaemon private: std::unique_ptr global_context; - TiFlashSecurityConfig security_config; - ServerInfo server_info; class TcpHttpServersHolder; diff --git a/dbms/src/Server/UserConfigParser.cpp b/dbms/src/Server/UserConfigParser.cpp index 5eeeac7b415..9ae8fc457b2 100644 --- a/dbms/src/Server/UserConfigParser.cpp +++ b/dbms/src/Server/UserConfigParser.cpp @@ -71,9 +71,9 @@ ConfigReloaderPtr parseSettings( return std::make_unique( users_config_path, // - /*updater=*/[&global_context](ConfigurationPtr cfg) { global_context->setUsersConfig(cfg); }, - /*already_loaded=*/false, - /*name=*/"UserCfgReloader"); + /* updater = */ [&global_context](ConfigurationPtr cfg) { global_context->setUsersConfig(cfg); }, + /* already_loaded = */ false, + /* name = */ "UserCfgReloader"); } } // namespace UserConfig diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index 07927956c40..bde42468495 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -88,6 +88,12 @@ TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config , wait_region_ready_timeout_sec(DEFAULT_WAIT_REGION_READY_TIMEOUT_SEC) {} +void TMTContext::updateSecurityConfig(const TiFlashRaftConfig & raft_config, const pingcap::ClusterConfig & cluster_config) +{ + if (!raft_config.pd_addrs.empty()) + cluster->update(raft_config.pd_addrs, cluster_config); +} + void TMTContext::restore(PathPool & path_pool, const TiFlashRaftProxyHelper * proxy_helper) { kvstore->restore(path_pool, proxy_helper); diff --git a/dbms/src/Storages/Transaction/TMTContext.h b/dbms/src/Storages/Transaction/TMTContext.h index 535da4329fc..74ca368db45 100644 --- a/dbms/src/Storages/Transaction/TMTContext.h +++ b/dbms/src/Storages/Transaction/TMTContext.h @@ -87,6 +87,8 @@ class TMTContext : private boost::noncopyable SchemaSyncerPtr getSchemaSyncer() const; + void updateSecurityConfig(const TiFlashRaftConfig & raft_config, const pingcap::ClusterConfig & cluster_config); + pingcap::pd::ClientPtr getPDClient() const; pingcap::kv::Cluster * getKVCluster() { return cluster.get(); } diff --git a/libs/libdaemon/CMakeLists.txt b/libs/libdaemon/CMakeLists.txt index 5fc8cf946b2..8c93bf8659d 100644 --- a/libs/libdaemon/CMakeLists.txt +++ b/libs/libdaemon/CMakeLists.txt @@ -35,7 +35,7 @@ endif () target_include_directories (daemon PUBLIC include) target_include_directories (daemon PRIVATE ${TiFlash_SOURCE_DIR}/libs/libpocoext/include) -target_link_libraries (daemon clickhouse_common_io clickhouse_common_config ${EXECINFO_LIBRARY}) +target_link_libraries (daemon clickhouse_common_io clickhouse_common_config grpc grpc++ ${EXECINFO_LIBRARY}) if (ENABLE_TESTS) add_subdirectory (src/tests EXCLUDE_FROM_ALL) endif () From 38e70c327437efe21a907c9536b425529e8ac158 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 15 Dec 2022 19:30:52 +0800 Subject: [PATCH 21/42] Minor refine of `TableLock` and `MPPTask::cancel` (#6484) --- dbms/src/Flash/Coprocessor/DAGContext.h | 5 +++++ dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp | 11 ++--------- dbms/src/Flash/Executor/DataStreamExecutor.cpp | 4 ++-- dbms/src/Flash/Executor/DataStreamExecutor.h | 2 +- dbms/src/Flash/Executor/QueryExecutor.h | 2 +- dbms/src/Flash/Mpp/MPPTask.cpp | 8 +++----- dbms/src/Flash/Mpp/MPPTask.h | 2 +- 7 files changed, 15 insertions(+), 19 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index bcfdefa7cf5..6f0b46c45b1 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -328,6 +328,8 @@ class DAGContext void setProcessListEntry(std::shared_ptr entry) { process_list_entry = entry; } std::shared_ptr getProcessListEntry() const { return process_list_entry; } + void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } + const tipb::DAGRequest * dag_request; /// Some existing code inherited from Clickhouse assume that each query must have a valid query string and query ast, /// dummy_query_string and dummy_ast is used for that @@ -378,6 +380,9 @@ class DAGContext private: std::shared_ptr process_list_entry; + /// Holding the table lock to make sure that the table wouldn't be dropped during the lifetime of this query, even if there are no local regions. + /// TableLockHolders need to be released after the BlockInputStream is destroyed to prevent data read exceptions. + TableLockHolders table_locks; /// profile_streams_map is a map that maps from executor_id to profile BlockInputStreams. std::unordered_map profile_streams_map; /// executor_id_to_join_id_map is a map that maps executor id to all the join executor id of itself and all its children. diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 1c4bce252c1..e3b298b613c 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -360,15 +360,8 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) remote_read_streams_start_index = 1; } - /// We don't want the table to be dropped during the lifetime of this query, - /// and sometimes if there is no local region, we will use the RemoteBlockInputStream - /// or even the null_stream to hold the lock. - pipeline.transform([&](auto & stream) { - // todo do not need to hold all locks in each stream, if the stream is reading from table a - // it only needs to hold the lock of table a - for (const auto & lock : drop_locks) - stream->addTableLock(lock); - }); + for (const auto & lock : drop_locks) + dagContext().addTableLock(lock); /// Set the limits and quota for reading data, the speed and time of the query. setQuotaAndLimitsOnTableScan(context, pipeline); diff --git a/dbms/src/Flash/Executor/DataStreamExecutor.cpp b/dbms/src/Flash/Executor/DataStreamExecutor.cpp index 8aceabaf210..eee368b31b8 100644 --- a/dbms/src/Flash/Executor/DataStreamExecutor.cpp +++ b/dbms/src/Flash/Executor/DataStreamExecutor.cpp @@ -42,10 +42,10 @@ ExecutionResult DataStreamExecutor::execute(ResultHandler result_handler) } } -void DataStreamExecutor::cancel(bool is_kill) +void DataStreamExecutor::cancel() { if (auto * p_stream = dynamic_cast(data_stream.get()); p_stream) - p_stream->cancel(is_kill); + p_stream->cancel(/*kill=*/false); } String DataStreamExecutor::dump() const diff --git a/dbms/src/Flash/Executor/DataStreamExecutor.h b/dbms/src/Flash/Executor/DataStreamExecutor.h index 4358cbfde5a..f9963c848fb 100644 --- a/dbms/src/Flash/Executor/DataStreamExecutor.h +++ b/dbms/src/Flash/Executor/DataStreamExecutor.h @@ -32,7 +32,7 @@ class DataStreamExecutor : public QueryExecutor String dump() const override; - void cancel(bool is_kill) override; + void cancel() override; int estimateNewThreadCount() override; diff --git a/dbms/src/Flash/Executor/QueryExecutor.h b/dbms/src/Flash/Executor/QueryExecutor.h index 57fff9a0330..b35dc0d8116 100644 --- a/dbms/src/Flash/Executor/QueryExecutor.h +++ b/dbms/src/Flash/Executor/QueryExecutor.h @@ -59,7 +59,7 @@ class QueryExecutor ExecutionResult execute(); ExecutionResult execute(ResultHandler::Handler handler); - virtual void cancel(bool is_kill) = 0; + virtual void cancel() = 0; virtual String dump() const = 0; diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 990da4c7ad8..45a180ff421 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -94,14 +94,12 @@ void MPPTask::abortReceivers() receiver_set->cancel(); } -void MPPTask::abortDataStreams(AbortType abort_type) +void MPPTask::abortQueryExecutor() { - /// When abort type is ONERROR, it means MPPTask already known it meet error, so let the remaining task stop silently to avoid too many useless error message - bool is_kill = abort_type == AbortType::ONCANCELLATION; if (auto query_executor = query_executor_holder.tryGet(); query_executor) { assert(query_executor.value()); - (*query_executor)->cancel(is_kill); + (*query_executor)->cancel(); } } @@ -496,7 +494,7 @@ void MPPTask::abort(const String & message, AbortType abort_type) /// the original error err_string = message; abortTunnels(message, false); - abortDataStreams(abort_type); + abortQueryExecutor(); abortReceivers(); scheduleThisTask(ScheduleState::FAILED); /// runImpl is running, leave remaining work to runImpl diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index f219fe67bfe..b2c203bf9f5 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -92,7 +92,7 @@ class MPPTask : public std::enable_shared_from_this void abortTunnels(const String & message, bool wait_sender_finish); void abortReceivers(); - void abortDataStreams(AbortType abort_type); + void abortQueryExecutor(); void finishWrite(); From d174e12e26ad150206d92796937d837dfd7aa140 Mon Sep 17 00:00:00 2001 From: hehechen Date: Fri, 16 Dec 2022 10:12:52 +0800 Subject: [PATCH 22/42] TiFlash supports stale read (#6459) close pingcap/tiflash#4483 --- contrib/kvproto | 2 +- dbms/src/Common/TiFlashMetrics.h | 1 + dbms/src/Debug/DAGProperties.h | 5 + dbms/src/Debug/MockComputeServerManager.cpp | 7 +- dbms/src/Debug/MockComputeServerManager.h | 3 +- dbms/src/Debug/MockExecutor/AstToPB.h | 9 + .../MockExecutor/ExchangeReceiverBinder.cpp | 3 + .../MockExecutor/ExchangeSenderBinder.cpp | 3 + dbms/src/Debug/dbgFuncMisc.cpp | 4 +- dbms/src/Debug/dbgQueryCompiler.h | 5 +- dbms/src/Debug/dbgQueryExecutor.cpp | 15 ++ dbms/src/Flash/Coprocessor/DAGContext.h | 2 +- dbms/src/Flash/FlashService.cpp | 4 +- dbms/src/Flash/Mpp/MPPTask.cpp | 8 +- dbms/src/Flash/Mpp/MPPTaskId.cpp | 76 ++++++- dbms/src/Flash/Mpp/MPPTaskId.h | 58 +++++- dbms/src/Flash/Mpp/MPPTaskManager.cpp | 50 ++--- dbms/src/Flash/Mpp/MPPTaskManager.h | 16 +- dbms/src/Flash/Mpp/MPPTaskStatistics.cpp | 5 +- dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 85 ++++---- dbms/src/Flash/Mpp/MinTSOScheduler.h | 20 +- dbms/src/Flash/Mpp/getMPPTaskTracingLog.cpp | 3 +- dbms/src/Flash/Mpp/getMPPTaskTracingLog.h | 3 - dbms/src/Flash/tests/bench_exchange.cpp | 4 +- dbms/src/Flash/tests/bench_window.cpp | 2 +- dbms/src/Flash/tests/gtest_compute_server.cpp | 51 +++-- dbms/src/Storages/StorageDisaggregated.cpp | 12 +- dbms/src/Storages/StorageDisaggregated.h | 7 +- dbms/src/Storages/Transaction/LearnerRead.cpp | 33 ++- dbms/src/Storages/Transaction/RegionTable.cpp | 11 + dbms/src/Storages/Transaction/RegionTable.h | 10 +- dbms/src/TestUtils/MPPTaskTestUtils.cpp | 15 +- dbms/src/TestUtils/MPPTaskTestUtils.h | 7 +- dbms/src/TestUtils/mockExecutor.cpp | 2 +- libs/libcommon/include/common/logger_useful.h | 6 + libs/libdaemon/src/BaseDaemon.cpp | 1 - metrics/grafana/tiflash_summary.json | 197 +++++++++++++----- 37 files changed, 520 insertions(+), 225 deletions(-) diff --git a/contrib/kvproto b/contrib/kvproto index 26e28e6a281..9ccc6beaf0a 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 26e28e6a281abb927f91ef992eb8f93b39698ffa +Subproject commit 9ccc6beaf0aa9b0a4adad43b497348898ba653cf diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 8accc59b155..e6c86911422 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -105,6 +105,7 @@ namespace DB M(tiflash_schema_apply_duration_seconds, "Bucketed histogram of ddl apply duration", Histogram, \ F(type_ddl_apply_duration, {{"req", "ddl_apply_duration"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_raft_read_index_count, "Total number of raft read index", Counter) \ + M(tiflash_stale_read_count, "Total number of stale read", Counter) \ M(tiflash_raft_read_index_duration_seconds, "Bucketed histogram of raft read index duration", Histogram, \ F(type_raft_read_index_duration, {{"type", "tmt_raft_read_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_raft_wait_index_duration_seconds, "Bucketed histogram of raft wait index duration", Histogram, \ diff --git a/dbms/src/Debug/DAGProperties.h b/dbms/src/Debug/DAGProperties.h index bcb4170c9ac..c4bfcc9b863 100644 --- a/dbms/src/Debug/DAGProperties.h +++ b/dbms/src/Debug/DAGProperties.h @@ -29,6 +29,11 @@ struct DAGProperties bool use_broadcast_join = false; Int32 mpp_partition_num = 1; Timestamp start_ts = DEFAULT_MAX_READ_TSO; + UInt64 query_ts = 0; + UInt64 server_id = 1; + UInt64 local_query_id = 1; + Int64 task_id = 1; + Int32 mpp_timeout = 10; }; } // namespace DB \ No newline at end of file diff --git a/dbms/src/Debug/MockComputeServerManager.cpp b/dbms/src/Debug/MockComputeServerManager.cpp index 41bf549fc61..839cd794b73 100644 --- a/dbms/src/Debug/MockComputeServerManager.cpp +++ b/dbms/src/Debug/MockComputeServerManager.cpp @@ -115,11 +115,14 @@ void MockComputeServerManager::addServer(size_t partition_id, std::unique_ptrset_start_ts(start_ts); + meta->set_query_ts(query_id.query_ts); + meta->set_local_query_id(query_id.local_query_id); + meta->set_server_id(query_id.server_id); + meta->set_start_ts(query_id.start_ts); mpp::CancelTaskResponse response; for (const auto & server : server_map) server.second->flashService()->cancelMPPTaskForTest(&req, &response); diff --git a/dbms/src/Debug/MockComputeServerManager.h b/dbms/src/Debug/MockComputeServerManager.h index 7e05224572b..6642388659f 100644 --- a/dbms/src/Debug/MockComputeServerManager.h +++ b/dbms/src/Debug/MockComputeServerManager.h @@ -22,7 +22,6 @@ namespace DB::tests { - /** Hold Mock Compute Server to manage the lifetime of them. * Maintains Mock Compute Server info. */ @@ -49,7 +48,7 @@ class MockComputeServerManager : public ext::Singleton void resetMockMPPServerInfo(size_t partition_num); - void cancelQuery(size_t start_ts); + void cancelQuery(const MPPQueryId & query_id); static String queryInfo(); diff --git a/dbms/src/Debug/MockExecutor/AstToPB.h b/dbms/src/Debug/MockExecutor/AstToPB.h index 518d04f89b9..c1560c90355 100644 --- a/dbms/src/Debug/MockExecutor/AstToPB.h +++ b/dbms/src/Debug/MockExecutor/AstToPB.h @@ -57,6 +57,9 @@ using MPPCtxPtr = std::shared_ptr; struct MPPInfo { Timestamp start_ts; + UInt64 query_ts; + UInt64 server_id; + UInt64 local_query_id; Int64 partition_id; Int64 task_id; const std::vector sender_target_task_ids; @@ -64,11 +67,17 @@ struct MPPInfo MPPInfo( Timestamp start_ts_, + UInt64 query_ts_, + UInt64 server_id_, + UInt64 local_query_id_, Int64 partition_id_, Int64 task_id_, const std::vector & sender_target_task_ids_, const std::unordered_map> & receiver_source_task_ids_map_) : start_ts(start_ts_) + , query_ts(query_ts_) + , server_id(server_id_) + , local_query_id(local_query_id_) , partition_id(partition_id_) , task_id(task_id_) , sender_target_task_ids(sender_target_task_ids_) diff --git a/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp index e7f0491b74f..706624856c0 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp +++ b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp @@ -42,6 +42,9 @@ bool ExchangeReceiverBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int3 { mpp::TaskMeta meta; meta.set_start_ts(mpp_info.start_ts); + meta.set_query_ts(mpp_info.query_ts); + meta.set_server_id(mpp_info.server_id); + meta.set_local_query_id(mpp_info.local_query_id); meta.set_task_id(it->second[i]); meta.set_partition_id(i); auto addr = context.isMPPTest() ? tests::MockComputeServerManager::instance().getServerConfigMap()[i].addr : Debug::LOCAL_HOST; diff --git a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp index 065d983cb60..aaba39868e1 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp +++ b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp @@ -41,6 +41,9 @@ bool ExchangeSenderBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_ { mpp::TaskMeta meta; meta.set_start_ts(mpp_info.start_ts); + meta.set_query_ts(mpp_info.query_ts); + meta.set_server_id(mpp_info.server_id); + meta.set_local_query_id(mpp_info.local_query_id); meta.set_task_id(task_id); meta.set_partition_id(i); auto addr = context.isMPPTest() ? tests::MockComputeServerManager::instance().getServerConfigMap()[i++].addr : Debug::LOCAL_HOST; diff --git a/dbms/src/Debug/dbgFuncMisc.cpp b/dbms/src/Debug/dbgFuncMisc.cpp index 50b6ebffb52..cbdf4629cb5 100644 --- a/dbms/src/Debug/dbgFuncMisc.cpp +++ b/dbms/src/Debug/dbgFuncMisc.cpp @@ -29,8 +29,8 @@ inline size_t getReadTSOForLog(const String & line) { std::regex rx(R"((0|[1-9][0-9]*))"); std::smatch m; - // Rely on that MPP task prefix "MPP" - auto pos = line.find("query:"); + // Rely on that MPP task prefix "MPP,task_id:42578433>" + auto pos = line.find(", start_ts:"); if (pos != std::string::npos && regex_search(line.cbegin() + pos, line.cend(), m, rx)) { return std::stoul(m[1]); diff --git a/dbms/src/Debug/dbgQueryCompiler.h b/dbms/src/Debug/dbgQueryCompiler.h index 2ab86df6dad..748b14d41e8 100644 --- a/dbms/src/Debug/dbgQueryCompiler.h +++ b/dbms/src/Debug/dbgQueryCompiler.h @@ -132,6 +132,9 @@ struct QueryFragment { MPPInfo mpp_info( properties.start_ts, + properties.query_ts, + properties.server_id, + properties.local_query_id, partition_id, task_ids[partition_id], sender_target_task_ids, @@ -141,7 +144,7 @@ struct QueryFragment } else { - MPPInfo mpp_info(properties.start_ts, /*partition_id*/ -1, /*task_id*/ -1, /*sender_target_task_ids*/ {}, /*receiver_source_task_ids_map*/ {}); + MPPInfo mpp_info(properties.start_ts, properties.query_ts, properties.server_id, properties.local_query_id, /*partition_id*/ -1, /*task_id*/ -1, /*sender_target_task_ids*/ {}, /*receiver_source_task_ids_map*/ {}); ret.push_back(toQueryTask(properties, mpp_info, context)); } return ret; diff --git a/dbms/src/Debug/dbgQueryExecutor.cpp b/dbms/src/Debug/dbgQueryExecutor.cpp index 38656747664..c5c0dbaf12c 100644 --- a/dbms/src/Debug/dbgQueryExecutor.cpp +++ b/dbms/src/Debug/dbgQueryExecutor.cpp @@ -42,6 +42,9 @@ BlockInputStreamPtr constructExchangeReceiverStream(Context & context, tipb::Exc mpp::TaskMeta root_tm; root_tm.set_start_ts(properties.start_ts); + root_tm.set_query_ts(properties.query_ts); + root_tm.set_local_query_id(properties.local_query_id); + root_tm.set_server_id(properties.server_id); root_tm.set_address(root_addr); root_tm.set_task_id(-1); root_tm.set_partition_id(-1); @@ -71,6 +74,9 @@ BlockInputStreamPtr prepareRootExchangeReceiver(Context & context, const DAGProp { mpp::TaskMeta tm; tm.set_start_ts(properties.start_ts); + tm.set_query_ts(properties.query_ts); + tm.set_local_query_id(properties.local_query_id); + tm.set_server_id(properties.server_id); tm.set_address(Debug::LOCAL_HOST); tm.set_task_id(root_task_id); tm.set_partition_id(-1); @@ -84,6 +90,9 @@ void prepareExchangeReceiverMetaWithMultipleContext(tipb::ExchangeReceiver & tip { mpp::TaskMeta tm; tm.set_start_ts(properties.start_ts); + tm.set_query_ts(properties.query_ts); + tm.set_local_query_id(properties.local_query_id); + tm.set_server_id(properties.server_id); tm.set_address(addr); tm.set_task_id(task_id); tm.set_partition_id(-1); @@ -109,6 +118,9 @@ void prepareDispatchTaskRequest(QueryTask & task, std::shared_ptrmutable_meta(); tm->set_start_ts(properties.start_ts); + tm->set_query_ts(properties.query_ts); + tm->set_local_query_id(properties.local_query_id); + tm->set_server_id(properties.server_id); tm->set_partition_id(task.partition_id); tm->set_address(addr); tm->set_task_id(task.task_id); @@ -128,6 +140,9 @@ void prepareDispatchTaskRequestWithMultipleContext(QueryTask & task, std::shared } auto * tm = req->mutable_meta(); tm->set_start_ts(properties.start_ts); + tm->set_query_ts(properties.query_ts); + tm->set_local_query_id(properties.local_query_id); + tm->set_server_id(properties.server_id); tm->set_partition_id(task.partition_id); tm->set_address(addr); tm->set_task_id(task.task_id); diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 6f0b46c45b1..ac17e3e2990 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -158,7 +158,7 @@ class DAGContext , flags(dag_request->flags()) , sql_mode(dag_request->sql_mode()) , mpp_task_meta(meta_) - , mpp_task_id(mpp_task_meta.start_ts(), mpp_task_meta.task_id()) + , mpp_task_id(mpp_task_meta) , max_recorded_error_count(getMaxErrorCount(*dag_request)) , warnings(max_recorded_error_count) , warning_count(0) diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index abd921a1119..57aa23a7e3c 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -365,7 +365,7 @@ grpc::Status FlashService::CancelMPPTask( auto & tmt_context = context->getTMTContext(); auto task_manager = tmt_context.getMPPTaskManager(); - task_manager->abortMPPQuery(request->meta().start_ts(), "Receive cancel request from TiDB", AbortType::ONCANCELLATION); + task_manager->abortMPPQuery(MPPQueryId(request->meta()), "Receive cancel request from TiDB", AbortType::ONCANCELLATION); return grpc::Status::OK; } @@ -407,7 +407,7 @@ ::grpc::Status FlashService::cancelMPPTaskForTest(const ::mpp::CancelTaskRequest } auto & tmt_context = context->getTMTContext(); auto task_manager = tmt_context.getMPPTaskManager(); - task_manager->abortMPPQuery(request->meta().start_ts(), "Receive cancel request from GTest", AbortType::ONCANCELLATION); + task_manager->abortMPPQuery(MPPQueryId(request->meta()), "Receive cancel request from GTest", AbortType::ONCANCELLATION); return grpc::Status::OK; } diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 45a180ff421..2b73629df7d 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -54,7 +54,7 @@ extern const char force_no_local_region_for_mpp_task[]; MPPTask::MPPTask(const mpp::TaskMeta & meta_, const ContextPtr & context_) : meta(meta_) - , id(meta.start_ts(), meta.task_id()) + , id(meta) , context(context_) , manager(context_->getTMTContext().getMPPTaskManager().get()) , schedule_entry(manager, id) @@ -137,7 +137,7 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) LOG_DEBUG(log, "begin to register the tunnel {}, is_local: {}, is_async: {}", tunnel->id(), is_local, is_async); if (status != INITIALIZING) throw Exception(fmt::format("The tunnel {} can not be registered, because the task is not in initializing state", tunnel->id())); - tunnel_set_local->registerTunnel(MPPTaskId{task_meta.start_ts(), task_meta.task_id()}, tunnel); + tunnel_set_local->registerTunnel(MPPTaskId(task_meta), tunnel); if (!dag_context->isRootMPPTask()) { FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_register_tunnel_for_non_root_mpp_task); @@ -202,7 +202,7 @@ std::pair MPPTask::getTunnel(const ::mpp::EstablishMPPConn return {nullptr, err_msg}; } - MPPTaskId receiver_id{request->receiver_meta().start_ts(), request->receiver_meta().task_id()}; + MPPTaskId receiver_id(request->receiver_meta()); RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); auto tunnel_ptr = tunnel_set->getTunnelByReceiverTaskId(receiver_id); if (tunnel_ptr == nullptr) @@ -450,7 +450,7 @@ void MPPTask::runImpl() void MPPTask::handleError(const String & error_msg) { auto updated_msg = fmt::format("From {}: {}", id.toString(), error_msg); - manager->abortMPPQuery(id.start_ts, updated_msg, AbortType::ONERROR); + manager->abortMPPQuery(id.query_id, updated_msg, AbortType::ONERROR); if (!registered) // if the task is not registered, need to cancel it explicitly abort(error_msg, AbortType::ONERROR); diff --git a/dbms/src/Flash/Mpp/MPPTaskId.cpp b/dbms/src/Flash/Mpp/MPPTaskId.cpp index bb025e6c24d..9f4a9ce3dfc 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskId.cpp @@ -13,19 +13,91 @@ // limitations under the License. #include +#include #include namespace DB { +bool isOldVersion(const MPPQueryId & mpp_query_id) +{ + return mpp_query_id.query_ts == 0 && mpp_query_id.local_query_id == 0 && mpp_query_id.server_id == 0; +} + + +bool MPPQueryId::operator<(const MPPQueryId & mpp_query_id) const +{ + // compare with MPP query generated by TiDB that version less than v6.6 + bool left_old_version = isOldVersion(*this); + bool right_old_version = isOldVersion(mpp_query_id); + if (unlikely(left_old_version && right_old_version)) + { + return start_ts < mpp_query_id.start_ts; + } + if (unlikely(left_old_version)) + { + return true; + } + if (unlikely(right_old_version)) + { + return false; + } + // compare with MPP query generated by TiDB that version after v6.6 + if (query_ts != mpp_query_id.query_ts) + { + return query_ts < mpp_query_id.query_ts; + } + if (server_id == mpp_query_id.server_id) + { + return local_query_id < mpp_query_id.local_query_id; + } + // now we can't compare reasonably, just choose one randomly by hash. + auto lhash = MPPQueryIdHash()(*this); + auto rhash = MPPQueryIdHash()(mpp_query_id); + if (lhash != rhash) + { + return lhash < rhash; + } + // hash values are same, just compare the rest fields. + if (local_query_id != mpp_query_id.local_query_id) + { + return local_query_id < mpp_query_id.local_query_id; + } + return server_id < mpp_query_id.server_id; +} +bool MPPQueryId::operator==(const MPPQueryId & rid) const +{ + return query_ts == rid.query_ts && local_query_id == rid.local_query_id && server_id == rid.server_id && start_ts == rid.start_ts; +} +bool MPPQueryId::operator!=(const MPPQueryId & rid) const +{ + return !(*this == rid); +} +bool MPPQueryId::operator<=(const MPPQueryId & rid) const +{ + return *this < rid || *this == rid; +} + +size_t MPPQueryIdHash::operator()(MPPQueryId const & mpp_query_id) const noexcept +{ + if (unlikely(isOldVersion(mpp_query_id))) + { + return std::hash()(mpp_query_id.start_ts); + } + return std::hash()(mpp_query_id.query_ts) ^ std::hash()(mpp_query_id.local_query_id) ^ std::hash()(mpp_query_id.server_id); +} + String MPPTaskId::toString() const { - return isUnknown() ? "MPP" : fmt::format("MPP", start_ts, task_id); + return isUnknown() ? "MPP" : fmt::format("MPP", query_id.toString(), task_id); } const MPPTaskId MPPTaskId::unknown_mpp_task_id = MPPTaskId{}; +constexpr UInt64 MAX_UINT64 = std::numeric_limits::max(); +const MPPQueryId MPPTaskId::Max_Query_Id = MPPQueryId(MAX_UINT64, MAX_UINT64, MAX_UINT64, MAX_UINT64); + bool operator==(const MPPTaskId & lid, const MPPTaskId & rid) { - return lid.start_ts == rid.start_ts && lid.task_id == rid.task_id; + return lid.query_id == rid.query_id && lid.task_id == rid.task_id; } } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTaskId.h b/dbms/src/Flash/Mpp/MPPTaskId.h index f1bee057206..eb81425c50c 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.h +++ b/dbms/src/Flash/Mpp/MPPTaskId.h @@ -15,28 +15,70 @@ #pragma once #include +#include +#include namespace DB { +// global unique MPP query id. +struct MPPQueryId +{ + UInt64 query_ts; + UInt64 local_query_id; + UInt64 server_id; + UInt64 start_ts; + MPPQueryId(UInt64 query_ts, UInt64 local_query_id, UInt64 server_id, UInt64 start_ts) + : query_ts(query_ts) + , local_query_id(local_query_id) + , server_id(server_id) + , start_ts(start_ts) + {} + explicit MPPQueryId(const mpp::TaskMeta & task_meta) + : query_ts(task_meta.query_ts()) + , local_query_id(task_meta.local_query_id()) + , server_id(task_meta.server_id()) + , start_ts(task_meta.start_ts()) + {} + bool operator<(const MPPQueryId & mpp_query_id) const; + bool operator==(const MPPQueryId & rid) const; + bool operator!=(const MPPQueryId & rid) const; + bool operator<=(const MPPQueryId & rid) const; + String toString() const + { + return fmt::format("", query_ts, local_query_id, server_id, start_ts); + } +}; + +struct MPPQueryIdHash +{ + size_t operator()(MPPQueryId const & mpp_query_id) const noexcept; +}; + // Identify a mpp task. struct MPPTaskId { MPPTaskId() - : start_ts(0) - , task_id(unknown_task_id){}; + : task_id(unknown_task_id) + , query_id({0, 0, 0, 0}){}; - MPPTaskId(UInt64 start_ts_, Int64 task_id_) - : start_ts(start_ts_) - , task_id(task_id_){}; + MPPTaskId(UInt64 start_ts, Int64 task_id_, UInt64 server_id, UInt64 query_ts, UInt64 local_query_id) + : task_id(task_id_) + , query_id(query_ts, local_query_id, server_id, start_ts) + {} + + explicit MPPTaskId(const mpp::TaskMeta & task_meta) + : task_id(task_meta.task_id()) + , query_id(task_meta) + {} - UInt64 start_ts; Int64 task_id; + MPPQueryId query_id; bool isUnknown() const { return task_id == unknown_task_id; } String toString() const; - static const MPPTaskId unknown_mpp_task_id; + static const MPPQueryId Max_Query_Id; private: static constexpr Int64 unknown_task_id = -1; @@ -53,7 +95,7 @@ class hash public: size_t operator()(const DB::MPPTaskId & id) const { - return hash()(id.start_ts) ^ hash()(id.task_id); + return DB::MPPQueryIdHash()(id.query_id) ^ hash()(id.task_id); } }; } // namespace std \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/MPPTaskManager.cpp b/dbms/src/Flash/Mpp/MPPTaskManager.cpp index bc54c5db8fc..41b6e35b727 100644 --- a/dbms/src/Flash/Mpp/MPPTaskManager.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskManager.cpp @@ -35,7 +35,7 @@ MPPTaskManager::MPPTaskManager(MPPTaskSchedulerPtr scheduler_) , log(Logger::get()) {} -MPPQueryTaskSetPtr MPPTaskManager::addMPPQueryTaskSet(UInt64 query_id) +MPPQueryTaskSetPtr MPPTaskManager::addMPPQueryTaskSet(const MPPQueryId & query_id) { auto ptr = std::make_shared(); mpp_query_map.insert({query_id, ptr}); @@ -43,7 +43,7 @@ MPPQueryTaskSetPtr MPPTaskManager::addMPPQueryTaskSet(UInt64 query_id) return ptr; } -void MPPTaskManager::removeMPPQueryTaskSet(UInt64 query_id, bool on_abort) +void MPPTaskManager::removeMPPQueryTaskSet(const MPPQueryId & query_id, bool on_abort) { scheduler->deleteQuery(query_id, *this, on_abort); mpp_query_map.erase(query_id); @@ -53,16 +53,16 @@ void MPPTaskManager::removeMPPQueryTaskSet(UInt64 query_id, bool on_abort) std::pair MPPTaskManager::findAsyncTunnel(const ::mpp::EstablishMPPConnectionRequest * request, EstablishCallData * call_data, grpc::CompletionQueue * cq) { const auto & meta = request->sender_meta(); - MPPTaskId id{meta.start_ts(), meta.task_id()}; + MPPTaskId id{meta}; Int64 sender_task_id = meta.task_id(); Int64 receiver_task_id = request->receiver_meta().task_id(); std::unique_lock lock(mu); - auto query_it = mpp_query_map.find(id.start_ts); + auto query_it = mpp_query_map.find(id.query_id); if (query_it != mpp_query_map.end() && !query_it->second->isInNormalState()) { /// if the query is aborted, return the error message - LOG_WARNING(log, fmt::format("Query {} is aborted, all its tasks are invalid.", id.start_ts)); + LOG_WARNING(log, fmt::format("Query {} is aborted, all its tasks are invalid.", id.query_id.toString())); /// meet error return {nullptr, query_it->second->error_message}; } @@ -73,7 +73,7 @@ std::pair MPPTaskManager::findAsyncTunnel(const ::mpp::Est if (!call_data->isWaitingTunnelState()) { /// if call_data is in new_request state, put it to waiting tunnel state - auto query_set = query_it == mpp_query_map.end() ? addMPPQueryTaskSet(id.start_ts) : query_it->second; + auto query_set = query_it == mpp_query_map.end() ? addMPPQueryTaskSet(id.query_id) : query_it->second; auto & alarm = query_set->alarms[sender_task_id][receiver_task_id]; call_data->setToWaitingTunnelState(); alarm.Set(cq, Clock::now() + std::chrono::seconds(10), call_data); @@ -96,11 +96,11 @@ std::pair MPPTaskManager::findAsyncTunnel(const ::mpp::Est { /// if the query task set has no mpp task, it has to be removed if there is no alarms left, /// otherwise the query task set itself may be left in MPPTaskManager forever - removeMPPQueryTaskSet(id.start_ts, false); + removeMPPQueryTaskSet(id.query_id, false); cv.notify_all(); } } - return {nullptr, fmt::format("Can't find task [{},{}] within 10 s.", id.start_ts, id.task_id)}; + return {nullptr, fmt::format("Can't find task [{}] within 10 s.", id.toString())}; } } /// don't need to delete the alarm here because registerMPPTask will delete all the related alarm @@ -112,13 +112,13 @@ std::pair MPPTaskManager::findAsyncTunnel(const ::mpp::Est std::pair MPPTaskManager::findTunnelWithTimeout(const ::mpp::EstablishMPPConnectionRequest * request, std::chrono::seconds timeout) { const auto & meta = request->sender_meta(); - MPPTaskId id{meta.start_ts(), meta.task_id()}; + MPPTaskId id{meta}; std::unordered_map::iterator it; bool cancelled = false; String error_message; std::unique_lock lock(mu); auto ret = cv.wait_for(lock, timeout, [&] { - auto query_it = mpp_query_map.find(id.start_ts); + auto query_it = mpp_query_map.find(id.query_id); // TODO: how about the query has been cancelled in advance? if (query_it == mpp_query_map.end()) { @@ -127,7 +127,7 @@ std::pair MPPTaskManager::findTunnelWithTimeout(const ::mp else if (!query_it->second->isInNormalState()) { /// if the query is aborted, return true to stop waiting timeout. - LOG_WARNING(log, fmt::format("Query {} is aborted, all its tasks are invalid.", id.start_ts)); + LOG_WARNING(log, fmt::format("Query {} is aborted, all its tasks are invalid.", id.query_id.toString())); cancelled = true; error_message = query_it->second->error_message; return true; @@ -147,9 +147,9 @@ std::pair MPPTaskManager::findTunnelWithTimeout(const ::mp return it->second->getTunnel(request); } -void MPPTaskManager::abortMPPQuery(UInt64 query_id, const String & reason, AbortType abort_type) +void MPPTaskManager::abortMPPQuery(const MPPQueryId & query_id, const String & reason, AbortType abort_type) { - LOG_WARNING(log, fmt::format("Begin to abort query: {}, abort type: {}, reason: {}", query_id, magic_enum::enum_name(abort_type), reason)); + LOG_WARNING(log, fmt::format("Begin to abort query: {}, abort type: {}, reason: {}", query_id.toString(), magic_enum::enum_name(abort_type), reason)); MPPQueryTaskSetPtr task_set; { /// abort task may take a long time, so first @@ -159,12 +159,12 @@ void MPPTaskManager::abortMPPQuery(UInt64 query_id, const String & reason, Abort auto it = mpp_query_map.find(query_id); if (it == mpp_query_map.end()) { - LOG_WARNING(log, fmt::format("{} does not found in task manager, skip abort", query_id)); + LOG_WARNING(log, fmt::format("{} does not found in task manager, skip abort", query_id.toString())); return; } else if (!it->second->isInNormalState()) { - LOG_WARNING(log, fmt::format("{} already in abort process, skip abort", query_id)); + LOG_WARNING(log, fmt::format("{} already in abort process, skip abort", query_id.toString())); return; } it->second->state = MPPQueryTaskSet::Aborting; @@ -178,7 +178,7 @@ void MPPTaskManager::abortMPPQuery(UInt64 query_id, const String & reason, Abort it->second->alarms.clear(); if (it->second->task_map.empty()) { - LOG_INFO(log, fmt::format("There is no mpp task for {}, finish abort", query_id)); + LOG_INFO(log, fmt::format("There is no mpp task for {}, finish abort", query_id.toString())); removeMPPQueryTaskSet(query_id, true); cv.notify_all(); return; @@ -189,7 +189,7 @@ void MPPTaskManager::abortMPPQuery(UInt64 query_id, const String & reason, Abort } FmtBuffer fmt_buf; - fmt_buf.fmtAppend("Remaining task in query {} are: ", query_id); + fmt_buf.fmtAppend("Remaining task in query {} are: ", query_id.toString()); for (auto & it : task_set->task_map) fmt_buf.fmtAppend("{} ", it.first.toString()); LOG_WARNING(log, fmt_buf.toString()); @@ -200,11 +200,11 @@ void MPPTaskManager::abortMPPQuery(UInt64 query_id, const String & reason, Abort { std::lock_guard lock(mu); auto it = mpp_query_map.find(query_id); - RUNTIME_ASSERT(it != mpp_query_map.end(), log, "MPPTaskQuerySet {} should remaining in MPPTaskManager", query_id); + RUNTIME_ASSERT(it != mpp_query_map.end(), log, "MPPTaskQuerySet {} should remaining in MPPTaskManager", query_id.toString()); it->second->state = MPPQueryTaskSet::Aborted; cv.notify_all(); } - LOG_WARNING(log, "Finish abort query: " + std::to_string(query_id)); + LOG_WARNING(log, "Finish abort query: " + query_id.toString()); } std::pair MPPTaskManager::registerTask(MPPTaskPtr task) @@ -214,7 +214,7 @@ std::pair MPPTaskManager::registerTask(MPPTaskPtr task) FAIL_POINT_PAUSE(FailPoints::pause_before_register_non_root_mpp_task); } std::unique_lock lock(mu); - const auto & it = mpp_query_map.find(task->id.start_ts); + const auto & it = mpp_query_map.find(task->id.query_id); if (it != mpp_query_map.end() && !it->second->isInNormalState()) { return {false, fmt::format("query is being aborted, error message = {}", it->second->error_message)}; @@ -226,7 +226,7 @@ std::pair MPPTaskManager::registerTask(MPPTaskPtr task) MPPQueryTaskSetPtr query_set; if (it == mpp_query_map.end()) /// the first one { - query_set = addMPPQueryTaskSet(task->id.start_ts); + query_set = addMPPQueryTaskSet(task->id.query_id); } else { @@ -251,7 +251,7 @@ std::pair MPPTaskManager::unregisterTask(const MPPTaskId & id) std::unique_lock lock(mu); auto it = mpp_query_map.end(); cv.wait(lock, [&] { - it = mpp_query_map.find(id.start_ts); + it = mpp_query_map.find(id.query_id); return it == mpp_query_map.end() || it->second->allowUnregisterTask(); }); if (it != mpp_query_map.end()) @@ -261,7 +261,7 @@ std::pair MPPTaskManager::unregisterTask(const MPPTaskId & id) { it->second->task_map.erase(task_it); if (it->second->task_map.empty() && it->second->alarms.empty()) - removeMPPQueryTaskSet(id.start_ts, false); + removeMPPQueryTaskSet(id.query_id, false); cv.notify_all(); return {true, ""}; } @@ -282,13 +282,13 @@ String MPPTaskManager::toString() return res + ")"; } -MPPQueryTaskSetPtr MPPTaskManager::getQueryTaskSetWithoutLock(UInt64 query_id) +MPPQueryTaskSetPtr MPPTaskManager::getQueryTaskSetWithoutLock(const MPPQueryId & query_id) { auto it = mpp_query_map.find(query_id); return it == mpp_query_map.end() ? nullptr : it->second; } -MPPQueryTaskSetPtr MPPTaskManager::getQueryTaskSet(UInt64 query_id) +MPPQueryTaskSetPtr MPPTaskManager::getQueryTaskSet(const MPPQueryId & query_id) { std::lock_guard lock(mu); return getQueryTaskSetWithoutLock(query_id); diff --git a/dbms/src/Flash/Mpp/MPPTaskManager.h b/dbms/src/Flash/Mpp/MPPTaskManager.h index 018a8631880..b986d8ae866 100644 --- a/dbms/src/Flash/Mpp/MPPTaskManager.h +++ b/dbms/src/Flash/Mpp/MPPTaskManager.h @@ -55,9 +55,9 @@ struct MPPQueryTaskSet using MPPQueryTaskSetPtr = std::shared_ptr; /// a map from the mpp query id to mpp query task set, we use -/// the start ts of a query as the query id as TiDB will guarantee -/// the uniqueness of the start ts -using MPPQueryMap = std::unordered_map; +/// the query_ts + local_query_id + serverID as the query id, because TiDB can't guarantee +/// the uniqueness of the start ts when stale read or set snapshot +using MPPQueryMap = std::unordered_map; // MPPTaskManger holds all running mpp tasks. It's a single instance holden in Context. class MPPTaskManager : private boost::noncopyable @@ -77,9 +77,9 @@ class MPPTaskManager : private boost::noncopyable ~MPPTaskManager() = default; - MPPQueryTaskSetPtr getQueryTaskSetWithoutLock(UInt64 query_id); + MPPQueryTaskSetPtr getQueryTaskSetWithoutLock(const MPPQueryId & query_id); - MPPQueryTaskSetPtr getQueryTaskSet(UInt64 query_id); + MPPQueryTaskSetPtr getQueryTaskSet(const MPPQueryId & query_id); std::pair registerTask(MPPTaskPtr task); @@ -93,13 +93,13 @@ class MPPTaskManager : private boost::noncopyable std::pair findAsyncTunnel(const ::mpp::EstablishMPPConnectionRequest * request, EstablishCallData * call_data, grpc::CompletionQueue * cq); - void abortMPPQuery(UInt64 query_id, const String & reason, AbortType abort_type); + void abortMPPQuery(const MPPQueryId & query_id, const String & reason, AbortType abort_type); String toString(); private: - MPPQueryTaskSetPtr addMPPQueryTaskSet(UInt64 query_id); - void removeMPPQueryTaskSet(UInt64 query_id, bool on_abort); + MPPQueryTaskSetPtr addMPPQueryTaskSet(const MPPQueryId & query_id); + void removeMPPQueryTaskSet(const MPPQueryId & query_id, bool on_abort); }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp b/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp index 523103611ca..a18403ba5d5 100644 --- a/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskStatistics.cpp @@ -96,14 +96,15 @@ void MPPTaskStatistics::logTracingJson() { LOG_INFO( logger, - R"({{"query_tso":{},"task_id":{},"is_root":{},"sender_executor_id":"{}","executors":{},"host":"{}")" + R"({{"query_tso":{},"task_id":{},"query_id":{},"is_root":{},"sender_executor_id":"{}","executors":{},"host":"{}")" R"(,"task_init_timestamp":{},"task_start_timestamp":{},"task_end_timestamp":{})" R"(,"compile_start_timestamp":{},"compile_end_timestamp":{})" R"(,"read_wait_index_start_timestamp":{},"read_wait_index_end_timestamp":{})" R"(,"local_input_bytes":{},"remote_input_bytes":{},"output_bytes":{})" R"(,"status":"{}","error_message":"{}","working_time":{},"memory_peak":{}}})", - id.start_ts, + id.query_id.start_ts, id.task_id, + id.query_id.toString(), is_root, sender_executor_id, executor_statistics_collector.resToJson(), diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp index b7deae93311..c22b3f7d063 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.cpp +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.cpp @@ -25,11 +25,10 @@ namespace FailPoints extern const char random_min_tso_scheduler_failpoint[]; } // namespace FailPoints -constexpr UInt64 MAX_UINT64 = std::numeric_limits::max(); constexpr UInt64 OS_THREAD_SOFT_LIMIT = 100000; MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit_) - : min_tso(MAX_UINT64) + : min_query_id(MPPTaskId::Max_Query_Id) , thread_soft_limit(soft_limit) , thread_hard_limit(hard_limit) , estimated_thread_usage(0) @@ -58,7 +57,7 @@ MinTSOScheduler::MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 ac { LOG_INFO(log, "thread_hard_limit is {}, thread_soft_limit is {}, and active_set_soft_limit is {} in MinTSOScheduler.", thread_hard_limit, thread_soft_limit, active_set_soft_limit); } - GET_METRIC(tiflash_task_scheduler, type_min_tso).Set(min_tso); + GET_METRIC(tiflash_task_scheduler, type_min_tso).Set(min_query_id.query_ts); GET_METRIC(tiflash_task_scheduler, type_thread_soft_limit).Set(thread_soft_limit); GET_METRIC(tiflash_task_scheduler, type_thread_hard_limit).Set(thread_hard_limit); GET_METRIC(tiflash_task_scheduler, type_estimated_thread_usage).Set(estimated_thread_usage); @@ -78,34 +77,34 @@ bool MinTSOScheduler::tryToSchedule(MPPTaskScheduleEntry & schedule_entry, MPPTa return true; } const auto & id = schedule_entry.getMPPTaskId(); - auto query_task_set = task_manager.getQueryTaskSetWithoutLock(id.start_ts); + auto query_task_set = task_manager.getQueryTaskSetWithoutLock(id.query_id); if (nullptr == query_task_set || !query_task_set->isInNormalState()) { LOG_WARNING(log, "{} is scheduled with miss or abort.", id.toString()); return true; } bool has_error = false; - return scheduleImp(id.start_ts, query_task_set, schedule_entry, false, has_error); + return scheduleImp(id.query_id, query_task_set, schedule_entry, false, has_error); } -/// after finishing the query, there would be no threads released soon, so the updated min-tso query with waiting tasks should be scheduled. +/// after finishing the query, there would be no threads released soon, so the updated min-query-id query with waiting tasks should be scheduled. /// the cancelled query maybe hang, so trigger scheduling as needed when deleting cancelled query. -void MinTSOScheduler::deleteQuery(const UInt64 tso, MPPTaskManager & task_manager, const bool is_cancelled) +void MinTSOScheduler::deleteQuery(const MPPQueryId & query_id, MPPTaskManager & task_manager, const bool is_cancelled) { if (isDisabled()) { return; } - LOG_DEBUG(log, "{} query {} (is min = {}) is deleted from active set {} left {} or waiting set {} left {}.", is_cancelled ? "Cancelled" : "Finished", tso, tso == min_tso, active_set.find(tso) != active_set.end(), active_set.size(), waiting_set.find(tso) != waiting_set.end(), waiting_set.size()); - active_set.erase(tso); - waiting_set.erase(tso); + LOG_DEBUG(log, "{} query {} (is min = {}) is deleted from active set {} left {} or waiting set {} left {}.", is_cancelled ? "Cancelled" : "Finished", query_id.toString(), query_id == min_query_id, active_set.find(query_id) != active_set.end(), active_set.size(), waiting_set.find(query_id) != waiting_set.end(), waiting_set.size()); + active_set.erase(query_id); + waiting_set.erase(query_id); GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); GET_METRIC(tiflash_task_scheduler, type_active_queries_count).Set(active_set.size()); if (is_cancelled) /// cancelled queries may have waiting tasks, and finished queries haven't. { - auto query_task_set = task_manager.getQueryTaskSetWithoutLock(tso); + auto query_task_set = task_manager.getQueryTaskSetWithoutLock(query_id); if (query_task_set) /// release all waiting tasks { while (!query_task_set->waiting_tasks.empty()) @@ -119,8 +118,8 @@ void MinTSOScheduler::deleteQuery(const UInt64 tso, MPPTaskManager & task_manage } } - /// NOTE: if updated min_tso query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. - if (updateMinTSO(tso, true, is_cancelled ? "when cancelling it" : "as finishing it")) + /// NOTE: if updated min_query_id query has waiting tasks, they should be scheduled, especially when the soft-limited threads are amost used and active tasks are in resources deadlock which cannot release threads soon. + if (updateMinQueryId(query_id, true, is_cancelled ? "when cancelling it" : "as finishing it")) { scheduleWaitingQueries(task_manager); } @@ -153,8 +152,8 @@ void MinTSOScheduler::scheduleWaitingQueries(MPPTaskManager & task_manager) auto query_task_set = task_manager.getQueryTaskSetWithoutLock(current_query_id); if (nullptr == query_task_set) /// silently solve this rare case { - LOG_ERROR(log, "the waiting query {} is not in the task manager.", current_query_id); - updateMinTSO(current_query_id, true, "as it is not in the task manager."); + LOG_ERROR(log, "the waiting query {} is not in the task manager.", current_query_id.toString()); + updateMinQueryId(current_query_id, true, "as it is not in the task manager."); active_set.erase(current_query_id); waiting_set.erase(current_query_id); GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); @@ -162,7 +161,7 @@ void MinTSOScheduler::scheduleWaitingQueries(MPPTaskManager & task_manager) continue; } - LOG_DEBUG(log, "query {} (is min = {}) with {} tasks is to be scheduled from waiting set (size = {}).", current_query_id, current_query_id == min_tso, query_task_set->waiting_tasks.size(), waiting_set.size()); + LOG_DEBUG(log, "query {} (is min = {}) with {} tasks is to be scheduled from waiting set (size = {}).", current_query_id.toString(), current_query_id == min_query_id, query_task_set->waiting_tasks.size(), waiting_set.size()); /// schedule tasks one by one while (!query_task_set->waiting_tasks.empty()) { @@ -180,22 +179,22 @@ void MinTSOScheduler::scheduleWaitingQueries(MPPTaskManager & task_manager) query_task_set->waiting_tasks.pop(); GET_METRIC(tiflash_task_scheduler, type_waiting_tasks_count).Decrement(); } - LOG_DEBUG(log, "query {} (is min = {}) is scheduled from waiting set (size = {}).", current_query_id, current_query_id == min_tso, waiting_set.size()); + LOG_DEBUG(log, "query {} (is min = {}) is scheduled from waiting set (size = {}).", current_query_id.toString(), current_query_id == min_query_id, waiting_set.size()); waiting_set.erase(current_query_id); /// all waiting tasks of this query are fully active GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); } } -/// [directly schedule, from waiting set] * [is min_tso query, not] * [can schedule, can't] totally 8 cases. -bool MinTSOScheduler::scheduleImp(const UInt64 tso, const MPPQueryTaskSetPtr & query_task_set, MPPTaskScheduleEntry & schedule_entry, const bool isWaiting, bool & has_error) +/// [directly schedule, from waiting set] * [is min_query_id query, not] * [can schedule, can't] totally 8 cases. +bool MinTSOScheduler::scheduleImp(const MPPQueryId & query_id, const MPPQueryTaskSetPtr & query_task_set, MPPTaskScheduleEntry & schedule_entry, const bool isWaiting, bool & has_error) { auto needed_threads = schedule_entry.getNeededThreads(); - auto check_for_new_min_tso = tso <= min_tso && estimated_thread_usage + needed_threads <= thread_hard_limit; - auto check_for_not_min_tso = (active_set.size() < active_set_soft_limit || tso <= *active_set.rbegin()) && (estimated_thread_usage + needed_threads <= thread_soft_limit); + auto check_for_new_min_tso = query_id <= min_query_id && estimated_thread_usage + needed_threads <= thread_hard_limit; + auto check_for_not_min_tso = (active_set.size() < active_set_soft_limit || query_id <= *active_set.rbegin()) && (estimated_thread_usage + needed_threads <= thread_soft_limit); if (check_for_new_min_tso || check_for_not_min_tso) { - updateMinTSO(tso, false, isWaiting ? "from the waiting set" : "when directly schedule it"); - active_set.insert(tso); + updateMinQueryId(query_id, false, isWaiting ? "from the waiting set" : "when directly schedule it"); + active_set.insert(query_id); if (schedule_entry.schedule(ScheduleState::SCHEDULED)) { estimated_thread_usage += needed_threads; @@ -203,24 +202,24 @@ bool MinTSOScheduler::scheduleImp(const UInt64 tso, const MPPQueryTaskSetPtr & q } GET_METRIC(tiflash_task_scheduler, type_active_queries_count).Set(active_set.size()); GET_METRIC(tiflash_task_scheduler, type_estimated_thread_usage).Set(estimated_thread_usage); - LOG_INFO(log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} of the thread {} limit {}.", schedule_entry.getMPPTaskId().toString(), active_set.size(), isWaiting ? "from the waiting set" : "directly", needed_threads, estimated_thread_usage, min_tso == tso ? "hard" : "soft", min_tso == tso ? thread_hard_limit : thread_soft_limit); + LOG_INFO(log, "{} is scheduled (active set size = {}) due to available threads {}, after applied for {} threads, used {} of the thread {} limit {}.", schedule_entry.getMPPTaskId().toString(), active_set.size(), isWaiting ? "from the waiting set" : "directly", needed_threads, estimated_thread_usage, min_query_id == query_id ? "hard" : "soft", min_query_id == query_id ? thread_hard_limit : thread_soft_limit); return true; } else { - bool is_tso_min = tso <= min_tso; - fiu_do_on(FailPoints::random_min_tso_scheduler_failpoint, is_tso_min = true;); - if (is_tso_min) /// the min_tso query should fully run, otherwise throw errors here. + bool is_query_id_min = query_id <= min_query_id; + fiu_do_on(FailPoints::random_min_tso_scheduler_failpoint, is_query_id_min = true;); + if (is_query_id_min) /// the min_query_id query should fully run, otherwise throw errors here. { has_error = true; - auto msg = fmt::format("threads are unavailable for the query {} ({} min_tso {}) {}, need {}, but used {} of the thread hard limit {}, {} active and {} waiting queries.", tso, tso == min_tso ? "is" : "is newer than", min_tso, isWaiting ? "from the waiting set" : "when directly schedule it", needed_threads, estimated_thread_usage, thread_hard_limit, active_set.size(), waiting_set.size()); + auto msg = fmt::format("threads are unavailable for the query {} ({} min_query_id {}) {}, need {}, but used {} of the thread hard limit {}, {} active and {} waiting queries.", query_id.toString(), query_id == min_query_id ? "is" : "is newer than", min_query_id.toString(), isWaiting ? "from the waiting set" : "when directly schedule it", needed_threads, estimated_thread_usage, thread_hard_limit, active_set.size(), waiting_set.size()); LOG_ERROR(log, "{}", msg); GET_METRIC(tiflash_task_scheduler, type_hard_limit_exceeded_count).Increment(); if (isWaiting) { - /// set this task be failed to schedule, and the task will throw exception, then TiDB will finally notify this tiflash node canceling all tasks of this tso and update metrics. + /// set this task be failed to schedule, and the task will throw exception, then TiDB will finally notify this tiflash node canceling all tasks of this query_id and update metrics. schedule_entry.schedule(ScheduleState::EXCEEDED); - waiting_set.erase(tso); /// avoid the left waiting tasks of this query reaching here many times. + waiting_set.erase(query_id); /// avoid the left waiting tasks of this query reaching here many times. } else { @@ -230,38 +229,38 @@ bool MinTSOScheduler::scheduleImp(const UInt64 tso, const MPPQueryTaskSetPtr & q } if (!isWaiting) { - waiting_set.insert(tso); + waiting_set.insert(query_id); query_task_set->waiting_tasks.push(schedule_entry.getMPPTaskId()); GET_METRIC(tiflash_task_scheduler, type_waiting_queries_count).Set(waiting_set.size()); GET_METRIC(tiflash_task_scheduler, type_waiting_tasks_count).Increment(); } - LOG_INFO(log, "threads are unavailable for the query {} or active set is full (size = {}), need {}, but used {} of the thread soft limit {},{} waiting set size = {}", tso, active_set.size(), needed_threads, estimated_thread_usage, thread_soft_limit, isWaiting ? "" : " put into", waiting_set.size()); + LOG_INFO(log, "threads are unavailable for the query {} or active set is full (size = {}), need {}, but used {} of the thread soft limit {},{} waiting set size = {}", query_id.toString(), active_set.size(), needed_threads, estimated_thread_usage, thread_soft_limit, isWaiting ? "" : " put into", waiting_set.size()); return false; } } -/// if return true, then need to schedule the waiting tasks of the min_tso. -bool MinTSOScheduler::updateMinTSO(const UInt64 tso, const bool retired, const String & msg) +/// if return true, then need to schedule the waiting tasks of the min_query_id. +bool MinTSOScheduler::updateMinQueryId(const MPPQueryId & query_id, const bool retired, const String & msg) { - auto old_min_tso = min_tso; + auto old_min_query_id = min_query_id; bool force_scheduling = false; if (retired) { - if (tso == min_tso) /// elect a new min_tso from all queries. + if (query_id == min_query_id) /// elect a new min_query_id from all queries. { - min_tso = active_set.empty() ? MAX_UINT64 : *active_set.begin(); - min_tso = waiting_set.empty() ? min_tso : std::min(*waiting_set.begin(), min_tso); - force_scheduling = waiting_set.find(min_tso) != waiting_set.end(); /// if this min_tso has waiting tasks, these tasks should force being scheduled. + min_query_id = active_set.empty() ? MPPTaskId::Max_Query_Id : *active_set.begin(); + min_query_id = waiting_set.empty() ? min_query_id : std::min(*waiting_set.begin(), min_query_id); + force_scheduling = waiting_set.find(min_query_id) != waiting_set.end(); /// if this min_query_id has waiting tasks, these tasks should force being scheduled. } } else { - min_tso = std::min(tso, min_tso); + min_query_id = std::min(query_id, min_query_id); } - if (min_tso != old_min_tso) /// if min_tso == MAX_UINT64 and the query tso is not to be cancelled, the used_threads, active_set.size() and waiting_set.size() must be 0. + if (min_query_id != old_min_query_id) /// if min_query_id == MPPTaskId::Max_Query_Id and the query_id is not to be cancelled, the used_threads, active_set.size() and waiting_set.size() must be 0. { - GET_METRIC(tiflash_task_scheduler, type_min_tso).Set(min_tso); - LOG_INFO(log, "min_tso query is updated from {} to {} {}, used threads = {}, {} active and {} waiting queries.", old_min_tso, min_tso, msg, estimated_thread_usage, active_set.size(), waiting_set.size()); + GET_METRIC(tiflash_task_scheduler, type_min_tso).Set(min_query_id.query_ts == 0 ? min_query_id.start_ts : min_query_id.query_ts); + LOG_INFO(log, "min_query_id query is updated from {} to {} {}, used threads = {}, {} active and {} waiting queries.", old_min_query_id.toString(), min_query_id.toString(), msg, estimated_thread_usage, active_set.size(), waiting_set.size()); } return force_scheduling; } diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.h b/dbms/src/Flash/Mpp/MinTSOScheduler.h index 25abb770e44..dfe2921d1fb 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.h +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.h @@ -28,8 +28,8 @@ using MPPTaskManagerPtr = std::shared_ptr; struct MPPQueryTaskSet; using MPPQueryTaskSetPtr = std::shared_ptr; -/// scheduling tasks in the set according to the tso order under the soft limit of threads, but allow the min_tso query to preempt threads under the hard limit of threads. -/// The min_tso query avoids the deadlock resulted from threads competition among nodes. +/// scheduling tasks in the set according to the tso order under the soft limit of threads, but allow the min_query_id query to preempt threads under the hard limit of threads. +/// The min_query_id query avoids the deadlock resulted from threads competition among nodes. /// schedule tasks under the lock protection of the task manager. /// NOTE: if the updated min-tso query has waiting tasks, necessarily scheduling them, otherwise the query would hang. class MinTSOScheduler : private boost::noncopyable @@ -37,28 +37,28 @@ class MinTSOScheduler : private boost::noncopyable public: MinTSOScheduler(UInt64 soft_limit, UInt64 hard_limit, UInt64 active_set_soft_limit_); ~MinTSOScheduler() = default; - /// try to schedule this task if it is the min_tso query or there are enough threads, otherwise put it into the waiting set. + /// try to schedule this task if it is the min_query_id query or there are enough threads, otherwise put it into the waiting set. /// NOTE: call tryToSchedule under the lock protection of MPPTaskManager bool tryToSchedule(MPPTaskScheduleEntry & schedule_entry, MPPTaskManager & task_manager); - /// delete this to-be cancelled/finished query from scheduler and update min_tso if needed, so that there aren't cancelled/finished queries in the scheduler. + /// delete this to-be cancelled/finished query from scheduler and update min_query_id if needed, so that there aren't cancelled/finished queries in the scheduler. /// NOTE: call deleteQuery under the lock protection of MPPTaskManager - void deleteQuery(const UInt64 tso, MPPTaskManager & task_manager, const bool is_cancelled); + void deleteQuery(const MPPQueryId & query_id, MPPTaskManager & task_manager, const bool is_cancelled); /// all scheduled tasks should finally call this function to release threads and schedule new tasks void releaseThreadsThenSchedule(const int needed_threads, MPPTaskManager & task_manager); private: - bool scheduleImp(const UInt64 tso, const MPPQueryTaskSetPtr & query_task_set, MPPTaskScheduleEntry & schedule_entry, const bool isWaiting, bool & has_error); - bool updateMinTSO(const UInt64 tso, const bool retired, const String & msg); + bool scheduleImp(const MPPQueryId & query_id, const MPPQueryTaskSetPtr & query_task_set, MPPTaskScheduleEntry & schedule_entry, const bool isWaiting, bool & has_error); + bool updateMinQueryId(const MPPQueryId & query_id, const bool retired, const String & msg); void scheduleWaitingQueries(MPPTaskManager & task_manager); bool isDisabled() { return thread_hard_limit == 0 && thread_soft_limit == 0; } - std::set waiting_set; - std::set active_set; - UInt64 min_tso; + std::set waiting_set; + std::set active_set; + MPPQueryId min_query_id; UInt64 thread_soft_limit; UInt64 thread_hard_limit; UInt64 estimated_thread_usage; diff --git a/dbms/src/Flash/Mpp/getMPPTaskTracingLog.cpp b/dbms/src/Flash/Mpp/getMPPTaskTracingLog.cpp index df04d25cac8..dce4e69db1f 100644 --- a/dbms/src/Flash/Mpp/getMPPTaskTracingLog.cpp +++ b/dbms/src/Flash/Mpp/getMPPTaskTracingLog.cpp @@ -13,11 +13,12 @@ // limitations under the License. #include +#include namespace DB { LoggerPtr getMPPTaskTracingLog(const MPPTaskId & mpp_task_id) { - return Logger::get(tracing_log_source, mpp_task_id.toString()); + return Logger::get(DB::tracing_log_source, mpp_task_id.toString()); } } // namespace DB diff --git a/dbms/src/Flash/Mpp/getMPPTaskTracingLog.h b/dbms/src/Flash/Mpp/getMPPTaskTracingLog.h index 1491d26dbf3..4da514f5795 100644 --- a/dbms/src/Flash/Mpp/getMPPTaskTracingLog.h +++ b/dbms/src/Flash/Mpp/getMPPTaskTracingLog.h @@ -19,9 +19,6 @@ namespace DB { -/// Tracing logs are filtered by SourceFilterChannel. -inline constexpr auto tracing_log_source = "mpp_task_tracing"; - /// All tracing logs must logged by the logger that got by `getMPPTaskTracingLog`. LoggerPtr getMPPTaskTracingLog(const MPPTaskId & mpp_task_id); } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/tests/bench_exchange.cpp b/dbms/src/Flash/tests/bench_exchange.cpp index 3454242f2a7..6691a3b74a4 100644 --- a/dbms/src/Flash/tests/bench_exchange.cpp +++ b/dbms/src/Flash/tests/bench_exchange.cpp @@ -28,7 +28,6 @@ namespace DB { namespace tests { - std::random_device rd; MockBlockInputStream::MockBlockInputStream(const std::vector & blocks_, StopFlag & stop_flag_) @@ -167,6 +166,9 @@ ReceiverHelper::ReceiverHelper(int concurrency_, int source_num_, uint32_t fine_ { mpp::TaskMeta task; task.set_start_ts(0); + task.set_query_ts(i); + task.set_server_id(i); + task.set_local_query_id(i); task.set_task_id(i); task.set_partition_id(i); task.set_address(""); diff --git a/dbms/src/Flash/tests/bench_window.cpp b/dbms/src/Flash/tests/bench_window.cpp index 9f68ba9beb9..eb14e12f314 100644 --- a/dbms/src/Flash/tests/bench_window.cpp +++ b/dbms/src/Flash/tests/bench_window.cpp @@ -47,7 +47,7 @@ class WindowFunctionBench : public ExchangeBench buildDefaultRowsFrame(), fine_grained_shuffle_stream_count); tipb::DAGRequest req; - MPPInfo mpp_info(0, -1, -1, {}, std::unordered_map>{}); + MPPInfo mpp_info(0, 0, 0, 0, -1, -1, {}, std::unordered_map>{}); builder.getRoot()->toTiPBExecutor(req.mutable_root_executor(), /*collator_id=*/0, mpp_info, TiFlashTestEnv::getContext()); assert(req.root_executor().tp() == tipb::TypeWindow); window = req.root_executor().window(); diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index 43da894ec64..264db3ea876 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -18,7 +18,6 @@ namespace DB { namespace tests { - LoggerPtr MPPTaskTestUtils::log_ptr = nullptr; size_t MPPTaskTestUtils::server_num = 0; MPPTestMeta MPPTaskTestUtils::test_meta = {}; @@ -296,13 +295,13 @@ try { startServers(4); { - auto [start_ts, res] = prepareMPPStreams(context + auto [query_id, res] = prepareMPPStreams(context .scan("test_db", "test_table_1") .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) .project({"max(s1)"})); - EXPECT_TRUE(assertQueryActive(start_ts)); - MockComputeServerManager::instance().cancelQuery(start_ts); - EXPECT_TRUE(assertQueryCancelled(start_ts)); + EXPECT_TRUE(assertQueryActive(query_id)); + MockComputeServerManager::instance().cancelQuery(query_id); + EXPECT_TRUE(assertQueryCancelled(query_id)); } } CATCH @@ -312,12 +311,12 @@ try { startServers(4); { - auto [start_ts, res] = prepareMPPStreams(context + auto [query_id, res] = prepareMPPStreams(context .scan("test_db", "l_table") .join(context.scan("test_db", "r_table"), tipb::JoinType::TypeLeftOuterJoin, {col("join_c")})); - EXPECT_TRUE(assertQueryActive(start_ts)); - MockComputeServerManager::instance().cancelQuery(start_ts); - EXPECT_TRUE(assertQueryCancelled(start_ts)); + EXPECT_TRUE(assertQueryActive(query_id)); + MockComputeServerManager::instance().cancelQuery(query_id); + EXPECT_TRUE(assertQueryCancelled(query_id)); } } CATCH @@ -327,14 +326,14 @@ try { startServers(4); { - auto [start_ts, _] = prepareMPPStreams(context + auto [query_id, _] = prepareMPPStreams(context .scan("test_db", "l_table") .join(context.scan("test_db", "r_table"), tipb::JoinType::TypeLeftOuterJoin, {col("join_c")}) .aggregation({Max(col("l_table.s"))}, {col("l_table.s")}) .project({col("max(l_table.s)"), col("l_table.s")})); - EXPECT_TRUE(assertQueryActive(start_ts)); - MockComputeServerManager::instance().cancelQuery(start_ts); - EXPECT_TRUE(assertQueryCancelled(start_ts)); + EXPECT_TRUE(assertQueryActive(query_id)); + MockComputeServerManager::instance().cancelQuery(query_id); + EXPECT_TRUE(assertQueryCancelled(query_id)); } } CATCH @@ -344,27 +343,27 @@ try { startServers(4); { - auto [start_ts1, res1] = prepareMPPStreams(context + auto [query_id1, res1] = prepareMPPStreams(context .scan("test_db", "l_table") .join(context.scan("test_db", "r_table"), tipb::JoinType::TypeLeftOuterJoin, {col("join_c")})); - auto [start_ts2, res2] = prepareMPPStreams(context + auto [query_id2, res2] = prepareMPPStreams(context .scan("test_db", "l_table") .join(context.scan("test_db", "r_table"), tipb::JoinType::TypeLeftOuterJoin, {col("join_c")}) .aggregation({Max(col("l_table.s"))}, {col("l_table.s")}) .project({col("max(l_table.s)"), col("l_table.s")})); - EXPECT_TRUE(assertQueryActive(start_ts1)); - MockComputeServerManager::instance().cancelQuery(start_ts1); - EXPECT_TRUE(assertQueryCancelled(start_ts1)); + EXPECT_TRUE(assertQueryActive(query_id1)); + MockComputeServerManager::instance().cancelQuery(query_id1); + EXPECT_TRUE(assertQueryCancelled(query_id1)); - EXPECT_TRUE(assertQueryActive(start_ts2)); - MockComputeServerManager::instance().cancelQuery(start_ts2); - EXPECT_TRUE(assertQueryCancelled(start_ts2)); + EXPECT_TRUE(assertQueryActive(query_id2)); + MockComputeServerManager::instance().cancelQuery(query_id2); + EXPECT_TRUE(assertQueryCancelled(query_id2)); } // start 10 queries { - std::vector>> queries; + std::vector>> queries; for (size_t i = 0; i < 10; ++i) { queries.push_back(prepareMPPStreams(context @@ -373,10 +372,10 @@ try } for (size_t i = 0; i < 10; ++i) { - auto start_ts = std::get<0>(queries[i]); - EXPECT_TRUE(assertQueryActive(start_ts)); - MockComputeServerManager::instance().cancelQuery(start_ts); - EXPECT_TRUE(assertQueryCancelled(start_ts)); + auto query_id = std::get<0>(queries[i]); + EXPECT_TRUE(assertQueryActive(query_id)); + MockComputeServerManager::instance().cancelQuery(query_id); + EXPECT_TRUE(assertQueryCancelled(query_id)); } } } diff --git a/dbms/src/Storages/StorageDisaggregated.cpp b/dbms/src/Storages/StorageDisaggregated.cpp index 8a447637b31..652032b8dc5 100644 --- a/dbms/src/Storages/StorageDisaggregated.cpp +++ b/dbms/src/Storages/StorageDisaggregated.cpp @@ -97,8 +97,11 @@ StorageDisaggregated::RequestAndRegionIDs StorageDisaggregated::buildDispatchMPP std::vector region_ids; auto dispatch_req = std::make_shared<::mpp::DispatchTaskRequest>(); ::mpp::TaskMeta * dispatch_req_meta = dispatch_req->mutable_meta(); - dispatch_req_meta->set_start_ts(sender_target_task_start_ts); - dispatch_req_meta->set_task_id(sender_target_task_task_id); + dispatch_req_meta->set_start_ts(sender_target_mpp_task_id.query_id.start_ts); + dispatch_req_meta->set_query_ts(sender_target_mpp_task_id.query_id.query_ts); + dispatch_req_meta->set_local_query_id(sender_target_mpp_task_id.query_id.local_query_id); + dispatch_req_meta->set_server_id(sender_target_mpp_task_id.query_id.server_id); + dispatch_req_meta->set_task_id(sender_target_mpp_task_id.task_id); dispatch_req_meta->set_address(batch_cop_task.store_addr); const auto & settings = context.getSettings(); dispatch_req->set_timeout(60); @@ -164,10 +167,9 @@ StorageDisaggregated::RequestAndRegionIDs StorageDisaggregated::buildDispatchMPP tipb::Executor * executor = sender_dag_req.mutable_root_executor(); executor->set_tp(tipb::ExecType::TypeExchangeSender); // Exec summary of ExchangeSender will be merged into TableScan. - executor->set_executor_id(fmt::format("{}_{}_{}", + executor->set_executor_id(fmt::format("{}_{}", ExecIDPrefixForTiFlashStorageSender, - sender_target_task_start_ts, - sender_target_task_task_id)); + sender_target_mpp_task_id.toString())); tipb::ExchangeSender * sender = executor->mutable_exchange_sender(); sender->set_tp(tipb::ExchangeType::PassThrough); diff --git a/dbms/src/Storages/StorageDisaggregated.h b/dbms/src/Storages/StorageDisaggregated.h index b878ff21695..59971b05012 100644 --- a/dbms/src/Storages/StorageDisaggregated.h +++ b/dbms/src/Storages/StorageDisaggregated.h @@ -29,7 +29,6 @@ namespace DB { - // Naive implementation of StorageDisaggregated, all region data will be transferred by GRPC, // rewrite this when local cache is supported. // Naive StorageDisaggregated will convert TableScan to ExchangeReceiver(executed in tiflash_compute node), @@ -45,8 +44,7 @@ class StorageDisaggregated : public IStorage , context(context_) , table_scan(table_scan_) , log(Logger::get(context_.getDAGContext()->log ? context_.getDAGContext()->log->identifier() : "")) - , sender_target_task_start_ts(context_.getDAGContext()->getMPPTaskMeta().start_ts()) - , sender_target_task_task_id(context_.getDAGContext()->getMPPTaskMeta().task_id()) + , sender_target_mpp_task_id(context_.getDAGContext()->getMPPTaskMeta()) , push_down_filter(push_down_filter_) {} @@ -87,8 +85,7 @@ class StorageDisaggregated : public IStorage Context & context; const TiDBTableScan & table_scan; LoggerPtr log; - uint64_t sender_target_task_start_ts; - int64_t sender_target_task_task_id; + MPPTaskId sender_target_mpp_task_id; const PushDownFilter & push_down_filter; std::shared_ptr exchange_receiver; diff --git a/dbms/src/Storages/Transaction/LearnerRead.cpp b/dbms/src/Storages/Transaction/LearnerRead.cpp index a6f4bf18cea..bc9cde099fb 100644 --- a/dbms/src/Storages/Transaction/LearnerRead.cpp +++ b/dbms/src/Storages/Transaction/LearnerRead.cpp @@ -208,29 +208,40 @@ LearnerReadSnapshot doLearnerRead( std::vector batch_read_index_req; batch_read_index_req.reserve(ori_batch_region_size); + size_t stale_read_count = 0; { // If using `std::numeric_limits::max()`, set `start-ts` 0 to get the latest index but let read-index-worker do not record as history. auto read_index_tso = mvcc_query_info->read_tso == std::numeric_limits::max() ? 0 : mvcc_query_info->read_tso; - + RegionTable & region_table = tmt.getRegionTable(); for (size_t region_idx = region_begin_idx; region_idx < region_end_idx; ++region_idx) { const auto & region_to_query = regions_info[region_idx]; const RegionID region_id = region_to_query.region_id; - if (auto ori_read_index = mvcc_query_info.getReadIndexRes(region_id); ori_read_index) + UInt64 physical_tso = read_index_tso >> TsoPhysicalShiftBits; + bool can_stale_read = physical_tso < region_table.getSelfSafeTS(region_id); + if (!can_stale_read) { - auto resp = kvrpcpb::ReadIndexResponse(); - resp.set_read_index(ori_read_index); - batch_read_index_result.emplace(region_id, std::move(resp)); + if (auto ori_read_index = mvcc_query_info.getReadIndexRes(region_id); ori_read_index) + { + auto resp = kvrpcpb::ReadIndexResponse(); + resp.set_read_index(ori_read_index); + batch_read_index_result.emplace(region_id, std::move(resp)); + } + else + { + auto & region = regions_snapshot.find(region_id)->second; + batch_read_index_req.emplace_back(GenRegionReadIndexReq(*region, read_index_tso)); + } } else { - auto & region = regions_snapshot.find(region_id)->second; - batch_read_index_req.emplace_back(GenRegionReadIndexReq(*region, read_index_tso)); + batch_read_index_result.emplace(region_id, kvrpcpb::ReadIndexResponse()); + ++stale_read_count; } } } - + GET_METRIC(tiflash_stale_read_count).Increment(stale_read_count); GET_METRIC(tiflash_raft_read_index_count).Increment(batch_read_index_req.size()); const auto & make_default_batch_read_index_result = [&](bool with_region_error) { @@ -307,7 +318,11 @@ LearnerReadSnapshot doLearnerRead( else { // cache read-index to avoid useless overhead about retry. - mvcc_query_info.addReadIndexRes(region_id, resp.read_index()); + // resp.read_index() is 0 when stale read, skip it to avoid overwriting read_index res in last retry. + if (resp.read_index() != 0) + { + mvcc_query_info.addReadIndexRes(region_id, resp.read_index()); + } } } diff --git a/dbms/src/Storages/Transaction/RegionTable.cpp b/dbms/src/Storages/Transaction/RegionTable.cpp index a82fef8b1ad..956ca227051 100644 --- a/dbms/src/Storages/Transaction/RegionTable.cpp +++ b/dbms/src/Storages/Transaction/RegionTable.cpp @@ -505,6 +505,17 @@ bool RegionTable::isSafeTSLag(UInt64 region_id, UInt64 * leader_safe_ts, UInt64 return (*leader_safe_ts > *self_safe_ts) && (*leader_safe_ts - *self_safe_ts > SafeTsDiffThreshold); } +UInt64 RegionTable::getSelfSafeTS(UInt64 region_id) +{ + std::shared_lock lock(rw_lock); + auto it = safe_ts_map.find(region_id); + if (it == safe_ts_map.end()) + { + return 0; + } + return it->second->self_safe_ts.load(std::memory_order_relaxed); +} + void RegionTable::updateSafeTS(UInt64 region_id, UInt64 leader_safe_ts, UInt64 self_safe_ts) { { diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index 9b4fe1a4286..36686b44d90 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -59,8 +59,15 @@ using RegionScanFilterPtr = std::shared_ptr; using SafeTS = UInt64; enum : SafeTS { - InvalidSafeTS = std::numeric_limits::max() + InvalidSafeTS = std::numeric_limits::max(), }; + +using TsoShiftBits = UInt64; +enum : TsoShiftBits +{ + TsoPhysicalShiftBits = 18, +}; + class RegionTable : private boost::noncopyable { public: @@ -189,6 +196,7 @@ class RegionTable : private boost::noncopyable static const UInt64 SafeTsDiffThreshold = 2 * 60 * 1000; bool isSafeTSLag(UInt64 region_id, UInt64 * leader_safe_ts, UInt64 * self_safe_ts); + UInt64 getSelfSafeTS(UInt64 region_id); private: friend class MockTiDB; diff --git a/dbms/src/TestUtils/MPPTaskTestUtils.cpp b/dbms/src/TestUtils/MPPTaskTestUtils.cpp index d86ce0befd8..f9a9123057c 100644 --- a/dbms/src/TestUtils/MPPTaskTestUtils.cpp +++ b/dbms/src/TestUtils/MPPTaskTestUtils.cpp @@ -23,6 +23,7 @@ DAGProperties getDAGPropertiesForTest(int server_num) properties.is_mpp_query = true; properties.mpp_partition_num = server_num; properties.start_ts = MockTimeStampGenerator::instance().nextTs(); + properties.local_query_id = properties.start_ts; return properties; } @@ -68,7 +69,7 @@ size_t MPPTaskTestUtils::serverNum() return server_num; } -std::tuple> MPPTaskTestUtils::prepareMPPStreams(DAGRequestBuilder builder) +std::tuple> MPPTaskTestUtils::prepareMPPStreams(DAGRequestBuilder builder) { auto properties = DB::tests::getDAGPropertiesForTest(serverNum()); auto tasks = builder.buildMPPTasks(context, properties); @@ -76,7 +77,7 @@ std::tuple> MPPTaskTestUtils::prepareMP TiFlashTestEnv::getGlobalContext(i).setCancelTest(); MockComputeServerManager::instance().setMockStorage(context.mockStorage()); auto res = executeMPPQueryWithMultipleContext(properties, tasks, MockComputeServerManager::instance().getServerConfigMap()); - return {properties.start_ts, res}; + return {MPPQueryId(properties.query_ts, properties.local_query_id, properties.server_id, properties.start_ts), res}; } ColumnsWithTypeAndName MPPTaskTestUtils::exeucteMPPTasks(QueryTasks & tasks, const DAGProperties & properties, std::unordered_map & server_config_map) @@ -133,14 +134,14 @@ String MPPTaskTestUtils::queryInfo(size_t server_id) return buf.toString(); } -::testing::AssertionResult MPPTaskTestUtils::assertQueryCancelled(size_t start_ts) +::testing::AssertionResult MPPTaskTestUtils::assertQueryCancelled(const MPPQueryId & query_id) { auto seconds = std::chrono::seconds(1); auto retry_times = 0; for (int i = test_meta.context_idx; i < TiFlashTestEnv::globalContextSize(); ++i) { // wait until the task is empty for - while (TiFlashTestEnv::getGlobalContext(i).getTMTContext().getMPPTaskManager()->getQueryTaskSet(start_ts) != nullptr) + while (TiFlashTestEnv::getGlobalContext(i).getTMTContext().getMPPTaskManager()->getQueryTaskSet(query_id) != nullptr) { std::this_thread::sleep_for(seconds); retry_times++; @@ -154,13 +155,13 @@ ::testing::AssertionResult MPPTaskTestUtils::assertQueryCancelled(size_t start_t return ::testing::AssertionSuccess(); } -::testing::AssertionResult MPPTaskTestUtils::assertQueryActive(size_t start_ts) +::testing::AssertionResult MPPTaskTestUtils::assertQueryActive(const MPPQueryId & query_id) { for (int i = test_meta.context_idx; i < TiFlashTestEnv::globalContextSize(); ++i) { - if (TiFlashTestEnv::getGlobalContext(i).getTMTContext().getMPPTaskManager()->getQueryTaskSet(start_ts) == nullptr) + if (TiFlashTestEnv::getGlobalContext(i).getTMTContext().getMPPTaskManager()->getQueryTaskSet(query_id) == nullptr) { - return ::testing::AssertionFailure() << "Query " << start_ts << "not active" << std::endl; + return ::testing::AssertionFailure() << "Query " << query_id.toString() << "not active" << std::endl; } } return ::testing::AssertionSuccess(); diff --git a/dbms/src/TestUtils/MPPTaskTestUtils.h b/dbms/src/TestUtils/MPPTaskTestUtils.h index ab147864492..cb0e84a2a14 100644 --- a/dbms/src/TestUtils/MPPTaskTestUtils.h +++ b/dbms/src/TestUtils/MPPTaskTestUtils.h @@ -23,7 +23,6 @@ namespace DB::tests { - DAGProperties getDAGPropertiesForTest(int server_num); class MockTimeStampGenerator : public ext::Singleton { @@ -81,14 +80,14 @@ class MPPTaskTestUtils : public ExecutorTest static size_t serverNum(); // run mpp tasks which are ready to cancel, the return value is the start_ts of query. - std::tuple> prepareMPPStreams(DAGRequestBuilder builder); + std::tuple> prepareMPPStreams(DAGRequestBuilder builder); ColumnsWithTypeAndName exeucteMPPTasks(QueryTasks & tasks, const DAGProperties & properties, std::unordered_map & server_config_map); ColumnsWithTypeAndName executeCoprocessorTask(std::shared_ptr & dag_request); - static ::testing::AssertionResult assertQueryCancelled(size_t start_ts); - static ::testing::AssertionResult assertQueryActive(size_t start_ts); + static ::testing::AssertionResult assertQueryCancelled(const MPPQueryId & query_id); + static ::testing::AssertionResult assertQueryActive(const MPPQueryId & query_id); static String queryInfo(size_t server_id); protected: diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 2ed1de79057..e6c9a82a231 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -90,7 +90,7 @@ void DAGRequestBuilder::initDAGRequest(tipb::DAGRequest & dag_request) std::shared_ptr DAGRequestBuilder::build(MockDAGRequestContext & mock_context, DAGRequestType type) { // build tree struct base executor - MPPInfo mpp_info(properties.start_ts, -1, -1, {}, mock_context.receiver_source_task_ids_map); + MPPInfo mpp_info(properties.start_ts, properties.query_ts, properties.server_id, properties.local_query_id, -1, -1, {}, mock_context.receiver_source_task_ids_map); std::shared_ptr dag_request_ptr = std::make_shared(); tipb::DAGRequest & dag_request = *dag_request_ptr; initDAGRequest(dag_request); diff --git a/libs/libcommon/include/common/logger_useful.h b/libs/libcommon/include/common/logger_useful.h index 3858258d2ce..7e9498ec853 100644 --- a/libs/libcommon/include/common/logger_useful.h +++ b/libs/libcommon/include/common/logger_useful.h @@ -26,6 +26,12 @@ #define QUERY_PREVIEW_LENGTH 160 #endif +namespace DB +{ +/// Tracing logs are filtered by SourceFilterChannel. +inline constexpr auto tracing_log_source = "mpp_task_tracing"; +} // namespace DB + namespace LogFmtDetails { // https://stackoverflow.com/questions/8487986/file-macro-shows-full-path/54335644#54335644 diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 61620e48735..0e572b13573 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -48,7 +48,6 @@ #include #include #include -#include #include #include #include diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 87c5c9ad035..4f173c6a7b3 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -8766,10 +8766,10 @@ "h": 7, "w": 12, "x": 0, - "y": 107 + "y": 9 }, "hiddenSeries": false, - "id": 35, + "id": 166, "legend": { "alignAsTable": false, "avg": false, @@ -8799,8 +8799,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_raft_read_index_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "exemplar": true, + "expr": "sum(rate(tiflash_stale_read_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", + "interval": "", "intervalFactor": 1, "legendFormat": "{{instance}}", "refId": "A" @@ -8810,7 +8812,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Read Index OPS", + "title": "Stale Read OPS", "tooltip": { "shared": true, "sort": 0, @@ -8826,6 +8828,7 @@ }, "yaxes": [ { + "$$hashKey": "object:435", "decimals": null, "format": "ops", "label": null, @@ -8835,6 +8838,7 @@ "show": true }, { + "$$hashKey": "object:436", "format": "none", "label": null, "logBase": 1, @@ -8858,16 +8862,16 @@ "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 107 + "y": 9 }, "hiddenSeries": false, - "id": 36, + "id": 35, "legend": { "alignAsTable": false, "avg": false, @@ -8897,39 +8901,18 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1.00, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "expr": "sum(rate(tiflash_raft_read_index_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "max", + "legendFormat": "{{instance}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "95", - "refId": "C" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "80", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Raft Batch Read Index Duration", + "title": "Raft Read Index OPS", "tooltip": { "shared": true, "sort": 0, @@ -8945,7 +8928,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": null, + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -8953,7 +8937,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -8982,7 +8966,7 @@ "h": 7, "w": 12, "x": 0, - "y": 114 + "y": 16 }, "hiddenSeries": false, "id": 37, @@ -9116,7 +9100,126 @@ "h": 7, "w": 12, "x": 12, - "y": 114 + "y": 16 + }, + "hiddenSeries": false, + "id": 36, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(1.00, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "max", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "95", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tiflash_raft_read_index_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "80", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Raft Batch Read Index Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of currently applying snapshots.", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, + "y": 23 }, "hiddenSeries": false, "id": 75, @@ -9220,7 +9323,7 @@ "h": 7, "w": 24, "x": 0, - "y": 121 + "y": 30 }, "hiddenSeries": false, "id": 82, @@ -9373,7 +9476,7 @@ "h": 7, "w": 12, "x": 0, - "y": 128 + "y": 37 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9443,7 +9546,7 @@ "h": 7, "w": 12, "x": 12, - "y": 128 + "y": 37 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9513,7 +9616,7 @@ "h": 7, "w": 12, "x": 0, - "y": 135 + "y": 44 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9583,7 +9686,7 @@ "h": 7, "w": 12, "x": 12, - "y": 135 + "y": 44 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9647,7 +9750,7 @@ "h": 7, "w": 24, "x": 0, - "y": 142 + "y": 51 }, "height": "", "hiddenSeries": false, @@ -9761,7 +9864,7 @@ "h": 7, "w": 12, "x": 0, - "y": 149 + "y": 58 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9830,7 +9933,7 @@ "h": 7, "w": 12, "x": 12, - "y": 149 + "y": 58 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9900,7 +10003,7 @@ "h": 7, "w": 12, "x": 0, - "y": 156 + "y": 65 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9970,7 +10073,7 @@ "h": 7, "w": 12, "x": 12, - "y": 156 + "y": 65 }, "heatmap": {}, "hideZeroBuckets": true, @@ -10040,7 +10143,7 @@ "h": 7, "w": 12, "x": 0, - "y": 163 + "y": 72 }, "heatmap": {}, "hideZeroBuckets": true, @@ -10106,7 +10209,7 @@ "h": 7, "w": 12, "x": 12, - "y": 163 + "y": 72 }, "hiddenSeries": false, "id": 91, From a96564ad09e09ca34851d46f5b106fbb7311196f Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 19 Dec 2022 11:44:53 +0800 Subject: [PATCH 23/42] move NonJoinedBlockInputStream out of Join.cpp (#6497) ref pingcap/tiflash#5900 --- .../DataStreams/NonJoinedBlockInputStream.cpp | 260 ++++++++++++++++ .../DataStreams/NonJoinedBlockInputStream.h | 74 +++++ dbms/src/Interpreters/Join.cpp | 284 +----------------- dbms/src/Interpreters/Join.h | 2 + 4 files changed, 343 insertions(+), 277 deletions(-) create mode 100644 dbms/src/DataStreams/NonJoinedBlockInputStream.cpp create mode 100644 dbms/src/DataStreams/NonJoinedBlockInputStream.h diff --git a/dbms/src/DataStreams/NonJoinedBlockInputStream.cpp b/dbms/src/DataStreams/NonJoinedBlockInputStream.cpp new file mode 100644 index 00000000000..d5d929eefd9 --- /dev/null +++ b/dbms/src/DataStreams/NonJoinedBlockInputStream.cpp @@ -0,0 +1,260 @@ +// 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 ErrorCodes +{ +extern const int UNKNOWN_SET_DATA_VARIANT; +extern const int LOGICAL_ERROR; +} // namespace ErrorCodes + +template +struct AdderNonJoined; + +template +struct AdderNonJoined +{ + static size_t add(const Mapped & mapped, size_t key_num, size_t num_columns_left, MutableColumns & columns_left, size_t num_columns_right, MutableColumns & columns_right) + { + for (size_t j = 0; j < num_columns_left; ++j) + columns_left[j]->insertDefault(); + + for (size_t j = 0; j < num_columns_right; ++j) + columns_right[j]->insertFrom(*mapped.block->getByPosition(key_num + j).column.get(), mapped.row_num); + return 1; + } +}; + +template +struct AdderNonJoined +{ + static size_t add(const Mapped & mapped, size_t key_num, size_t num_columns_left, MutableColumns & columns_left, size_t num_columns_right, MutableColumns & columns_right) + { + size_t rows_added = 0; + for (auto current = &static_cast(mapped); current != nullptr; current = current->next) + { + for (size_t j = 0; j < num_columns_left; ++j) + columns_left[j]->insertDefault(); + + for (size_t j = 0; j < num_columns_right; ++j) + columns_right[j]->insertFrom(*current->block->getByPosition(key_num + j).column.get(), current->row_num); + rows_added++; + } + return rows_added; + } +}; + +NonJoinedBlockInputStream::NonJoinedBlockInputStream(const Join & parent_, const Block & left_sample_block, size_t index_, size_t step_, size_t max_block_size_) + : parent(parent_) + , index(index_) + , step(step_) + , max_block_size(max_block_size_) + , add_not_mapped_rows(true) +{ + size_t build_concurrency = parent.getBuildConcurrency(); + if (unlikely(step > build_concurrency || index >= build_concurrency)) + throw Exception("The concurrency of NonJoinedBlockInputStream should not be larger than join build concurrency"); + + /** left_sample_block contains keys and "left" columns. + * result_sample_block - keys, "left" columns, and "right" columns. + */ + + size_t num_columns_left = left_sample_block.columns(); + size_t num_columns_right = parent.sample_block_with_columns_to_add.columns(); + + result_sample_block = materializeBlock(left_sample_block); + + /// Add columns from the right-side table to the block. + for (size_t i = 0; i < num_columns_right; ++i) + { + const ColumnWithTypeAndName & src_column = parent.sample_block_with_columns_to_add.getByPosition(i); + result_sample_block.insert(src_column.cloneEmpty()); + } + + column_indices_left.reserve(num_columns_left); + column_indices_right.reserve(num_columns_right); + BoolVec is_key_column_in_left_block(num_columns_left, false); + + for (size_t i = 0; i < num_columns_left; ++i) + { + column_indices_left.push_back(i); + } + + for (size_t i = 0; i < num_columns_right; ++i) + column_indices_right.push_back(num_columns_left + i); + + /// If use_nulls, convert left columns to Nullable. + if (parent.use_nulls) + { + for (size_t i = 0; i < num_columns_left; ++i) + { + convertColumnToNullable(result_sample_block.getByPosition(column_indices_left[i])); + } + } + + columns_left.resize(num_columns_left); + columns_right.resize(num_columns_right); + next_index = index; +} + +Block NonJoinedBlockInputStream::readImpl() +{ + if (parent.blocks.empty()) + return Block(); + + if (add_not_mapped_rows) + { + setNextCurrentNotMappedRow(); + add_not_mapped_rows = false; + } + + if (parent.strictness == ASTTableJoin::Strictness::Any) + return createBlock(parent.maps_any_full); + else if (parent.strictness == ASTTableJoin::Strictness::All) + return createBlock(parent.maps_all_full); + else + throw Exception("Logical error: unknown JOIN strictness (must be ANY or ALL)", ErrorCodes::LOGICAL_ERROR); +} + +void NonJoinedBlockInputStream::setNextCurrentNotMappedRow() +{ + while (current_not_mapped_row == nullptr && next_index < parent.rows_not_inserted_to_map.size()) + { + current_not_mapped_row = parent.rows_not_inserted_to_map[next_index]->next; + next_index += step; + } +} + +template +Block NonJoinedBlockInputStream::createBlock(const Maps & maps) +{ + size_t num_columns_left = column_indices_left.size(); + size_t num_columns_right = column_indices_right.size(); + + for (size_t i = 0; i < num_columns_left; ++i) + { + const auto & src_col = result_sample_block.safeGetByPosition(column_indices_left[i]); + columns_left[i] = src_col.type->createColumn(); + } + + for (size_t i = 0; i < num_columns_right; ++i) + { + const auto & src_col = result_sample_block.safeGetByPosition(column_indices_right[i]); + columns_right[i] = src_col.type->createColumn(); + } + + size_t rows_added = 0; + + switch (parent.type) + { +#define M(TYPE) \ + case Join::Type::TYPE: \ + rows_added = fillColumns(*maps.TYPE, num_columns_left, columns_left, num_columns_right, columns_right); \ + break; + APPLY_FOR_JOIN_VARIANTS(M) +#undef M + + default: + throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT); + } + + if (!rows_added) + return {}; + + Block res = result_sample_block.cloneEmpty(); + for (size_t i = 0; i < num_columns_left; ++i) + res.getByPosition(column_indices_left[i]).column = std::move(columns_left[i]); + for (size_t i = 0; i < num_columns_right; ++i) + res.getByPosition(column_indices_right[i]).column = std::move(columns_right[i]); + + return res; +} + + +template +size_t NonJoinedBlockInputStream::fillColumns(const Map & map, + size_t num_columns_left, + MutableColumns & mutable_columns_left, + size_t num_columns_right, + MutableColumns & mutable_columns_right) +{ + size_t rows_added = 0; + size_t key_num = parent.key_names_right.size(); + while (current_not_mapped_row != nullptr) + { + rows_added++; + for (size_t j = 0; j < num_columns_left; ++j) + mutable_columns_left[j]->insertDefault(); + + for (size_t j = 0; j < num_columns_right; ++j) + mutable_columns_right[j]->insertFrom(*current_not_mapped_row->block->getByPosition(key_num + j).column.get(), + current_not_mapped_row->row_num); + + current_not_mapped_row = current_not_mapped_row->next; + setNextCurrentNotMappedRow(); + if (rows_added == max_block_size) + { + return rows_added; + } + } + + if (!position) + { + current_segment = index; + position = decltype(position)( + static_cast(new typename Map::SegmentType::HashTable::const_iterator(map.getSegmentTable(current_segment).begin())), + [](void * ptr) { delete reinterpret_cast(ptr); }); + } + + /// use pointer instead of reference because `it` need to be re-assigned latter + auto it = reinterpret_cast(position.get()); + auto end = map.getSegmentTable(current_segment).end(); + + for (; *it != end || current_segment < map.getSegmentSize() - step; ++(*it)) + { + if (*it == end) + { + // move to next internal hash table + do + { + current_segment += step; + position = decltype(position)( + static_cast(new typename Map::SegmentType::HashTable::const_iterator( + map.getSegmentTable(current_segment).begin())), + [](void * ptr) { delete reinterpret_cast(ptr); }); + it = reinterpret_cast(position.get()); + end = map.getSegmentTable(current_segment).end(); + } while (*it == end && current_segment < map.getSegmentSize() - step); + if (*it == end) + break; + } + if ((*it)->getMapped().getUsed()) + continue; + + rows_added += AdderNonJoined::add((*it)->getMapped(), key_num, num_columns_left, mutable_columns_left, num_columns_right, mutable_columns_right); + + if (rows_added >= max_block_size) + { + ++(*it); + break; + } + } + return rows_added; +} +} // namespace DB diff --git a/dbms/src/DataStreams/NonJoinedBlockInputStream.h b/dbms/src/DataStreams/NonJoinedBlockInputStream.h new file mode 100644 index 00000000000..bb254437c84 --- /dev/null +++ b/dbms/src/DataStreams/NonJoinedBlockInputStream.h @@ -0,0 +1,74 @@ +// 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. + +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Stream from not joined earlier rows of the right table. +class NonJoinedBlockInputStream : public IProfilingBlockInputStream +{ +public: + NonJoinedBlockInputStream(const Join & parent_, const Block & left_sample_block, size_t index_, size_t step_, size_t max_block_size_); + + String getName() const override { return "NonJoined"; } + + Block getHeader() const override { return result_sample_block; }; + + +protected: + Block readImpl() override; + +private: + const Join & parent; + size_t index; + size_t step; + size_t max_block_size; + bool add_not_mapped_rows; + size_t next_index; + + Block result_sample_block; + /// Indices of columns in result_sample_block that come from the left-side table (except key columns). + ColumnNumbers column_indices_left; + /// Indices of columns that come from the right-side table. + /// Order is significant: it is the same as the order of columns in the blocks of the right-side table that are saved in parent.blocks. + ColumnNumbers column_indices_right; + /// Columns of the current output block corresponding to column_indices_left. + MutableColumns columns_left; + /// Columns of the current output block corresponding to column_indices_right. + MutableColumns columns_right; + + std::unique_ptr> position; /// type erasure + size_t current_segment = 0; + Join::RowRefList * current_not_mapped_row = nullptr; + + void setNextCurrentNotMappedRow(); + + template + Block createBlock(const Maps & maps); + + + template + size_t fillColumns(const Map & map, + size_t num_columns_left, + MutableColumns & mutable_columns_left, + size_t num_columns_right, + MutableColumns & mutable_columns_right); +}; +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index fc4043d5006..15d94e0c195 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -84,13 +85,6 @@ bool isLeftSemiFamily(ASTTableJoin::Kind kind) || kind == ASTTableJoin::Kind::Cross_LeftSemi || kind == ASTTableJoin::Kind::Cross_LeftAnti; } -void convertColumnToNullable(ColumnWithTypeAndName & column) -{ - column.type = makeNullable(column.type); - if (column.column) - column.column = makeNullable(column.column); -} - ColumnRawPtrs getKeyColumns(const Names & key_names, const Block & block) { size_t keys_size = key_names.size(); @@ -112,6 +106,12 @@ ColumnRawPtrs getKeyColumns(const Names & key_names, const Block & block) const std::string Join::match_helper_prefix = "__left-semi-join-match-helper"; const DataTypePtr Join::match_helper_type = makeNullable(std::make_shared()); +void convertColumnToNullable(ColumnWithTypeAndName & column) +{ + column.type = makeNullable(column.type); + if (column.column) + column.column = makeNullable(column.column); +} Join::Join( const Names & key_names_left_, @@ -2117,276 +2117,6 @@ void Join::joinTotals(Block & block) const } } -template -struct AdderNonJoined; - -template -struct AdderNonJoined -{ - static size_t add(const Mapped & mapped, size_t key_num, size_t num_columns_left, MutableColumns & columns_left, size_t num_columns_right, MutableColumns & columns_right) - { - for (size_t j = 0; j < num_columns_left; ++j) - columns_left[j]->insertDefault(); - - for (size_t j = 0; j < num_columns_right; ++j) - columns_right[j]->insertFrom(*mapped.block->getByPosition(key_num + j).column.get(), mapped.row_num); - return 1; - } -}; - -template -struct AdderNonJoined -{ - static size_t add(const Mapped & mapped, size_t key_num, size_t num_columns_left, MutableColumns & columns_left, size_t num_columns_right, MutableColumns & columns_right) - { - size_t rows_added = 0; - for (auto current = &static_cast(mapped); current != nullptr; current = current->next) - { - for (size_t j = 0; j < num_columns_left; ++j) - columns_left[j]->insertDefault(); - - for (size_t j = 0; j < num_columns_right; ++j) - columns_right[j]->insertFrom(*current->block->getByPosition(key_num + j).column.get(), current->row_num); - rows_added++; - } - return rows_added; - } -}; - - -/// Stream from not joined earlier rows of the right table. -class NonJoinedBlockInputStream : public IProfilingBlockInputStream -{ -public: - NonJoinedBlockInputStream(const Join & parent_, const Block & left_sample_block, size_t index_, size_t step_, size_t max_block_size_) - : parent(parent_) - , index(index_) - , step(step_) - , max_block_size(max_block_size_) - , add_not_mapped_rows(true) - { - size_t build_concurrency = parent.getBuildConcurrency(); - if (unlikely(step > build_concurrency || index >= build_concurrency)) - throw Exception("The concurrency of NonJoinedBlockInputStream should not be larger than join build concurrency"); - - /** left_sample_block contains keys and "left" columns. - * result_sample_block - keys, "left" columns, and "right" columns. - */ - - size_t num_columns_left = left_sample_block.columns(); - size_t num_columns_right = parent.sample_block_with_columns_to_add.columns(); - - result_sample_block = materializeBlock(left_sample_block); - - /// Add columns from the right-side table to the block. - for (size_t i = 0; i < num_columns_right; ++i) - { - const ColumnWithTypeAndName & src_column = parent.sample_block_with_columns_to_add.getByPosition(i); - result_sample_block.insert(src_column.cloneEmpty()); - } - - column_indices_left.reserve(num_columns_left); - column_indices_right.reserve(num_columns_right); - BoolVec is_key_column_in_left_block(num_columns_left, false); - - for (size_t i = 0; i < num_columns_left; ++i) - { - column_indices_left.push_back(i); - } - - for (size_t i = 0; i < num_columns_right; ++i) - column_indices_right.push_back(num_columns_left + i); - - /// If use_nulls, convert left columns to Nullable. - if (parent.use_nulls) - { - for (size_t i = 0; i < num_columns_left; ++i) - { - convertColumnToNullable(result_sample_block.getByPosition(column_indices_left[i])); - } - } - - columns_left.resize(num_columns_left); - columns_right.resize(num_columns_right); - next_index = index; - } - - String getName() const override { return "NonJoined"; } - - Block getHeader() const override { return result_sample_block; }; - - -protected: - Block readImpl() override - { - if (parent.blocks.empty()) - return Block(); - - if (add_not_mapped_rows) - { - setNextCurrentNotMappedRow(); - add_not_mapped_rows = false; - } - - if (parent.strictness == ASTTableJoin::Strictness::Any) - return createBlock(parent.maps_any_full); - else if (parent.strictness == ASTTableJoin::Strictness::All) - return createBlock(parent.maps_all_full); - else - throw Exception("Logical error: unknown JOIN strictness (must be ANY or ALL)", ErrorCodes::LOGICAL_ERROR); - } - -private: - const Join & parent; - size_t index; - size_t step; - size_t max_block_size; - bool add_not_mapped_rows; - size_t next_index; - - Block result_sample_block; - /// Indices of columns in result_sample_block that come from the left-side table (except key columns). - ColumnNumbers column_indices_left; - /// Indices of columns that come from the right-side table. - /// Order is significant: it is the same as the order of columns in the blocks of the right-side table that are saved in parent.blocks. - ColumnNumbers column_indices_right; - /// Columns of the current output block corresponding to column_indices_left. - MutableColumns columns_left; - /// Columns of the current output block corresponding to column_indices_right. - MutableColumns columns_right; - - std::unique_ptr> position; /// type erasure - size_t current_segment = 0; - Join::RowRefList * current_not_mapped_row = nullptr; - - void setNextCurrentNotMappedRow() - { - while (current_not_mapped_row == nullptr && next_index < parent.rows_not_inserted_to_map.size()) - { - current_not_mapped_row = parent.rows_not_inserted_to_map[next_index]->next; - next_index += step; - } - } - - template - Block createBlock(const Maps & maps) - { - size_t num_columns_left = column_indices_left.size(); - size_t num_columns_right = column_indices_right.size(); - - for (size_t i = 0; i < num_columns_left; ++i) - { - const auto & src_col = result_sample_block.safeGetByPosition(column_indices_left[i]); - columns_left[i] = src_col.type->createColumn(); - } - - for (size_t i = 0; i < num_columns_right; ++i) - { - const auto & src_col = result_sample_block.safeGetByPosition(column_indices_right[i]); - columns_right[i] = src_col.type->createColumn(); - } - - size_t rows_added = 0; - - switch (parent.type) - { -#define M(TYPE) \ - case Join::Type::TYPE: \ - rows_added = fillColumns(*maps.TYPE, num_columns_left, columns_left, num_columns_right, columns_right); \ - break; - APPLY_FOR_JOIN_VARIANTS(M) -#undef M - - default: - throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT); - } - - if (!rows_added) - return {}; - - Block res = result_sample_block.cloneEmpty(); - for (size_t i = 0; i < num_columns_left; ++i) - res.getByPosition(column_indices_left[i]).column = std::move(columns_left[i]); - for (size_t i = 0; i < num_columns_right; ++i) - res.getByPosition(column_indices_right[i]).column = std::move(columns_right[i]); - - return res; - } - - - template - size_t fillColumns(const Map & map, - size_t num_columns_left, - MutableColumns & mutable_columns_left, - size_t num_columns_right, - MutableColumns & mutable_columns_right) - { - size_t rows_added = 0; - size_t key_num = parent.key_names_right.size(); - while (current_not_mapped_row != nullptr) - { - rows_added++; - for (size_t j = 0; j < num_columns_left; ++j) - mutable_columns_left[j]->insertDefault(); - - for (size_t j = 0; j < num_columns_right; ++j) - mutable_columns_right[j]->insertFrom(*current_not_mapped_row->block->getByPosition(key_num + j).column.get(), - current_not_mapped_row->row_num); - - current_not_mapped_row = current_not_mapped_row->next; - setNextCurrentNotMappedRow(); - if (rows_added == max_block_size) - { - return rows_added; - } - } - - if (!position) - { - current_segment = index; - position = decltype(position)( - static_cast(new typename Map::SegmentType::HashTable::const_iterator(map.getSegmentTable(current_segment).begin())), - [](void * ptr) { delete reinterpret_cast(ptr); }); - } - - /// use pointer instead of reference because `it` need to be re-assigned latter - auto it = reinterpret_cast(position.get()); - auto end = map.getSegmentTable(current_segment).end(); - - for (; *it != end || current_segment < map.getSegmentSize() - step; ++(*it)) - { - if (*it == end) - { - // move to next internal hash table - do - { - current_segment += step; - position = decltype(position)( - static_cast(new typename Map::SegmentType::HashTable::const_iterator( - map.getSegmentTable(current_segment).begin())), - [](void * ptr) { delete reinterpret_cast(ptr); }); - it = reinterpret_cast(position.get()); - end = map.getSegmentTable(current_segment).end(); - } while (*it == end && current_segment < map.getSegmentSize() - step); - if (*it == end) - break; - } - if ((*it)->getMapped().getUsed()) - continue; - - rows_added += AdderNonJoined::add((*it)->getMapped(), key_num, num_columns_left, mutable_columns_left, num_columns_right, mutable_columns_right); - - if (rows_added >= max_block_size) - { - ++(*it); - break; - } - } - return rows_added; - } -}; - - BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sample_block, size_t index, size_t step, size_t max_block_size) const { return std::make_shared(*this, left_sample_block, index, step, max_block_size); diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 0565e0fe79e..ae86f78c1f4 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -423,4 +423,6 @@ struct ProbeProcessInfo void updateStartRow(); }; +void convertColumnToNullable(ColumnWithTypeAndName & column); + } // namespace DB From f0307c58b82a4f51b22e5163d97742fc822428e2 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 19 Dec 2022 17:06:54 +0800 Subject: [PATCH 24/42] fix typo in config-template.toml (#6499) close pingcap/tiflash#6500 --- etc/config-template.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/etc/config-template.toml b/etc/config-template.toml index fb45d426fb7..90046404231 100644 --- a/etc/config-template.toml +++ b/etc/config-template.toml @@ -107,7 +107,7 @@ # read_index_runner_count = 1 ## The minimum duration to handle read-index tasks in each worker. # read_index_worker_tick_ms = 10 -# disaggregate_mode = "tiflash_storage" or "tiflash_compute" +# disaggregated_mode = "tiflash_storage" or "tiflash_compute" [flash.proxy] # addr = "0.0.0.0:20170" From f9696e378d4fb103ff8ca26302b4ec76bf18dd55 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Mon, 19 Dec 2022 18:10:54 +0800 Subject: [PATCH 25/42] *: remove redundant log (#6501) ref pingcap/tiflash#5900 --- dbms/src/Common/TiFlashSecurity.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Common/TiFlashSecurity.h b/dbms/src/Common/TiFlashSecurity.h index 3a0f3fc4f96..5985d1df509 100644 --- a/dbms/src/Common/TiFlashSecurity.h +++ b/dbms/src/Common/TiFlashSecurity.h @@ -197,7 +197,6 @@ class TiFlashSecurityConfig : public ConfigObject new_files.addIfExists(file.path); bool updated = new_files.isDifferOrNewerThan(cert_files); - LOG_DEBUG(log, "Ssl certificate files updated: {}", updated); return updated; } From 34d7fcd398df5c1e418ba4f68217c0cad51cb39c Mon Sep 17 00:00:00 2001 From: JaySon Date: Tue, 20 Dec 2022 00:42:54 +0800 Subject: [PATCH 26/42] *: .clang-format ignores protobuf files (#6502) ref pingcap/tiflash#6233 --- .clang-format | 4 ++++ dbms/src/Flash/FlashService.h | 3 +-- dbms/src/Interpreters/Context.cpp | 1 + dbms/src/Interpreters/Context.h | 1 - dbms/src/Server/CertificateReloader.cpp | 1 + dbms/src/Server/CertificateReloader.h | 1 - dbms/src/Server/MetricsPrometheus.cpp | 1 + dbms/src/Server/MetricsPrometheus.h | 1 - .../DeltaMerge/DeltaMergeStore_Statistics.cpp | 1 - dbms/src/Storages/StorageDisaggregated.cpp | 24 ++++++++++++------- 10 files changed, 24 insertions(+), 14 deletions(-) diff --git a/.clang-format b/.clang-format index 740c996f77f..e7a353b4f58 100644 --- a/.clang-format +++ b/.clang-format @@ -89,4 +89,8 @@ SpacesInContainerLiterals: true SpacesInCStyleCastParentheses: false SpacesInParentheses: false SpacesInSquareBrackets: false +--- +Language: Proto +# Added to avoid annoying error message +DisableFormat: true ... diff --git a/dbms/src/Flash/FlashService.h b/dbms/src/Flash/FlashService.h index 99e1ff66c53..9223060db37 100644 --- a/dbms/src/Flash/FlashService.h +++ b/dbms/src/Flash/FlashService.h @@ -14,7 +14,6 @@ #pragma once -#include #include #include #include @@ -26,8 +25,8 @@ #ifdef __clang__ #pragma GCC diagnostic ignored "-Wdeprecated-declarations" #endif +#include #include - #pragma GCC diagnostic pop namespace DB diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 93ddd06b931..868de593933 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 65fb2b91908..9625bff7dc1 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -14,7 +14,6 @@ #pragma once -#include #include #include #include diff --git a/dbms/src/Server/CertificateReloader.cpp b/dbms/src/Server/CertificateReloader.cpp index 1ebb14f21a9..2cbe64401a2 100644 --- a/dbms/src/Server/CertificateReloader.cpp +++ b/dbms/src/Server/CertificateReloader.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #if Poco_NetSSL_FOUND diff --git a/dbms/src/Server/CertificateReloader.h b/dbms/src/Server/CertificateReloader.h index 3c6f3d2714e..bda3ee61b1a 100644 --- a/dbms/src/Server/CertificateReloader.h +++ b/dbms/src/Server/CertificateReloader.h @@ -19,7 +19,6 @@ #if Poco_NetSSL_FOUND #include -#include #include #include #include diff --git a/dbms/src/Server/MetricsPrometheus.cpp b/dbms/src/Server/MetricsPrometheus.cpp index abe99333e42..09a8542c9ee 100644 --- a/dbms/src/Server/MetricsPrometheus.cpp +++ b/dbms/src/Server/MetricsPrometheus.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Server/MetricsPrometheus.h b/dbms/src/Server/MetricsPrometheus.h index 4ef84f6f142..f6650173031 100644 --- a/dbms/src/Server/MetricsPrometheus.h +++ b/dbms/src/Server/MetricsPrometheus.h @@ -14,7 +14,6 @@ #pragma once -#include #include #include #include diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp index e794913152c..97c9ac250f0 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -15,7 +15,6 @@ #include #include #include -#include namespace DB { diff --git a/dbms/src/Storages/StorageDisaggregated.cpp b/dbms/src/Storages/StorageDisaggregated.cpp index 652032b8dc5..d2c6880db45 100644 --- a/dbms/src/Storages/StorageDisaggregated.cpp +++ b/dbms/src/Storages/StorageDisaggregated.cpp @@ -85,7 +85,14 @@ std::vector StorageDisaggregated::buildBatch pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); pingcap::kv::Backoffer bo(pingcap::kv::copBuildTaskMaxBackoff); pingcap::kv::StoreType store_type = pingcap::kv::StoreType::TiFlash; - auto batch_cop_tasks = pingcap::coprocessor::buildBatchCopTasks(bo, cluster, table_scan.isPartitionTableScan(), physical_table_ids, ranges_for_each_physical_table, store_type, &Poco::Logger::get("pingcap/coprocessor")); + auto batch_cop_tasks = pingcap::coprocessor::buildBatchCopTasks( + bo, + cluster, + table_scan.isPartitionTableScan(), + physical_table_ids, + ranges_for_each_physical_table, + store_type, + &Poco::Logger::get("pingcap/coprocessor")); LOG_DEBUG(log, "batch cop tasks(nums: {}) build finish for tiflash_storage node", batch_cop_tasks.size()); return batch_cop_tasks; } @@ -232,7 +239,7 @@ void StorageDisaggregated::buildReceiverStreams(const std::vectorgetMPPTaskMeta(); - const String executor_id = table_scan.getTableScanExecutorID(); + const String & executor_id = table_scan.getTableScanExecutorID(); exchange_receiver = std::make_shared( std::make_shared( @@ -257,16 +264,17 @@ void StorageDisaggregated::buildReceiverStreams(const std::vector(exchange_receiver, - log->identifier(), - executor_id, - /*stream_id=*/0); + BlockInputStreamPtr stream = std::make_shared( + exchange_receiver, + log->identifier(), + executor_id, + /*stream_id=*/0); stream->setExtraInfo(extra_info); pipeline.streams.push_back(stream); } - auto & table_scan_io_input_streams = context.getDAGContext()->getInBoundIOInputStreamsMap()[table_scan.getTableScanExecutorID()]; - auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[table_scan.getTableScanExecutorID()]; + auto & table_scan_io_input_streams = context.getDAGContext()->getInBoundIOInputStreamsMap()[executor_id]; + auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[executor_id]; pipeline.transform([&](auto & stream) { table_scan_io_input_streams.push_back(stream); profile_streams.push_back(stream); From d198b7f31843183bc0f892c24171429285f9e8ce Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 20 Dec 2022 11:26:54 +0800 Subject: [PATCH 27/42] Avoid the occurrence of 0 physical cpu cores (#6505) close pingcap/tiflash#6504 --- dbms/src/Common/getNumberOfCPUCores.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/dbms/src/Common/getNumberOfCPUCores.cpp b/dbms/src/Common/getNumberOfCPUCores.cpp index c3d3264c627..fd615d6a471 100644 --- a/dbms/src/Common/getNumberOfCPUCores.cpp +++ b/dbms/src/Common/getNumberOfCPUCores.cpp @@ -12,7 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include +#include #include namespace CPUCores @@ -58,5 +60,15 @@ void computeAndSetNumberOfPhysicalCPUCores(UInt16 number_of_logical_cpu_cores_, // - `(hardware_logical_cpu_cores / number_of_hardware_physical_cores)` means how many logical cpu core a physical cpu core has. // - `number_of_logical_cpu_cores_ / (hardware_logical_cpu_cores / number_of_hardware_physical_cores)` means how many physical cpu cores the tiflash process could use. (Actually, it's needless to get physical cpu cores in virtual environment, but we must ensure the behavior `1` is not broken) auto hardware_logical_cpu_cores = std::thread::hardware_concurrency(); - CPUCores::number_of_physical_cpu_cores = number_of_logical_cpu_cores_ / (hardware_logical_cpu_cores / number_of_hardware_physical_cores); + UInt16 physical_cpu_cores = number_of_logical_cpu_cores_ / (hardware_logical_cpu_cores / number_of_hardware_physical_cores); + CPUCores::number_of_physical_cpu_cores = physical_cpu_cores > 0 ? physical_cpu_cores : 1; + auto log = DB::Logger::get("CPUCores"); + LOG_INFO( + log, + "logical cpu cores: {}, hardware logical cpu cores: {}, hardware physical cpu cores: {}, physical cpu cores: {}, number_of_physical_cpu_cores: {}", + number_of_logical_cpu_cores_, + hardware_logical_cpu_cores, + number_of_hardware_physical_cores, + physical_cpu_cores, + CPUCores::number_of_physical_cpu_cores); } From cadb7cd4362fa6a7a4b554f5497b2998cfb39aa8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Tue, 20 Dec 2022 12:10:54 +0800 Subject: [PATCH 28/42] .*: Refine `FilterBlockInputStream` and `FilterTransformAction` (#6503) ref pingcap/tiflash#6233 --- .../DataStreams/FilterBlockInputStream.cpp | 5 +++-- .../src/DataStreams/FilterTransformAction.cpp | 19 ++++++++++++++++--- dbms/src/DataStreams/FilterTransformAction.h | 6 +++++- 3 files changed, 24 insertions(+), 6 deletions(-) diff --git a/dbms/src/DataStreams/FilterBlockInputStream.cpp b/dbms/src/DataStreams/FilterBlockInputStream.cpp index dec1dc18fca..9d507e3323f 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.cpp +++ b/dbms/src/DataStreams/FilterBlockInputStream.cpp @@ -67,12 +67,13 @@ Block FilterBlockInputStream::readImpl() /// Until non-empty block after filtering or end of stream. while (true) { - res = children.back()->read(); + FilterPtr child_filter = nullptr; + res = children.back()->read(child_filter, true); if (!res) return res; - if (filter_transform_action.transform(res)) + if (filter_transform_action.transform(res, child_filter)) return res; } } diff --git a/dbms/src/DataStreams/FilterTransformAction.cpp b/dbms/src/DataStreams/FilterTransformAction.cpp index e2017535aca..98bf6718e45 100644 --- a/dbms/src/DataStreams/FilterTransformAction.cpp +++ b/dbms/src/DataStreams/FilterTransformAction.cpp @@ -19,6 +19,8 @@ #include #include +#include + namespace DB { namespace ErrorCodes @@ -68,20 +70,22 @@ ExpressionActionsPtr FilterTransformAction::getExperssion() const return expression; } -bool FilterTransformAction::transform(Block & block) +bool FilterTransformAction::transform(Block & block, FilterPtr child_filter) { if (unlikely(!block)) return true; expression->execute(block); - if (constant_filter_description.always_true) + if (constant_filter_description.always_true && !child_filter) return true; size_t columns = block.columns(); size_t rows = block.rows(); ColumnPtr column_of_filter = block.safeGetByPosition(filter_column).column; + RUNTIME_CHECK_MSG(!child_filter || child_filter->size() == rows, "Unexpected child filter size"); + /** It happens that at the stage of analysis of expressions (in sample_block) the columns-constants have not been calculated yet, * and now - are calculated. That is, not all cases are covered by the code above. * This happens if the function returns a constant for a non-constant argument. @@ -100,13 +104,22 @@ bool FilterTransformAction::transform(Block & block) if (constant_filter_description.always_true) { - return true; + if (child_filter) + filter = child_filter; + else + return true; } else { FilterDescription filter_and_holder(*column_of_filter); filter = const_cast(filter_and_holder.data); filter_holder = filter_and_holder.data_holder; + + if (child_filter) + { + /// Merge child_filter + std::transform(filter->cbegin(), filter->cend(), child_filter->cbegin(), filter->begin(), [](const UInt8 a, const UInt8 b) { return a > 0 && b != 0; }); + } } /** Let's find out how many rows will be in result. diff --git a/dbms/src/DataStreams/FilterTransformAction.h b/dbms/src/DataStreams/FilterTransformAction.h index a4cecd10afa..dd32133561c 100644 --- a/dbms/src/DataStreams/FilterTransformAction.h +++ b/dbms/src/DataStreams/FilterTransformAction.h @@ -19,6 +19,9 @@ namespace DB { + +using FilterPtr = IColumn::Filter *; + struct FilterTransformAction { public: @@ -29,7 +32,7 @@ struct FilterTransformAction bool alwaysFalse() const; // return false if all filter out. - bool transform(Block & block); + bool transform(Block & block, FilterPtr child_filter); Block getHeader() const; ExpressionActionsPtr getExperssion() const; @@ -40,4 +43,5 @@ struct FilterTransformAction ConstantFilterDescription constant_filter_description; }; + } // namespace DB From 972769ce353b7e259789f6c369982ad69038d0f6 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Tue, 20 Dec 2022 18:06:55 +0800 Subject: [PATCH 29/42] refine SegmentRead streams (#6274) ref pingcap/tiflash#5900 --- .../SegmentReadTransformAction.cpp | 45 ++++++++++++++++++ .../DataStreams/SegmentReadTransformAction.h | 47 +++++++++++++++++++ .../DeltaMerge/DMSegmentThreadInputStream.h | 25 +++------- .../ReadThread/UnorderedInputStream.h | 33 +++++-------- 4 files changed, 110 insertions(+), 40 deletions(-) create mode 100644 dbms/src/DataStreams/SegmentReadTransformAction.cpp create mode 100644 dbms/src/DataStreams/SegmentReadTransformAction.h diff --git a/dbms/src/DataStreams/SegmentReadTransformAction.cpp b/dbms/src/DataStreams/SegmentReadTransformAction.cpp new file mode 100644 index 00000000000..e1f34ab7266 --- /dev/null +++ b/dbms/src/DataStreams/SegmentReadTransformAction.cpp @@ -0,0 +1,45 @@ +// 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 + +namespace DB +{ +Block SegmentReadTransformAction::getHeader() const +{ + return header; +} + +bool SegmentReadTransformAction::transform(Block & block) +{ + if (extra_table_id_index != InvalidColumnID) + { + const auto & extra_table_id_col_define = DM::getExtraTableIDColumnDefine(); + ColumnWithTypeAndName col{{}, extra_table_id_col_define.type, extra_table_id_col_define.name, extra_table_id_col_define.id}; + size_t row_number = block.rows(); + auto col_data = col.type->createColumnConst(row_number, Field(physical_table_id)); + col.column = std::move(col_data); + block.insert(extra_table_id_index, std::move(col)); + } + if (!block.rows()) + { + return false; + } + else + { + total_rows += block.rows(); + return true; + } +} +} // namespace DB diff --git a/dbms/src/DataStreams/SegmentReadTransformAction.h b/dbms/src/DataStreams/SegmentReadTransformAction.h new file mode 100644 index 00000000000..7512ebda725 --- /dev/null +++ b/dbms/src/DataStreams/SegmentReadTransformAction.h @@ -0,0 +1,47 @@ +// 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. +#pragma once + +#include +#include + +namespace DB +{ +struct SegmentReadTransformAction +{ +public: + SegmentReadTransformAction( + const Block & header_, + int extra_table_id_index_, + TableID physical_table_id_) + : header(header_) + , extra_table_id_index(extra_table_id_index_) + , physical_table_id(physical_table_id_) + { + } + bool transform(Block & block); + Block getHeader() const; + size_t totalRows() const + { + return total_rows; + } + +private: + Block header; + // position of the ExtraPhysTblID column in column_names parameter in the StorageDeltaMerge::read function. + const int extra_table_id_index; + const TableID physical_table_id; + size_t total_rows = 0; +}; +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DMSegmentThreadInputStream.h b/dbms/src/Storages/DeltaMerge/DMSegmentThreadInputStream.h index 7326600dfaf..6cf66ca0bf2 100644 --- a/dbms/src/Storages/DeltaMerge/DMSegmentThreadInputStream.h +++ b/dbms/src/Storages/DeltaMerge/DMSegmentThreadInputStream.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -60,8 +61,7 @@ class DMSegmentThreadInputStream : public IProfilingBlockInputStream , max_version(max_version_) , expected_block_size(expected_block_size_) , read_mode(read_mode_) - , extra_table_id_index(extra_table_id_index) - , physical_table_id(physical_table_id) + , action(header, extra_table_id_index, physical_table_id) , log(Logger::get(req_id)) { if (extra_table_id_index != InvalidColumnID) @@ -110,21 +110,13 @@ class DMSegmentThreadInputStream : public IProfilingBlockInputStream if (res) { - if (extra_table_id_index != InvalidColumnID) + if (action.transform(res)) { - ColumnDefine extra_table_id_col_define = getExtraTableIDColumnDefine(); - ColumnWithTypeAndName col{{}, extra_table_id_col_define.type, extra_table_id_col_define.name, extra_table_id_col_define.id}; - size_t row_number = res.rows(); - auto col_data = col.type->createColumnConst(row_number, Field(physical_table_id)); - col.column = std::move(col_data); - res.insert(extra_table_id_index, std::move(col)); + return res; } - if (!res.rows()) - continue; else { - total_rows += res.rows(); - return res; + continue; } } else @@ -139,7 +131,7 @@ class DMSegmentThreadInputStream : public IProfilingBlockInputStream void readSuffixImpl() override { - LOG_DEBUG(log, "finish read {} rows from storage", total_rows); + LOG_DEBUG(log, "finish read {} rows from storage", action.totalRows()); } private: @@ -152,18 +144,15 @@ class DMSegmentThreadInputStream : public IProfilingBlockInputStream const UInt64 max_version; const size_t expected_block_size; const ReadMode read_mode; - // position of the ExtraPhysTblID column in column_names parameter in the StorageDeltaMerge::read function. - const int extra_table_id_index; bool done = false; BlockInputStreamPtr cur_stream; SegmentPtr cur_segment; - TableID physical_table_id; + SegmentReadTransformAction action; LoggerPtr log; - size_t total_rows = 0; }; } // namespace DM diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/UnorderedInputStream.h b/dbms/src/Storages/DeltaMerge/ReadThread/UnorderedInputStream.h index f53d2e78b3c..4fe729578de 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/UnorderedInputStream.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/UnorderedInputStream.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -39,15 +40,15 @@ class UnorderedInputStream : public IProfilingBlockInputStream const String & req_id) : task_pool(task_pool_) , header(toEmptyBlock(columns_to_read_)) - , extra_table_id_index(extra_table_id_index) - , physical_table_id(physical_table_id) + , action(header, extra_table_id_index, physical_table_id) , log(Logger::get(req_id)) , ref_no(0) , task_pool_added(false) + { if (extra_table_id_index != InvalidColumnID) { - auto & extra_table_id_col_define = getExtraTableIDColumnDefine(); + const auto & extra_table_id_col_define = getExtraTableIDColumnDefine(); ColumnWithTypeAndName col{extra_table_id_col_define.type->createColumn(), extra_table_id_col_define.type, extra_table_id_col_define.name, extra_table_id_col_define.id, extra_table_id_col_define.default_value}; header.insert(extra_table_id_index, col); } @@ -55,7 +56,7 @@ class UnorderedInputStream : public IProfilingBlockInputStream LOG_DEBUG(log, "Created, pool_id={} ref_no={}", task_pool->poolId(), ref_no); } - ~UnorderedInputStream() + ~UnorderedInputStream() override { task_pool->decreaseUnorderedInputStreamRefCount(); LOG_DEBUG(log, "Destroy, pool_id={} ref_no={}", task_pool->poolId(), ref_no); @@ -87,23 +88,13 @@ class UnorderedInputStream : public IProfilingBlockInputStream task_pool->popBlock(res); if (res) { - if (extra_table_id_index != InvalidColumnID) - { - auto & extra_table_id_col_define = getExtraTableIDColumnDefine(); - ColumnWithTypeAndName col{{}, extra_table_id_col_define.type, extra_table_id_col_define.name, extra_table_id_col_define.id}; - size_t row_number = res.rows(); - auto col_data = col.type->createColumnConst(row_number, Field(physical_table_id)); - col.column = std::move(col_data); - res.insert(extra_table_id_index, std::move(col)); - } - if (!res.rows()) + if (action.transform(res)) { - continue; + return res; } else { - total_rows += res.rows(); - return res; + continue; } } else @@ -116,7 +107,7 @@ class UnorderedInputStream : public IProfilingBlockInputStream void readSuffixImpl() override { - LOG_DEBUG(log, "Finish read from storage, pool_id={} ref_no={} rows={}", task_pool->poolId(), ref_no, total_rows); + LOG_DEBUG(log, "Finish read from storage, pool_id={} ref_no={} rows={}", task_pool->poolId(), ref_no, action.totalRows()); } void addReadTaskPoolToScheduler() @@ -132,13 +123,11 @@ class UnorderedInputStream : public IProfilingBlockInputStream private: SegmentReadTaskPoolPtr task_pool; Block header; - // position of the ExtraPhysTblID column in column_names parameter in the StorageDeltaMerge::read function. - const int extra_table_id_index; + SegmentReadTransformAction action; + bool done = false; - TableID physical_table_id; LoggerPtr log; int64_t ref_no; - size_t total_rows = 0; bool task_pool_added; }; } // namespace DB::DM From e88ddaa9b5a2a69c83345370d18e57d2f4a9e1e8 Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 21 Dec 2022 11:16:55 +0800 Subject: [PATCH 30/42] Optimize deltamerge store restore speed (#6420) close pingcap/tiflash#6395 --- dbms/src/DataTypes/DataTypeFactory.cpp | 44 ++++++++++++++++--- dbms/src/DataTypes/DataTypeFactory.h | 12 ++++- dbms/src/Server/Server.cpp | 5 ++- .../ColumnFile/ColumnFilePersisted.cpp | 2 +- .../DeltaMerge/ColumnFile/ColumnFile_V3.cpp | 1 + dbms/src/Storages/DeltaMerge/ColumnStat.h | 4 +- .../tests/gtest_dm_storage_delta_merge.cpp | 2 +- 7 files changed, 58 insertions(+), 12 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index 30a5813602b..7fe8e3d357c 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -40,22 +40,56 @@ DataTypePtr DataTypeFactory::get(const String & full_name) const ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0); return get(ast); } +// DataTypeFactory is a Singleton, so need to be protected by lock. +DataTypePtr DataTypeFactory::getOrSet(const String & full_name) +{ + { + std::shared_lock lock(rw_lock); + auto it = fullname_types.find(full_name); + if (it != fullname_types.end()) + { + return it->second; + } + } + ParserIdentifierWithOptionalParameters parser; + ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0); + DataTypePtr datatype_ptr = get(ast); + // avoid big hashmap in rare cases. + std::unique_lock lock(rw_lock); + if (fullname_types.size() < MAX_FULLNAME_TYPES) + { + // DataTypeEnum may generate too many full_name, so just skip inserting DataTypeEnum into fullname_types when + // the capacity limit is almost reached, which ensures that most datatypes can be cached. + if (fullname_types.size() > FULLNAME_TYPES_HIGH_WATER_MARK && (datatype_ptr->getTypeId() == TypeIndex::Enum8 || datatype_ptr->getTypeId() == TypeIndex::Enum16)) + { + return datatype_ptr; + } + fullname_types.emplace(full_name, datatype_ptr); + } + return datatype_ptr; +} + +size_t DataTypeFactory::getFullNameCacheSize() const +{ + std::shared_lock lock(rw_lock); + return fullname_types.size(); +} DataTypePtr DataTypeFactory::get(const ASTPtr & ast) const { - if (const ASTFunction * func = typeid_cast(ast.get())) + if (const auto * func = typeid_cast(ast.get())) { if (func->parameters) throw Exception("Data type cannot have multiple parenthesed parameters.", ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE); return get(func->name, func->arguments); } - if (const ASTIdentifier * ident = typeid_cast(ast.get())) + if (const auto * ident = typeid_cast(ast.get())) { return get(ident->name, {}); } - if (const ASTLiteral * lit = typeid_cast(ast.get())) + if (const auto * lit = typeid_cast(ast.get())) { if (lit->value.isNull()) return get("Null", {}); @@ -67,14 +101,14 @@ DataTypePtr DataTypeFactory::get(const ASTPtr & ast) const DataTypePtr DataTypeFactory::get(const String & family_name, const ASTPtr & parameters) const { { - DataTypesDictionary::const_iterator it = data_types.find(family_name); + auto it = data_types.find(family_name); if (data_types.end() != it) return it->second(parameters); } { String family_name_lowercase = Poco::toLower(family_name); - DataTypesDictionary::const_iterator it = case_insensitive_data_types.find(family_name_lowercase); + auto it = case_insensitive_data_types.find(family_name_lowercase); if (case_insensitive_data_types.end() != it) return it->second(parameters); } diff --git a/dbms/src/DataTypes/DataTypeFactory.h b/dbms/src/DataTypes/DataTypeFactory.h index 67a7799721e..a637857fa1e 100644 --- a/dbms/src/DataTypes/DataTypeFactory.h +++ b/dbms/src/DataTypes/DataTypeFactory.h @@ -19,9 +19,9 @@ #include #include #include +#include #include - namespace DB { class IDataType; @@ -38,12 +38,18 @@ class DataTypeFactory final : public ext::Singleton private: using Creator = std::function; using SimpleCreator = std::function; + // family_name -> Creator using DataTypesDictionary = std::unordered_map; + // full_name -> DataTypePtr + using FullnameTypes = std::unordered_map; public: DataTypePtr get(const String & full_name) const; + // In order to optimize the speed of generating data type instances, this will cache the full_name -> DataTypePtr. + DataTypePtr getOrSet(const String & full_name); DataTypePtr get(const String & family_name, const ASTPtr & parameters) const; DataTypePtr get(const ASTPtr & ast) const; + size_t getFullNameCacheSize() const; /// For compatibility with SQL, it's possible to specify that certain data type name is case insensitive. enum CaseSensitiveness @@ -64,6 +70,10 @@ class DataTypeFactory final : public ext::Singleton /// Case insensitive data types will be additionally added here with lowercased name. DataTypesDictionary case_insensitive_data_types; + static constexpr int MAX_FULLNAME_TYPES = 50000; + static constexpr int FULLNAME_TYPES_HIGH_WATER_MARK = 49000; + mutable std::shared_mutex rw_lock; + FullnameTypes fullname_types; DataTypeFactory(); friend class ext::Singleton; }; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index e4893518e23..eab7f811a64 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -504,10 +504,11 @@ void initStores(Context & global_context, const LoggerPtr & log, bool lazily_ini } LOG_INFO( log, - "Storage inited finish. [total_count={}] [init_count={}] [error_count={}]", + "Storage inited finish. [total_count={}] [init_count={}] [error_count={}] [datatype_fullname_count={}]", storages.size(), init_cnt, - err_cnt); + err_cnt, + DataTypeFactory::instance().getFullNameCacheSize()); }; if (lazily_init_store) { diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.cpp index dd80c5f2e6e..31388c909e8 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFilePersisted.cpp @@ -58,7 +58,7 @@ BlockPtr deserializeSchema(ReadBuffer & buf) readIntBinary(column_id, buf); readStringBinary(name, buf); readStringBinary(type_name, buf); - schema->insert(ColumnWithTypeAndName({}, DataTypeFactory::instance().get(type_name), name, column_id)); + schema->insert(ColumnWithTypeAndName({}, DataTypeFactory::instance().getOrSet(type_name), name, column_id)); } return schema; } diff --git a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile_V3.cpp b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile_V3.cpp index 22514b7aa58..dcf063b2fe1 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile_V3.cpp +++ b/dbms/src/Storages/DeltaMerge/ColumnFile/ColumnFile_V3.cpp @@ -66,6 +66,7 @@ ColumnFilePersisteds deserializeSavedColumnFilesInV3Format(DMContext & context, size_t column_file_count; readIntBinary(column_file_count, buf); ColumnFilePersisteds column_files; + column_files.reserve(column_file_count); BlockPtr last_schema; for (size_t i = 0; i < column_file_count; ++i) { diff --git a/dbms/src/Storages/DeltaMerge/ColumnStat.h b/dbms/src/Storages/DeltaMerge/ColumnStat.h index e794608368d..85032ec45c9 100644 --- a/dbms/src/Storages/DeltaMerge/ColumnStat.h +++ b/dbms/src/Storages/DeltaMerge/ColumnStat.h @@ -36,7 +36,7 @@ using ColumnStats = std::unordered_map; inline void readText(ColumnStats & column_sats, DMFileFormat::Version ver, ReadBuffer & buf) { - const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); + DataTypeFactory & data_type_factory = DataTypeFactory::instance(); size_t count; DB::assertString("Columns: ", buf); @@ -61,7 +61,7 @@ inline void readText(ColumnStats & column_sats, DMFileFormat::Version ver, ReadB DB::readString(type_name, buf); DB::assertChar('\n', buf); - auto type = data_type_factory.get(type_name); + auto type = data_type_factory.getOrSet(type_name); column_sats.emplace(id, ColumnStat{id, type, avg_size, serialized_bytes}); } } diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp index 8808b93c92d..1d55de2e319 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp @@ -708,7 +708,7 @@ try auto create_table = [&]() { NamesAndTypesList names_and_types_list{ {"col1", std::make_shared()}, - {"col2", std::make_shared()}, + {"col2", std::make_shared(std::make_shared())}, }; for (const auto & name_type : names_and_types_list) { From cd8e8c14c94bb2c2e8a9205cdf1ab1b50ec8308a Mon Sep 17 00:00:00 2001 From: JaySon Date: Wed, 21 Dec 2022 15:24:55 +0800 Subject: [PATCH 31/42] MPP: Fix bug when building disaggregated regions info of mpp dispatch task (#6512) close pingcap/tiflash#6513 --- .../ExchangeSenderBlockInputStream.cpp | 1 + dbms/src/Debug/dbgFuncCoprocessorUtils.cpp | 1 + dbms/src/Debug/dbgFuncCoprocessorUtils.h | 1 - dbms/src/Debug/dbgQueryExecutor.cpp | 8 +- dbms/src/Flash/BatchCoprocessorHandler.cpp | 12 +- dbms/src/Flash/Coprocessor/DAGContext.h | 12 +- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 1 + .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 1 - dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 1 - dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 1 + dbms/src/Flash/Coprocessor/DAGQuerySource.h | 3 +- .../Flash/Coprocessor/DAGResponseWriter.cpp | 1 + .../src/Flash/Coprocessor/DAGResponseWriter.h | 4 +- .../Flash/Coprocessor/FineGrainedShuffle.h | 1 - dbms/src/Flash/Coprocessor/InterpreterDAG.h | 2 +- dbms/src/Flash/Coprocessor/MockSourceStream.h | 3 +- dbms/src/Flash/Coprocessor/RemoteRequest.cpp | 1 + dbms/src/Flash/Coprocessor/RemoteRequest.h | 1 + dbms/src/Flash/Coprocessor/RequestUtils.h | 78 +++++++++ .../StreamingDAGResponseWriter.cpp | 1 + .../Coprocessor/StreamingDAGResponseWriter.h | 3 +- dbms/src/Flash/Coprocessor/TiDBTableScan.cpp | 1 + dbms/src/Flash/Coprocessor/TiDBTableScan.h | 6 +- .../Coprocessor/UnaryDAGResponseWriter.cpp | 1 + .../Coprocessor/tests/gtest_req_encode.cpp | 150 ++++++++++++++++++ .../tests/gtest_streaming_writer.cpp | 1 + dbms/src/Flash/CoprocessorHandler.cpp | 11 +- .../Mpp/BroadcastOrPassThroughWriter.cpp | 1 + .../Flash/Mpp/BroadcastOrPassThroughWriter.h | 3 +- dbms/src/Flash/Mpp/ExchangeReceiver.h | 1 - .../Flash/Mpp/FineGrainedShuffleWriter.cpp | 1 + dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h | 3 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 1 + dbms/src/Flash/Mpp/HashPartitionWriter.h | 3 +- dbms/src/Flash/Mpp/MPPReceiverSet.h | 1 - .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 1 + dbms/src/Flash/Planner/PlanQuerySource.cpp | 6 + dbms/src/Flash/Planner/PlanQuerySource.h | 5 +- .../plans/PhysicalMockExchangeReceiver.cpp | 1 + .../Planner/plans/PhysicalMockTableScan.cpp | 1 + .../Flash/Statistics/ExchangeSenderImpl.cpp | 3 +- .../src/Flash/Statistics/ExecutorStatistics.h | 4 +- .../tests/gtest_storage_disaggregated.cpp | 1 + .../DeltaMerge/File/DMFilePackFilter.h | 1 - dbms/src/Storages/StorageDisaggregated.cpp | 60 ++----- dbms/src/Storages/StorageDisaggregated.h | 10 +- .../Storages/tests/gtest_filter_parser.cpp | 2 +- dbms/src/TestUtils/ColumnsToTiPBExpr.h | 1 - dbms/src/TestUtils/MPPTaskTestUtils.cpp | 2 +- 49 files changed, 324 insertions(+), 95 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/RequestUtils.h create mode 100644 dbms/src/Flash/Coprocessor/tests/gtest_req_encode.cpp diff --git a/dbms/src/DataStreams/ExchangeSenderBlockInputStream.cpp b/dbms/src/DataStreams/ExchangeSenderBlockInputStream.cpp index 92fd36f881e..0f9dd3ac7d9 100644 --- a/dbms/src/DataStreams/ExchangeSenderBlockInputStream.cpp +++ b/dbms/src/DataStreams/ExchangeSenderBlockInputStream.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { diff --git a/dbms/src/Debug/dbgFuncCoprocessorUtils.cpp b/dbms/src/Debug/dbgFuncCoprocessorUtils.cpp index d7cd5c5a295..b1dd70feba7 100644 --- a/dbms/src/Debug/dbgFuncCoprocessorUtils.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessorUtils.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include namespace DB { diff --git a/dbms/src/Debug/dbgFuncCoprocessorUtils.h b/dbms/src/Debug/dbgFuncCoprocessorUtils.h index 8428a908a3b..7d9ca5a1075 100644 --- a/dbms/src/Debug/dbgFuncCoprocessorUtils.h +++ b/dbms/src/Debug/dbgFuncCoprocessorUtils.h @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Debug/dbgQueryExecutor.cpp b/dbms/src/Debug/dbgQueryExecutor.cpp index c5c0dbaf12c..be7ee9b9ca6 100644 --- a/dbms/src/Debug/dbgQueryExecutor.cpp +++ b/dbms/src/Debug/dbgQueryExecutor.cpp @@ -300,9 +300,7 @@ tipb::SelectResponse executeDAGRequest(Context & context, const tipb::DAGRequest table_regions_info.local_regions.emplace(region_id, RegionInfo(region_id, region_version, region_conf_version, std::move(key_ranges), nullptr)); - DAGContext dag_context(dag_request); - dag_context.tables_regions_info = std::move(tables_regions_info); - dag_context.log = log; + DAGContext dag_context(dag_request, std::move(tables_regions_info), "", false, log); context.setDAGContext(&dag_context); DAGDriver driver(context, start_ts, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, &dag_response, true); @@ -330,9 +328,7 @@ bool runAndCompareDagReq(const coprocessor::Request & req, const coprocessor::Re auto & table_regions_info = tables_regions_info.getSingleTableRegions(); table_regions_info.local_regions.emplace(region_id, RegionInfo(region_id, region->version(), region->confVer(), std::move(key_ranges), nullptr)); - DAGContext dag_context(dag_request); - dag_context.tables_regions_info = std::move(tables_regions_info); - dag_context.log = log; + DAGContext dag_context(dag_request, std::move(tables_regions_info), "", false, log); context.setDAGContext(&dag_context); DAGDriver driver(context, properties.start_ts, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, &dag_response, true); driver.execute(); diff --git a/dbms/src/Flash/BatchCoprocessorHandler.cpp b/dbms/src/Flash/BatchCoprocessorHandler.cpp index a25c14dc84e..5633d91a463 100644 --- a/dbms/src/Flash/BatchCoprocessorHandler.cpp +++ b/dbms/src/Flash/BatchCoprocessorHandler.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -68,11 +69,12 @@ grpc::Status BatchCoprocessorHandler::execute() tables_regions_info.tableCount(), dag_request.DebugString()); - DAGContext dag_context(dag_request); - dag_context.is_batch_cop = true; - dag_context.tables_regions_info = std::move(tables_regions_info); - dag_context.log = Logger::get("BatchCoprocessorHandler"); - dag_context.tidb_host = cop_context.db_context.getClientInfo().current_address.toString(); + DAGContext dag_context( + dag_request, + std::move(tables_regions_info), + cop_context.db_context.getClientInfo().current_address.toString(), + /*is_batch_cop=*/true, + Logger::get("BatchCoprocessorHandler")); cop_context.db_context.setDAGContext(&dag_context); DAGDriver driver(cop_context.db_context, cop_request->start_ts() > 0 ? cop_request->start_ts() : dag_request.start_ts_fallback(), cop_request->schema_ver(), writer); diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index ac17e3e2990..aaf218ba24e 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -127,13 +127,17 @@ class DAGContext { public: // for non-mpp(cop/batchCop) - explicit DAGContext(const tipb::DAGRequest & dag_request_) + explicit DAGContext(const tipb::DAGRequest & dag_request_, TablesRegionsInfo && tables_regions_info_, const String & tidb_host_, bool is_batch_cop_, LoggerPtr log_) : dag_request(&dag_request_) , dummy_query_string(dag_request->DebugString()) , dummy_ast(makeDummyQuery()) + , tidb_host(tidb_host_) , collect_execution_summaries(dag_request->has_collect_execution_summaries() && dag_request->collect_execution_summaries()) , is_mpp_task(false) , is_root_mpp_task(false) + , is_batch_cop(is_batch_cop_) + , tables_regions_info(std::move(tables_regions_info_)) + , log(std::move(log_)) , flags(dag_request->flags()) , sql_mode(dag_request->sql_mode()) , max_recorded_error_count(getMaxErrorCount(*dag_request)) @@ -345,9 +349,9 @@ class DAGContext String tidb_host = "Unknown"; bool collect_execution_summaries{}; bool return_executor_id{}; - bool is_mpp_task = false; - bool is_root_mpp_task = false; - bool is_batch_cop = false; + /* const */ bool is_mpp_task = false; + /* const */ bool is_root_mpp_task = false; + /* const */ bool is_batch_cop = false; // `tunnel_set` is always set by `MPPTask` and is intended to be used for `DAGQueryBlockInterpreter`. MPPTunnelSetPtr tunnel_set; TablesRegionsInfo tables_regions_info; diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index cfde043963f..f181f5329ca 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 13e751c4b85..484131ef583 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -19,7 +19,6 @@ #include #pragma GCC diagnostic pop -#include #include #include #include diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h index 5a38282c02c..297a679d4e9 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -19,7 +19,6 @@ #include #pragma GCC diagnostic pop -#include #include #include #include diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 834fd830158..345fc6caa6e 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.h b/dbms/src/Flash/Coprocessor/DAGQuerySource.h index e79958f3f48..bb052f538c1 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.h +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.h @@ -14,13 +14,14 @@ #pragma once -#include #include #include #include namespace DB { +class DAGContext; + /// DAGQuerySource is an adaptor between DAG and CH's executeQuery. /// TODO: consider to directly use DAGContext instead. class DAGQuerySource : public IQuerySource diff --git a/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp b/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp index 5d97fe0edad..f008146189b 100644 --- a/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGResponseWriter.h b/dbms/src/Flash/Coprocessor/DAGResponseWriter.h index f6a87880c4a..3ca0580482b 100644 --- a/dbms/src/Flash/Coprocessor/DAGResponseWriter.h +++ b/dbms/src/Flash/Coprocessor/DAGResponseWriter.h @@ -14,11 +14,13 @@ #pragma once -#include +#include #include namespace DB { +class DAGContext; + class DAGResponseWriter { public: diff --git a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h index 4d3583c47b3..c6bced12ba4 100644 --- a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h +++ b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h @@ -14,7 +14,6 @@ #pragma once -#include #include #include diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.h b/dbms/src/Flash/Coprocessor/InterpreterDAG.h index 40f7d8c62cf..c5c0406c8c3 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.h +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.h @@ -21,7 +21,6 @@ #pragma GCC diagnostic pop #include -#include #include #include @@ -30,6 +29,7 @@ namespace DB class Context; class Region; using RegionPtr = std::shared_ptr; +class DAGContext; /** build ch plan from dag request: dag executors -> ch plan */ diff --git a/dbms/src/Flash/Coprocessor/MockSourceStream.h b/dbms/src/Flash/Coprocessor/MockSourceStream.h index 16dbfa3509f..7cb0ffc95e7 100644 --- a/dbms/src/Flash/Coprocessor/MockSourceStream.h +++ b/dbms/src/Flash/Coprocessor/MockSourceStream.h @@ -18,7 +18,6 @@ #include #include #include -#include #include namespace DB @@ -69,4 +68,4 @@ std::pair>> mockSourceStr RUNTIME_ASSERT(start == rows, log, "mock source streams' total size must same as user input"); return {names_and_types, mock_source_streams}; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp index 6a0766f900e..0180f0d5900 100644 --- a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.h b/dbms/src/Flash/Coprocessor/RemoteRequest.h index ad6d425010c..7973d48b492 100644 --- a/dbms/src/Flash/Coprocessor/RemoteRequest.h +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include diff --git a/dbms/src/Flash/Coprocessor/RequestUtils.h b/dbms/src/Flash/Coprocessor/RequestUtils.h new file mode 100644 index 00000000000..e4bc524e675 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/RequestUtils.h @@ -0,0 +1,78 @@ +// 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. + +#pragma once + +#include + +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wdeprecated-declarations" +#pragma clang diagnostic ignored "-Wunused-parameter" +#endif +#include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif + +namespace DB::RequestUtils +{ +template +void setUpRegion(const pingcap::coprocessor::RegionInfo & region_info, PbRegion * region) +{ + region->set_region_id(region_info.region_id.id); + region->mutable_region_epoch()->set_version(region_info.region_id.ver); + region->mutable_region_epoch()->set_conf_ver(region_info.region_id.conf_ver); + for (const auto & key_range : region_info.ranges) + { + key_range.setKeyRange(region->add_ranges()); + } +} + +template +std::vector +setUpRegionInfos(const pingcap::coprocessor::BatchCopTask & batch_cop_task, const RequestPtr & req) +{ + RUNTIME_CHECK_MSG(batch_cop_task.region_infos.empty() != batch_cop_task.table_regions.empty(), + "region_infos and table_regions should not exist at the same time, single table region info: {}, partition table region info: {}", + batch_cop_task.region_infos.size(), + batch_cop_task.table_regions.size()); + + std::vector region_ids; + if (!batch_cop_task.region_infos.empty()) + { + // For non-partition table + region_ids.reserve(batch_cop_task.region_infos.size()); + for (const auto & region_info : batch_cop_task.region_infos) + { + region_ids.push_back(region_info.region_id); + setUpRegion(region_info, req->add_regions()); + } + return region_ids; + } + // For partition table + for (const auto & table_region : batch_cop_task.table_regions) + { + auto * req_table_region = req->add_table_regions(); + req_table_region->set_physical_table_id(table_region.physical_table_id); + for (const auto & region_info : table_region.region_infos) + { + region_ids.push_back(region_info.region_id); + setUpRegion(region_info, req_table_region->add_regions()); + } + } + return region_ids; +} + +} // namespace DB::RequestUtils diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp index dd347ca8929..9fc78cfeb1c 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp +++ b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h index def4f59d01b..16e2014cbd5 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h +++ b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h @@ -18,13 +18,14 @@ #include #include #include -#include #include #include #include namespace DB { +class DAGContext; + /// Serializes the stream of blocks and sends them to TiDB/TiSpark with different serialization paths. template class StreamingDAGResponseWriter : public DAGResponseWriter diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp index 97af20992be..5b78ec72dfa 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include namespace DB diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.h b/dbms/src/Flash/Coprocessor/TiDBTableScan.h index 11c5b974476..6903c4ca5bb 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.h +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.h @@ -14,11 +14,13 @@ #pragma once -#include +#include +#include +#include namespace DB { - +class DAGContext; using ColumnInfos = std::vector; /// TiDBTableScan is a wrap to hide the difference of `TableScan` and `PartitionTableScan` diff --git a/dbms/src/Flash/Coprocessor/UnaryDAGResponseWriter.cpp b/dbms/src/Flash/Coprocessor/UnaryDAGResponseWriter.cpp index b245c353c49..ff692b910a2 100644 --- a/dbms/src/Flash/Coprocessor/UnaryDAGResponseWriter.cpp +++ b/dbms/src/Flash/Coprocessor/UnaryDAGResponseWriter.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_req_encode.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_req_encode.cpp new file mode 100644 index 00000000000..f5aeab13abc --- /dev/null +++ b/dbms/src/Flash/Coprocessor/tests/gtest_req_encode.cpp @@ -0,0 +1,150 @@ +// 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 + +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wdeprecated-declarations" +#endif +#include +#include +#include +#include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif + +namespace DB::tests +{ +bool isSameKeyRange(const coprocessor::KeyRange & actual, const pingcap::coprocessor::KeyRange & expect) +{ + return (actual.start() == expect.start_key && actual.end() == expect.end_key); +} +#define ASSERT_RANGE_EQ(pb_keyrange, expect) ASSERT_TRUE(isSameKeyRange((pb_keyrange), (expect))) + +bool isSameRegionId(const coprocessor::RegionInfo & actual, const pingcap::kv::RegionVerID & expect) +{ + return actual.region_id() == expect.id + && actual.region_epoch().conf_ver() == expect.conf_ver + && actual.region_epoch().version() == expect.ver; +} +#define ASSERT_REGION_ID_EQ(pb_region, expect) ASSERT_TRUE(isSameRegionId((pb_region), (expect))) + +TEST(RequestEncodeTest, MPPNonPartitionRegions) +{ + namespace pc = pingcap::coprocessor; + namespace pkv = pingcap::kv; + pc::BatchCopTask batch_cop_task; + std::vector regions{ + pc::RegionInfo{.region_id = {111, 1, 4}, .ranges = {{"a", "b"}, {"b", "c"}}}, + pc::RegionInfo{.region_id = {222, 2, 5}, .ranges = {{"dd", "e"}, {"f", "g"}}}, + pc::RegionInfo{.region_id = {333, 3, 6}, .ranges = {{"x", "y"}}}, + }; + batch_cop_task.region_infos = regions; + + ::mpp::DispatchTaskRequest req; + RequestUtils::setUpRegionInfos(batch_cop_task, &req); + + ASSERT_EQ(req.regions_size(), 3); + ASSERT_EQ(req.table_regions_size(), 0); + + auto reg0 = req.regions(0); + ASSERT_REGION_ID_EQ(reg0, pkv::RegionVerID(111, 1, 4)); + ASSERT_EQ(reg0.ranges_size(), 2); + ASSERT_RANGE_EQ(reg0.ranges(0), pc::KeyRange("a", "b")); + ASSERT_RANGE_EQ(reg0.ranges(1), pc::KeyRange("b", "c")); + + auto reg1 = req.regions(1); + ASSERT_REGION_ID_EQ(reg1, pkv::RegionVerID(222, 2, 5)); + ASSERT_EQ(reg1.ranges_size(), 2); + ASSERT_RANGE_EQ(reg1.ranges(0), pc::KeyRange("dd", "e")); + ASSERT_RANGE_EQ(reg1.ranges(1), pc::KeyRange("f", "g")); + + auto reg2 = req.regions(2); + ASSERT_REGION_ID_EQ(reg2, pkv::RegionVerID(333, 3, 6)); + ASSERT_EQ(reg2.ranges_size(), 1); + ASSERT_RANGE_EQ(reg2.ranges(0), pc::KeyRange("x", "y")); +} + +TEST(RequestEncodeTest, MPPPartitionRegions) +{ + namespace pc = pingcap::coprocessor; + namespace pkv = pingcap::kv; + pc::BatchCopTask batch_cop_task; + std::vector regions{ + // clang-format off + pc::TableRegions{ + .physical_table_id = 70, + .region_infos = { + pc::RegionInfo{.region_id = {111, 1, 4}, .ranges = {{"a", "b"}, {"b", "c"}}}, + pc::RegionInfo{.region_id = {222, 2, 5}, .ranges = {{"dd", "e"}, {"f", "g"}}}, + pc::RegionInfo{.region_id = {333, 3, 6}, .ranges = {{"x", "y"}}}, + }}, + pc::TableRegions{ + .physical_table_id = 71, + .region_infos = { + pc::RegionInfo{.region_id = {444, 1, 4}, .ranges = {{"z", "z0"}, {"z2", "z3"}}}, + pc::RegionInfo{.region_id = {555, 3, 6}, .ranges = {{"zy", "zz"}}}, + }}, + // clang-format on + }; + batch_cop_task.table_regions = regions; + + ::mpp::DispatchTaskRequest req; + RequestUtils::setUpRegionInfos(batch_cop_task, &req); + + ASSERT_EQ(req.regions_size(), 0); + ASSERT_EQ(req.table_regions_size(), 2); + + { + const auto & table0 = req.table_regions(0); + ASSERT_EQ(table0.regions_size(), 3); + + const auto & reg0 = table0.regions(0); + ASSERT_REGION_ID_EQ(reg0, pkv::RegionVerID(111, 1, 4)); + ASSERT_EQ(reg0.ranges_size(), 2); + ASSERT_RANGE_EQ(reg0.ranges(0), pc::KeyRange("a", "b")); + ASSERT_RANGE_EQ(reg0.ranges(1), pc::KeyRange("b", "c")); + + const auto & reg1 = table0.regions(1); + ASSERT_REGION_ID_EQ(reg1, pkv::RegionVerID(222, 2, 5)); + ASSERT_EQ(reg1.ranges_size(), 2); + ASSERT_RANGE_EQ(reg1.ranges(0), pc::KeyRange("dd", "e")); + ASSERT_RANGE_EQ(reg1.ranges(1), pc::KeyRange("f", "g")); + + const auto & reg2 = table0.regions(2); + ASSERT_REGION_ID_EQ(reg2, pkv::RegionVerID(333, 3, 6)); + ASSERT_EQ(reg2.ranges_size(), 1); + ASSERT_RANGE_EQ(reg2.ranges(0), pc::KeyRange("x", "y")); + } + + { + const auto & table1 = req.table_regions(1); + ASSERT_EQ(table1.regions_size(), 2); + + const auto & reg0 = table1.regions(0); + ASSERT_REGION_ID_EQ(reg0, pkv::RegionVerID(444, 1, 4)); + ASSERT_EQ(reg0.ranges_size(), 2); + ASSERT_RANGE_EQ(reg0.ranges(0), pc::KeyRange("z", "z0")); + ASSERT_RANGE_EQ(reg0.ranges(1), pc::KeyRange("z2", "z3")); + + const auto & reg1 = table1.regions(1); + ASSERT_REGION_ID_EQ(reg1, pkv::RegionVerID(555, 3, 6)); + ASSERT_EQ(reg1.ranges_size(), 1); + ASSERT_RANGE_EQ(reg1.ranges(0), pc::KeyRange("zy", "zz")); + } +} +} // namespace DB::tests diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_streaming_writer.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_streaming_writer.cpp index 5442cd02d29..1b429184996 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_streaming_writer.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_streaming_writer.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Flash/CoprocessorHandler.cpp b/dbms/src/Flash/CoprocessorHandler.cpp index f428add83c0..f5f4b313363 100644 --- a/dbms/src/Flash/CoprocessorHandler.cpp +++ b/dbms/src/Flash/CoprocessorHandler.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -105,10 +106,12 @@ grpc::Status CoprocessorHandler::execute() genCopKeyRange(cop_request->ranges()), &bypass_lock_ts)); - DAGContext dag_context(dag_request); - dag_context.tables_regions_info = std::move(tables_regions_info); - dag_context.log = Logger::get("CoprocessorHandler"); - dag_context.tidb_host = cop_context.db_context.getClientInfo().current_address.toString(); + DAGContext dag_context( + dag_request, + std::move(tables_regions_info), + cop_context.db_context.getClientInfo().current_address.toString(), + /*is_batch_cop=*/false, + Logger::get("CoprocessorHandler")); cop_context.db_context.setDAGContext(&dag_context); DAGDriver driver(cop_context.db_context, cop_request->start_ts() > 0 ? cop_request->start_ts() : dag_request.start_ts_fallback(), cop_request->schema_ver(), &dag_response); diff --git a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp index 9303b4ad8a4..f8fea4c1c9e 100644 --- a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp +++ b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include diff --git a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h index 47c46307ee2..322d68541b3 100644 --- a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h +++ b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h @@ -15,13 +15,14 @@ #pragma once #include -#include #include #include #include namespace DB { +class DAGContext; + template class BroadcastOrPassThroughWriter : public DAGResponseWriter { diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.h b/dbms/src/Flash/Mpp/ExchangeReceiver.h index b5cc4877903..d5a9812eb56 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.h +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.h @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index cc6dba5631e..b5bb5852c5e 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h index 3c91518cea4..6b2db46770c 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h @@ -15,13 +15,14 @@ #pragma once #include -#include #include #include #include namespace DB { +class DAGContext; + template class FineGrainedShuffleWriter : public DAGResponseWriter { diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 5f7286833f9..7d48b43484b 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index eb82c54592e..deebcd3dce7 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -15,13 +15,14 @@ #pragma once #include -#include #include #include #include namespace DB { +class DAGContext; + template class HashPartitionWriter : public DAGResponseWriter { diff --git a/dbms/src/Flash/Mpp/MPPReceiverSet.h b/dbms/src/Flash/Mpp/MPPReceiverSet.h index 2ae06216418..3319011e876 100644 --- a/dbms/src/Flash/Mpp/MPPReceiverSet.h +++ b/dbms/src/Flash/Mpp/MPPReceiverSet.h @@ -15,7 +15,6 @@ #pragma once #include -#include namespace DB { diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index fa3456d96eb..95660c67bc4 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Planner/PlanQuerySource.cpp b/dbms/src/Flash/Planner/PlanQuerySource.cpp index a0f29c4630a..464be2ee111 100644 --- a/dbms/src/Flash/Planner/PlanQuerySource.cpp +++ b/dbms/src/Flash/Planner/PlanQuerySource.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -40,4 +41,9 @@ std::unique_ptr PlanQuerySource::interpreter(Context &, QueryProce return std::make_unique(context, *this); } +const tipb::DAGRequest & PlanQuerySource::getDAGRequest() const +{ + return *getDAGContext().dag_request; +} + } // namespace DB diff --git a/dbms/src/Flash/Planner/PlanQuerySource.h b/dbms/src/Flash/Planner/PlanQuerySource.h index a54c5791e11..1e67eee5dd7 100644 --- a/dbms/src/Flash/Planner/PlanQuerySource.h +++ b/dbms/src/Flash/Planner/PlanQuerySource.h @@ -14,12 +14,13 @@ #pragma once -#include #include #include namespace DB { +class DAGContext; + class PlanQuerySource : public IQuerySource { public: @@ -30,7 +31,7 @@ class PlanQuerySource : public IQuerySource std::unique_ptr interpreter(Context & context, QueryProcessingStage::Enum stage) override; DAGContext & getDAGContext() const { return *context.getDAGContext(); } - const tipb::DAGRequest & getDAGRequest() const { return *getDAGContext().dag_request; } + const tipb::DAGRequest & getDAGRequest() const; private: Context & context; diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp index d06099b17e7..76928e73490 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp index 26f31bf4400..c3019f5ed8b 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include diff --git a/dbms/src/Flash/Statistics/ExchangeSenderImpl.cpp b/dbms/src/Flash/Statistics/ExchangeSenderImpl.cpp index 10fc4275620..c7370724bbc 100644 --- a/dbms/src/Flash/Statistics/ExchangeSenderImpl.cpp +++ b/dbms/src/Flash/Statistics/ExchangeSenderImpl.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -91,4 +92,4 @@ ExchangeSenderStatistics::ExchangeSenderStatistics(const tipb::Executor * execut mpp_tunnel_details.back().sender_target_host = dag_context.tidb_host; } } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Statistics/ExecutorStatistics.h b/dbms/src/Flash/Statistics/ExecutorStatistics.h index 0844a18b4a2..3f77ad9b33c 100644 --- a/dbms/src/Flash/Statistics/ExecutorStatistics.h +++ b/dbms/src/Flash/Statistics/ExecutorStatistics.h @@ -31,6 +31,8 @@ namespace DB { +class DAGContext; + template class ExecutorStatistics : public ExecutorStatisticsBase { @@ -114,4 +116,4 @@ class ExecutorStatistics : public ExecutorStatisticsBase virtual void collectExtraRuntimeDetail() {} }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp b/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp index c9855e61ec6..6837e1cc8fc 100644 --- a/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp +++ b/dbms/src/Flash/tests/gtest_storage_disaggregated.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include diff --git a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h index 6fb6026d370..84f60409147 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Storages/StorageDisaggregated.cpp b/dbms/src/Storages/StorageDisaggregated.cpp index d2c6880db45..6cdb210b45a 100644 --- a/dbms/src/Storages/StorageDisaggregated.cpp +++ b/dbms/src/Storages/StorageDisaggregated.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include @@ -21,6 +22,19 @@ namespace DB { const String StorageDisaggregated::ExecIDPrefixForTiFlashStorageSender = "exec_id_disaggregated_tiflash_storage_sender"; +StorageDisaggregated::StorageDisaggregated( + Context & context_, + const TiDBTableScan & table_scan_, + const PushDownFilter & push_down_filter_) + : IStorage() + , context(context_) + , table_scan(table_scan_) + , log(Logger::get(context_.getDAGContext()->log ? context_.getDAGContext()->log->identifier() : "")) + , sender_target_mpp_task_id(context_.getDAGContext()->getMPPTaskMeta()) + , push_down_filter(push_down_filter_) +{ +} + BlockInputStreams StorageDisaggregated::read( const Names &, const SelectQueryInfo &, @@ -100,8 +114,6 @@ std::vector StorageDisaggregated::buildBatch StorageDisaggregated::RequestAndRegionIDs StorageDisaggregated::buildDispatchMPPTaskRequest( const pingcap::coprocessor::BatchCopTask & batch_cop_task) { - // For error handling, need to record region_ids and store_id to invalidate cache. - std::vector region_ids; auto dispatch_req = std::make_shared<::mpp::DispatchTaskRequest>(); ::mpp::TaskMeta * dispatch_req_meta = dispatch_req->mutable_meta(); dispatch_req_meta->set_start_ts(sender_target_mpp_task_id.query_id.start_ts); @@ -113,47 +125,9 @@ StorageDisaggregated::RequestAndRegionIDs StorageDisaggregated::buildDispatchMPP const auto & settings = context.getSettings(); dispatch_req->set_timeout(60); dispatch_req->set_schema_ver(settings.schema_version); - RUNTIME_CHECK_MSG(batch_cop_task.region_infos.empty() != batch_cop_task.table_regions.empty(), - "region_infos and table_regions should not exist at the same time, single table region info: {}, partition table region info: {}", - batch_cop_task.region_infos.size(), - batch_cop_task.table_regions.size()); - if (!batch_cop_task.region_infos.empty()) - { - // For non-partition table. - for (const auto & region_info : batch_cop_task.region_infos) - { - region_ids.push_back(region_info.region_id); - auto * region = dispatch_req->add_regions(); - region->set_region_id(region_info.region_id.id); - region->mutable_region_epoch()->set_version(region_info.region_id.ver); - region->mutable_region_epoch()->set_conf_ver(region_info.region_id.conf_ver); - for (const auto & key_range : region_info.ranges) - { - key_range.setKeyRange(region->add_ranges()); - } - } - } - else - { - // For partition table. - for (const auto & table_region : batch_cop_task.table_regions) - { - auto * req_table_region = dispatch_req->add_table_regions(); - req_table_region->set_physical_table_id(table_region.physical_table_id); - auto * region = req_table_region->add_regions(); - for (const auto & region_info : table_region.region_infos) - { - region_ids.push_back(region_info.region_id); - region->set_region_id(region_info.region_id.id); - region->mutable_region_epoch()->set_version(region_info.region_id.ver); - region->mutable_region_epoch()->set_conf_ver(region_info.region_id.conf_ver); - for (const auto & key_range : region_info.ranges) - { - key_range.setKeyRange(region->add_ranges()); - } - } - } - } + + // For error handling, need to record region_ids and store_id to invalidate cache. + std::vector region_ids = RequestUtils::setUpRegionInfos(batch_cop_task, dispatch_req); const auto & sender_target_task_meta = context.getDAGContext()->getMPPTaskMeta(); const auto * dag_req = context.getDAGContext()->dag_request; diff --git a/dbms/src/Storages/StorageDisaggregated.h b/dbms/src/Storages/StorageDisaggregated.h index 59971b05012..2414d5828b2 100644 --- a/dbms/src/Storages/StorageDisaggregated.h +++ b/dbms/src/Storages/StorageDisaggregated.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -39,14 +40,7 @@ class StorageDisaggregated : public IStorage StorageDisaggregated( Context & context_, const TiDBTableScan & table_scan_, - const PushDownFilter & push_down_filter_) - : IStorage() - , context(context_) - , table_scan(table_scan_) - , log(Logger::get(context_.getDAGContext()->log ? context_.getDAGContext()->log->identifier() : "")) - , sender_target_mpp_task_id(context_.getDAGContext()->getMPPTaskMeta()) - , push_down_filter(push_down_filter_) - {} + const PushDownFilter & push_down_filter_); std::string getName() const override { diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index 583380aac8c..a6e60b46a04 100644 --- a/dbms/src/Storages/tests/gtest_filter_parser.cpp +++ b/dbms/src/Storages/tests/gtest_filter_parser.cpp @@ -83,7 +83,7 @@ DM::RSOperatorPtr FilterParserTest::generateRsOperator(const String table_info_j }, getDAGProperties("")); auto & dag_request = *query_tasks[0].dag_request; - DAGContext dag_context(dag_request); + DAGContext dag_context(dag_request, {}, "", false, log); ctx.setDAGContext(&dag_context); // Don't care about regions information in this test DAGQuerySource dag(ctx); diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.h b/dbms/src/TestUtils/ColumnsToTiPBExpr.h index e7a2e81d59e..3bbd24fbbd5 100644 --- a/dbms/src/TestUtils/ColumnsToTiPBExpr.h +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.h @@ -20,7 +20,6 @@ #include #include #include -#include namespace DB { diff --git a/dbms/src/TestUtils/MPPTaskTestUtils.cpp b/dbms/src/TestUtils/MPPTaskTestUtils.cpp index f9a9123057c..6d7275a19a0 100644 --- a/dbms/src/TestUtils/MPPTaskTestUtils.cpp +++ b/dbms/src/TestUtils/MPPTaskTestUtils.cpp @@ -104,7 +104,7 @@ ColumnsWithTypeAndName MPPTaskTestUtils::executeCoprocessorTask(std::shared_ptr< auto * data = req->mutable_data(); dag_request->AppendToString(data); - DAGContext dag_context(*dag_request); + DAGContext dag_context(*dag_request, {}, "", false, Logger::get()); TiFlashTestEnv::getGlobalContext(test_meta.context_idx).setDAGContext(&dag_context); TiFlashTestEnv::getGlobalContext(test_meta.context_idx).setCopTest(); From b99541f240c2d56c4aaadfb80de0d51f1a64c389 Mon Sep 17 00:00:00 2001 From: Wenxuan Date: Wed, 21 Dec 2022 18:40:55 +0800 Subject: [PATCH 32/42] Disable ingest range check (#6519) close pingcap/tiflash#6507 --- .../DeltaMerge/DeltaMergeStore_Ingest.cpp | 17 ++++-- .../DeltaMerge/tests/gtest_dm_ingest.cpp | 58 +++++++++---------- 2 files changed, 41 insertions(+), 34 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp index 0722d7ee3ba..b68a7dab299 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Ingest.cpp @@ -538,11 +538,18 @@ void DeltaMergeStore::ingestFiles( } // Check whether all external files are contained by the range. - for (const auto & ext_file : external_files) - { - RUNTIME_CHECK(compare(range.getStart(), ext_file.range.getStart()) <= 0); - RUNTIME_CHECK(compare(range.getEnd(), ext_file.range.getEnd()) >= 0); - } + // Currently this check is disabled, see https://github.com/pingcap/tiflash/pull/6519 + // for (const auto & ext_file : external_files) + // { + // RUNTIME_CHECK( + // compare(range.getStart(), ext_file.range.getStart()) <= 0, + // range.toDebugString(), + // ext_file.range.toDebugString()); + // RUNTIME_CHECK( + // compare(range.getEnd(), ext_file.range.getEnd()) >= 0, + // range.toDebugString(), + // ext_file.range.toDebugString()); + // } } EventRecorder write_block_recorder(ProfileEvents::DMWriteFile, ProfileEvents::DMWriteFileNS); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_ingest.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_ingest.cpp index 2c04541be4e..c6a426fc79b 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_ingest.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_ingest.cpp @@ -78,35 +78,35 @@ try } CATCH -TEST_P(StoreIngestTest, RangeSmallerThanData) -try -{ - ASSERT_EQ(0, getRowsN()); - auto block1 = fillBlock({.range = {0, 100}}); - ASSERT_THROW({ - ingestFiles({.range = {20, 40}, .blocks = {block1}, .clear = false}); - }, - DB::Exception); -} -CATCH - -TEST_P(StoreIngestTest, RangeLargerThanData) -try -{ - ASSERT_EQ(0, getRowsN()); - auto block1 = fillBlock({.range = {0, 100}}); - ingestFiles({.range = {-100, 110}, .blocks = {block1}, .clear = false}); - ASSERT_TRUE(isFilled(0, 100)); - ASSERT_EQ(100, getRowsN()); - - fill(-500, 500); - ingestFiles({.range = {-100, 110}, .blocks = {block1}, .clear = true}); - ASSERT_TRUE(isFilled(-500, -100)); - ASSERT_TRUE(isFilled(0, 100)); - ASSERT_TRUE(isFilled(110, 500)); - ASSERT_EQ(890, getRowsN()); -} -CATCH +//TEST_P(StoreIngestTest, RangeSmallerThanData) +//try +//{ +// ASSERT_EQ(0, getRowsN()); +// auto block1 = fillBlock({.range = {0, 100}}); +// ASSERT_THROW({ +// ingestFiles({.range = {20, 40}, .blocks = {block1}, .clear = false}); +// }, +// DB::Exception); +//} +//CATCH +// +//TEST_P(StoreIngestTest, RangeLargerThanData) +//try +//{ +// ASSERT_EQ(0, getRowsN()); +// auto block1 = fillBlock({.range = {0, 100}}); +// ingestFiles({.range = {-100, 110}, .blocks = {block1}, .clear = false}); +// ASSERT_TRUE(isFilled(0, 100)); +// ASSERT_EQ(100, getRowsN()); +// +// fill(-500, 500); +// ingestFiles({.range = {-100, 110}, .blocks = {block1}, .clear = true}); +// ASSERT_TRUE(isFilled(-500, -100)); +// ASSERT_TRUE(isFilled(0, 100)); +// ASSERT_TRUE(isFilled(110, 500)); +// ASSERT_EQ(890, getRowsN()); +//} +//CATCH TEST_P(StoreIngestTest, OverlappedFiles) try From 2ea6253a875192ce6e362d212ddaa1e9e8615ac8 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Thu, 22 Dec 2022 11:08:55 +0800 Subject: [PATCH 33/42] Test: fix mpp test bug (#6511) ref pingcap/tiflash#4609, close pingcap/tiflash#6442 --- dbms/src/Debug/MockComputeServerManager.cpp | 2 + dbms/src/Debug/MockComputeServerManager.h | 4 +- .../Debug/MockExecutor/AggregationBinder.cpp | 5 +- .../Debug/MockExecutor/AggregationBinder.h | 6 ++- dbms/src/Debug/MockExecutor/AstToPB.cpp | 18 +++++++ dbms/src/Debug/MockExecutor/AstToPB.h | 25 +++------- .../MockExecutor/ExchangeReceiverBinder.cpp | 5 ++ .../MockExecutor/ExchangeReceiverBinder.h | 1 + .../MockExecutor/ExchangeSenderBinder.cpp | 5 ++ .../Debug/MockExecutor/ExchangeSenderBinder.h | 1 + dbms/src/Debug/MockExecutor/ExecutorBinder.h | 10 ++-- dbms/src/Debug/MockExecutor/JoinBinder.cpp | 4 ++ dbms/src/Debug/MockExecutor/JoinBinder.h | 4 +- dbms/src/Debug/MockExecutor/LimitBinder.cpp | 1 + dbms/src/Debug/MockExecutor/ProjectBinder.cpp | 4 ++ .../Debug/MockExecutor/SelectionBinder.cpp | 2 +- dbms/src/Debug/MockExecutor/SelectionBinder.h | 1 - dbms/src/Debug/MockExecutor/SortBinder.cpp | 1 + .../Debug/MockExecutor/TableScanBinder.cpp | 1 + dbms/src/Debug/MockExecutor/TopNBinder.cpp | 2 + dbms/src/Debug/MockExecutor/WindowBinder.cpp | 3 ++ dbms/src/Debug/MockExecutor/WindowBinder.h | 1 - dbms/src/Debug/MockStorage.cpp | 33 +++++++++---- dbms/src/Debug/MockStorage.h | 5 +- dbms/src/Debug/dbgFuncCoprocessor.cpp | 3 ++ dbms/src/Debug/dbgFuncCoprocessor.h | 1 - dbms/src/Debug/dbgFuncCoprocessorUtils.cpp | 13 +++++ dbms/src/Debug/dbgFuncCoprocessorUtils.h | 17 +++---- dbms/src/Debug/dbgNaturalDag.h | 1 + dbms/src/Debug/dbgQueryCompiler.cpp | 27 +++++++++++ dbms/src/Debug/dbgQueryCompiler.h | 23 +-------- dbms/src/Debug/dbgQueryExecutor.cpp | 8 +++- dbms/src/Debug/dbgQueryExecutor.h | 1 - .../Coprocessor/DAGQueryBlockInterpreter.cpp | 12 ++++- .../Flash/Coprocessor/MockSourceStream.cpp | 24 ++++++++++ dbms/src/Flash/Coprocessor/MockSourceStream.h | 28 +++++++---- .../Planner/plans/PhysicalMockTableScan.cpp | 11 ++++- .../Flash/Planner/plans/PhysicalTableScan.cpp | 1 - dbms/src/Flash/tests/gtest_compute_server.cpp | 47 +++++++++++++++++++ dbms/src/Server/FlashGrpcServerHolder.cpp | 1 + dbms/src/TestUtils/ColumnsToTiPBExpr.cpp | 3 +- dbms/src/TestUtils/MPPTaskTestUtils.cpp | 22 +++++++-- dbms/src/TestUtils/MPPTaskTestUtils.h | 14 +++--- dbms/src/TestUtils/mockExecutor.cpp | 29 ++++++++---- dbms/src/TestUtils/mockExecutor.h | 1 + 45 files changed, 319 insertions(+), 112 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/MockSourceStream.cpp diff --git a/dbms/src/Debug/MockComputeServerManager.cpp b/dbms/src/Debug/MockComputeServerManager.cpp index 839cd794b73..64b5c6fc7d4 100644 --- a/dbms/src/Debug/MockComputeServerManager.cpp +++ b/dbms/src/Debug/MockComputeServerManager.cpp @@ -13,6 +13,8 @@ // limitations under the License. #include #include +#include +#include #include #include diff --git a/dbms/src/Debug/MockComputeServerManager.h b/dbms/src/Debug/MockComputeServerManager.h index 6642388659f..dd622e00b70 100644 --- a/dbms/src/Debug/MockComputeServerManager.h +++ b/dbms/src/Debug/MockComputeServerManager.h @@ -15,10 +15,8 @@ #pragma once #include -#include +#include #include -#include -#include namespace DB::tests { diff --git a/dbms/src/Debug/MockExecutor/AggregationBinder.cpp b/dbms/src/Debug/MockExecutor/AggregationBinder.cpp index a39f196a389..7b5d9b9d134 100644 --- a/dbms/src/Debug/MockExecutor/AggregationBinder.cpp +++ b/dbms/src/Debug/MockExecutor/AggregationBinder.cpp @@ -12,11 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include +#include #include #include -#include +#include +#include #include namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/AggregationBinder.h b/dbms/src/Debug/MockExecutor/AggregationBinder.h index 4ece3ff7838..84821594988 100644 --- a/dbms/src/Debug/MockExecutor/AggregationBinder.h +++ b/dbms/src/Debug/MockExecutor/AggregationBinder.h @@ -14,12 +14,14 @@ #pragma once -#include -#include #include +#include namespace DB::mock { +class ExchangeSenderBinder; +class ExchangeReceiverBinder; + class AggregationBinder : public ExecutorBinder { public: diff --git a/dbms/src/Debug/MockExecutor/AstToPB.cpp b/dbms/src/Debug/MockExecutor/AstToPB.cpp index 306d2c24813..fa58e2e3fc8 100644 --- a/dbms/src/Debug/MockExecutor/AstToPB.cpp +++ b/dbms/src/Debug/MockExecutor/AstToPB.cpp @@ -12,7 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { diff --git a/dbms/src/Debug/MockExecutor/AstToPB.h b/dbms/src/Debug/MockExecutor/AstToPB.h index c1560c90355..2f25618c361 100644 --- a/dbms/src/Debug/MockExecutor/AstToPB.h +++ b/dbms/src/Debug/MockExecutor/AstToPB.h @@ -14,24 +14,8 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include namespace DB { @@ -41,6 +25,11 @@ extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int NO_SUCH_COLUMN_IN_TABLE; } // namespace ErrorCodes + +class ASTFunction; +class ASTIdentifier; +class Context; + struct MPPCtx { Timestamp start_ts; diff --git a/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp index 706624856c0..21d4d649ffe 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp +++ b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.cpp @@ -12,8 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include #include #include +#include +#include namespace DB::mock { diff --git a/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.h b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.h index 2885dfd895d..c2327c87861 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.h +++ b/dbms/src/Debug/MockExecutor/ExchangeReceiverBinder.h @@ -14,6 +14,7 @@ #pragma once +#include #include namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp index aaba39868e1..45abb7de9fa 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp +++ b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp @@ -12,8 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include #include #include +#include +#include namespace DB::mock { diff --git a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h index 0b8b33821cf..ed6710ac22e 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h +++ b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h @@ -14,6 +14,7 @@ #pragma once +#include #include namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/ExecutorBinder.h b/dbms/src/Debug/MockExecutor/ExecutorBinder.h index de8e3c9928c..d1a03ff96d3 100644 --- a/dbms/src/Debug/MockExecutor/ExecutorBinder.h +++ b/dbms/src/Debug/MockExecutor/ExecutorBinder.h @@ -14,10 +14,13 @@ #pragma once +#include #include -#include #include -#include +#include +#include +#include +#include namespace DB::mock @@ -25,7 +28,6 @@ namespace DB::mock class ExchangeSenderBinder; class ExchangeReceiverBinder; - // Convert CH AST to tipb::Executor // Used in integration test framework and Unit test framework. class ExecutorBinder @@ -45,7 +47,7 @@ class ExecutorBinder index_++; } - std::vector> getChildren() + std::vector> getChildren() const { return children; } diff --git a/dbms/src/Debug/MockExecutor/JoinBinder.cpp b/dbms/src/Debug/MockExecutor/JoinBinder.cpp index 92109b73f1b..df0f11c2133 100644 --- a/dbms/src/Debug/MockExecutor/JoinBinder.cpp +++ b/dbms/src/Debug/MockExecutor/JoinBinder.cpp @@ -12,10 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include #include #include #include #include +#include +#include #include namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/JoinBinder.h b/dbms/src/Debug/MockExecutor/JoinBinder.h index 5ab1fb83f4b..c649420b8a9 100644 --- a/dbms/src/Debug/MockExecutor/JoinBinder.h +++ b/dbms/src/Debug/MockExecutor/JoinBinder.h @@ -14,12 +14,12 @@ #pragma once -#include -#include #include namespace DB::mock { +class ExchangeSenderBinder; +class ExchangeReceiverBinder; class JoinBinder : public ExecutorBinder { public: diff --git a/dbms/src/Debug/MockExecutor/LimitBinder.cpp b/dbms/src/Debug/MockExecutor/LimitBinder.cpp index c0a9bf17a82..de90b96f252 100644 --- a/dbms/src/Debug/MockExecutor/LimitBinder.cpp +++ b/dbms/src/Debug/MockExecutor/LimitBinder.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB::mock { diff --git a/dbms/src/Debug/MockExecutor/ProjectBinder.cpp b/dbms/src/Debug/MockExecutor/ProjectBinder.cpp index ebe8e5d8bde..50f0646c864 100644 --- a/dbms/src/Debug/MockExecutor/ProjectBinder.cpp +++ b/dbms/src/Debug/MockExecutor/ProjectBinder.cpp @@ -12,9 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include #include #include +#include #include +#include namespace DB::mock { diff --git a/dbms/src/Debug/MockExecutor/SelectionBinder.cpp b/dbms/src/Debug/MockExecutor/SelectionBinder.cpp index cea52b56922..c3171fa5e2c 100644 --- a/dbms/src/Debug/MockExecutor/SelectionBinder.cpp +++ b/dbms/src/Debug/MockExecutor/SelectionBinder.cpp @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -#include +#include #include namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/SelectionBinder.h b/dbms/src/Debug/MockExecutor/SelectionBinder.h index d4270ed5fac..b5e1c2000f3 100644 --- a/dbms/src/Debug/MockExecutor/SelectionBinder.h +++ b/dbms/src/Debug/MockExecutor/SelectionBinder.h @@ -14,7 +14,6 @@ #pragma once -#include #include namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/SortBinder.cpp b/dbms/src/Debug/MockExecutor/SortBinder.cpp index 80265448824..1af2820c71e 100644 --- a/dbms/src/Debug/MockExecutor/SortBinder.cpp +++ b/dbms/src/Debug/MockExecutor/SortBinder.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include diff --git a/dbms/src/Debug/MockExecutor/TableScanBinder.cpp b/dbms/src/Debug/MockExecutor/TableScanBinder.cpp index e35a14e4269..27f399f6d40 100644 --- a/dbms/src/Debug/MockExecutor/TableScanBinder.cpp +++ b/dbms/src/Debug/MockExecutor/TableScanBinder.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include diff --git a/dbms/src/Debug/MockExecutor/TopNBinder.cpp b/dbms/src/Debug/MockExecutor/TopNBinder.cpp index f8d7dd5f006..aee74b9300b 100644 --- a/dbms/src/Debug/MockExecutor/TopNBinder.cpp +++ b/dbms/src/Debug/MockExecutor/TopNBinder.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include +#include #include namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/WindowBinder.cpp b/dbms/src/Debug/MockExecutor/WindowBinder.cpp index 8da8ae5d8ef..0642300cecb 100644 --- a/dbms/src/Debug/MockExecutor/WindowBinder.cpp +++ b/dbms/src/Debug/MockExecutor/WindowBinder.cpp @@ -12,8 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include +#include #include +#include namespace DB::mock { diff --git a/dbms/src/Debug/MockExecutor/WindowBinder.h b/dbms/src/Debug/MockExecutor/WindowBinder.h index 443506baa33..b9745d3358b 100644 --- a/dbms/src/Debug/MockExecutor/WindowBinder.h +++ b/dbms/src/Debug/MockExecutor/WindowBinder.h @@ -27,7 +27,6 @@ struct MockWindowFrame std::optional end; // TODO: support calcFuncs }; - using ASTPartitionByElement = ASTOrderByElement; class WindowBinder : public ExecutorBinder diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index dbcf38c831b..7a19da7085b 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. #include +#include namespace DB::tests { @@ -22,8 +23,11 @@ void MockStorage::addTableSchema(const String & name, const MockColumnInfoVec & addTableInfo(name, columnInfos); } -void MockStorage::addTableData(const String & name, const ColumnsWithTypeAndName & columns) +void MockStorage::addTableData(const String & name, ColumnsWithTypeAndName & columns) { + for (size_t i = 0; i < columns.size(); ++i) + columns[i].column_id = i; + table_columns[getTableId(name)] = columns; } @@ -123,11 +127,12 @@ CutColumnInfo getCutColumnInfo(size_t rows, Int64 partition_id, Int64 partition_ return {start, cur_rows}; } -ColumnsWithTypeAndName MockStorage::getColumnsForMPPTableScan(Int64 table_id, Int64 partition_id, Int64 partition_num) +ColumnsWithTypeAndName MockStorage::getColumnsForMPPTableScan(const TiDBTableScan & table_scan, Int64 partition_id, Int64 partition_num) { + auto table_id = table_scan.getLogicalTableID(); if (tableExists(table_id)) { - auto columns_with_type_and_name = table_columns[table_id]; + auto columns_with_type_and_name = table_columns[table_scan.getLogicalTableID()]; size_t rows = 0; for (const auto & col : columns_with_type_and_name) { @@ -141,11 +146,23 @@ ColumnsWithTypeAndName MockStorage::getColumnsForMPPTableScan(Int64 table_id, In ColumnsWithTypeAndName res; for (const auto & column_with_type_and_name : columns_with_type_and_name) { - res.push_back( - ColumnWithTypeAndName( - column_with_type_and_name.column->cut(cut_info.first, cut_info.second), - column_with_type_and_name.type, - column_with_type_and_name.name)); + bool contains = false; + for (const auto & column : table_scan.getColumns()) + { + if (column.id == column_with_type_and_name.column_id) + { + contains = true; + break; + } + } + if (contains) + { + res.push_back( + ColumnWithTypeAndName( + column_with_type_and_name.column->cut(cut_info.first, cut_info.second), + column_with_type_and_name.type, + column_with_type_and_name.name)); + } } return res; } diff --git a/dbms/src/Debug/MockStorage.h b/dbms/src/Debug/MockStorage.h index 46e8331602f..ff5ff0627b3 100644 --- a/dbms/src/Debug/MockStorage.h +++ b/dbms/src/Debug/MockStorage.h @@ -13,6 +13,7 @@ // limitations under the License. #pragma once #include +#include #include #include @@ -47,7 +48,7 @@ class MockStorage /// for table scan void addTableSchema(const String & name, const MockColumnInfoVec & columnInfos); - void addTableData(const String & name, const ColumnsWithTypeAndName & columns); + void addTableData(const String & name, ColumnsWithTypeAndName & columns); Int64 getTableId(const String & name); @@ -72,7 +73,7 @@ class MockStorage MockColumnInfoVec getExchangeSchema(const String & exchange_name); /// for MPP Tasks, it will split data by partition num, then each MPP service will have a subset of mock data. - ColumnsWithTypeAndName getColumnsForMPPTableScan(Int64 table_id, Int64 partition_id, Int64 partition_num); + ColumnsWithTypeAndName getColumnsForMPPTableScan(const TiDBTableScan & table_scan, Int64 partition_id, Int64 partition_num); TableInfo getTableInfo(const String & name); diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index 07ee8703b92..112f43b568b 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -15,7 +15,10 @@ #include #include #include +#include +#include #include +#include namespace DB { diff --git a/dbms/src/Debug/dbgFuncCoprocessor.h b/dbms/src/Debug/dbgFuncCoprocessor.h index a296e93d410..9a21842fa50 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.h +++ b/dbms/src/Debug/dbgFuncCoprocessor.h @@ -15,7 +15,6 @@ #pragma once #include -#include namespace DB { class Context; diff --git a/dbms/src/Debug/dbgFuncCoprocessorUtils.cpp b/dbms/src/Debug/dbgFuncCoprocessorUtils.cpp index b1dd70feba7..e89163c2c1d 100644 --- a/dbms/src/Debug/dbgFuncCoprocessorUtils.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessorUtils.cpp @@ -12,8 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include +#include #include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include namespace DB { diff --git a/dbms/src/Debug/dbgFuncCoprocessorUtils.h b/dbms/src/Debug/dbgFuncCoprocessorUtils.h index 7d9ca5a1075..0f2c3d85533 100644 --- a/dbms/src/Debug/dbgFuncCoprocessorUtils.h +++ b/dbms/src/Debug/dbgFuncCoprocessorUtils.h @@ -13,19 +13,10 @@ // limitations under the License. #pragma once -#include -#include -#include -#include -#include -#include +#include #include -#include -#include #include -#include -#include -#include +#include namespace DB { @@ -33,6 +24,10 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; } +class Context; +struct DAGProperties; +class IBlockInputStream; +using BlockInputStreamPtr = std::shared_ptr; std::unique_ptr getCodec(tipb::EncodeType encode_type); DAGSchema getSelectSchema(Context & context); diff --git a/dbms/src/Debug/dbgNaturalDag.h b/dbms/src/Debug/dbgNaturalDag.h index f7c1d850ebe..67c7dca288e 100644 --- a/dbms/src/Debug/dbgNaturalDag.h +++ b/dbms/src/Debug/dbgNaturalDag.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Debug/dbgQueryCompiler.cpp b/dbms/src/Debug/dbgQueryCompiler.cpp index 2562e6b2efc..f9e58b1a424 100644 --- a/dbms/src/Debug/dbgQueryCompiler.cpp +++ b/dbms/src/Debug/dbgQueryCompiler.cpp @@ -12,7 +12,34 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { diff --git a/dbms/src/Debug/dbgQueryCompiler.h b/dbms/src/Debug/dbgQueryCompiler.h index 748b14d41e8..87397ab0728 100644 --- a/dbms/src/Debug/dbgQueryCompiler.h +++ b/dbms/src/Debug/dbgQueryCompiler.h @@ -16,32 +16,10 @@ #include #include -#include #include -#include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include namespace DB @@ -49,6 +27,7 @@ namespace DB using MakeResOutputStream = std::function; using ExecutorBinderPtr = mock::ExecutorBinderPtr; using TableInfo = TiDB::TableInfo; +struct ASTTablesInSelectQueryElement; enum class QueryTaskType { diff --git a/dbms/src/Debug/dbgQueryExecutor.cpp b/dbms/src/Debug/dbgQueryExecutor.cpp index be7ee9b9ca6..359aa833f25 100644 --- a/dbms/src/Debug/dbgQueryExecutor.cpp +++ b/dbms/src/Debug/dbgQueryExecutor.cpp @@ -12,12 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include - +#include +#include +#include +#include namespace DB { +using TiFlashTestEnv = tests::TiFlashTestEnv; + void setTipbRegionInfo(coprocessor::RegionInfo * tipb_region_info, const std::pair & region, TableID table_id) { tipb_region_info->set_region_id(region.first); diff --git a/dbms/src/Debug/dbgQueryExecutor.h b/dbms/src/Debug/dbgQueryExecutor.h index 0b3c639a20c..aa308ada9fc 100644 --- a/dbms/src/Debug/dbgQueryExecutor.h +++ b/dbms/src/Debug/dbgQueryExecutor.h @@ -18,7 +18,6 @@ namespace DB { using MockServerConfig = tests::MockServerConfig; -using TiFlashTestEnv = tests::TiFlashTestEnv; BlockInputStreamPtr executeQuery(Context & context, RegionID region_id, const DAGProperties & properties, QueryTasks & query_tasks, MakeResOutputStream & func_wrap_output_stream); BlockInputStreamPtr executeMPPQuery(Context & context, const DAGProperties & properties, QueryTasks & query_tasks); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 2902d66b57a..34fd53c3455 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -176,7 +176,17 @@ void DAGQueryBlockInterpreter::handleMockTableScan(const TiDBTableScan & table_s } else { - auto [names_and_types, mock_table_scan_streams] = mockSourceStream(context, max_streams, log, table_scan.getTableScanExecutorID(), table_scan.getLogicalTableID()); + NamesAndTypes names_and_types; + std::vector> mock_table_scan_streams; + if (context.isMPPTest()) + { + std::tie(names_and_types, mock_table_scan_streams) = mockSourceStreamForMpp(context, max_streams, log, table_scan); + } + else + { + std::tie(names_and_types, mock_table_scan_streams) = mockSourceStream(context, max_streams, log, table_scan.getTableScanExecutorID(), table_scan.getLogicalTableID()); + } + analyzer = std::make_unique(std::move(names_and_types), context); pipeline.streams.insert(pipeline.streams.end(), mock_table_scan_streams.begin(), mock_table_scan_streams.end()); } diff --git a/dbms/src/Flash/Coprocessor/MockSourceStream.cpp b/dbms/src/Flash/Coprocessor/MockSourceStream.cpp new file mode 100644 index 00000000000..c8e662adc32 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/MockSourceStream.cpp @@ -0,0 +1,24 @@ +// 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 + +namespace DB +{ +std::pair>> mockSourceStreamForMpp(Context & context, size_t max_streams, DB::LoggerPtr log, const TiDBTableScan & table_scan) +{ + ColumnsWithTypeAndName columns_with_type_and_name = context.mockStorage().getColumnsForMPPTableScan(table_scan, context.mockMPPServerInfo().partition_id, context.mockMPPServerInfo().partition_num); + return cutStreams(context, columns_with_type_and_name, max_streams, log); +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/MockSourceStream.h b/dbms/src/Flash/Coprocessor/MockSourceStream.h index 7cb0ffc95e7..c84d37d2a06 100644 --- a/dbms/src/Flash/Coprocessor/MockSourceStream.h +++ b/dbms/src/Flash/Coprocessor/MockSourceStream.h @@ -18,25 +18,19 @@ #include #include #include +#include #include +#include + namespace DB { - template -std::pair>> mockSourceStream(Context & context, size_t max_streams, DB::LoggerPtr log, String executor_id, Int64 table_id = 0) +std::pair>> cutStreams(Context & context, ColumnsWithTypeAndName & columns_with_type_and_name, size_t max_streams, DB::LoggerPtr log) { - ColumnsWithTypeAndName columns_with_type_and_name; NamesAndTypes names_and_types; size_t rows = 0; std::vector> mock_source_streams; - if constexpr (std::is_same_v) - columns_with_type_and_name = context.mockStorage().getExchangeColumns(executor_id); - else if (context.isMPPTest()) - columns_with_type_and_name = context.mockStorage().getColumnsForMPPTableScan(table_id, context.mockMPPServerInfo().partition_id, context.mockMPPServerInfo().partition_num); - else - columns_with_type_and_name = context.mockStorage().getColumns(table_id); - for (const auto & col : columns_with_type_and_name) { if (rows == 0) @@ -68,4 +62,18 @@ std::pair>> mockSourceStr RUNTIME_ASSERT(start == rows, log, "mock source streams' total size must same as user input"); return {names_and_types, mock_source_streams}; } + +std::pair>> mockSourceStreamForMpp(Context & context, size_t max_streams, DB::LoggerPtr log, const TiDBTableScan & table_scan); + +template +std::pair>> mockSourceStream(Context & context, size_t max_streams, DB::LoggerPtr log, String executor_id, Int64 table_id = 0) +{ + ColumnsWithTypeAndName columns_with_type_and_name; + if constexpr (std::is_same_v) + columns_with_type_and_name = context.mockStorage().getExchangeColumns(executor_id); + else + columns_with_type_and_name = context.mockStorage().getColumns(table_id); + + return cutStreams(context, columns_with_type_and_name, max_streams, log); +} } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp index c3019f5ed8b..23833b9af0b 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp @@ -50,7 +50,16 @@ std::pair mockSchemaAndStreams( else { /// build from user input blocks. - auto [names_and_types, mock_table_scan_streams] = mockSourceStream(context, max_streams, log, executor_id, table_scan.getLogicalTableID()); + NamesAndTypes names_and_types; + std::vector> mock_table_scan_streams; + if (context.isMPPTest()) + { + std::tie(names_and_types, mock_table_scan_streams) = mockSourceStreamForMpp(context, max_streams, log, table_scan); + } + else + { + std::tie(names_and_types, mock_table_scan_streams) = mockSourceStream(context, max_streams, log, executor_id, table_scan.getLogicalTableID()); + } schema = std::move(names_and_types); mock_streams.insert(mock_streams.end(), mock_table_scan_streams.begin(), mock_table_scan_streams.end()); } diff --git a/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp b/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp index 06710f4dc98..ab8c60a4ba3 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index 264db3ea876..e90e88f2289 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -290,6 +290,53 @@ try } CATCH +TEST_F(ComputeServerRunner, aggWithColumnPrune) +try +{ + startServers(3); + + context.addMockTable( + {"test_db", "test_table_2"}, + {{"i1", TiDB::TP::TypeLong}, {"i2", TiDB::TP::TypeLong}, {"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}, {"s4", TiDB::TP::TypeString}, {"s5", TiDB::TP::TypeString}}, + {toNullableVec("i1", {0, 0, 0}), toNullableVec("i2", {1, 1, 1}), toNullableVec("s1", {"1", "9", "8"}), toNullableVec("s2", {"1", "9", "8"}), toNullableVec("s3", {"4", "9", "99"}), toNullableVec("s4", {"4", "9", "999"}), toNullableVec("s5", {"4", "9", "9999"})}); + std::vector res{"9", "9", "99", "999", "9999"}; + std::vector max_cols{"s1", "s2", "s3", "s4", "s5"}; + for (size_t i = 0; i < 1; ++i) + { + { + auto request = context + .scan("test_db", "test_table_2") + .aggregation({Max(col(max_cols[i]))}, {col("i1")}); + auto expected_cols = { + toNullableVec({res[i]}), + toNullableVec({{0}})}; + ASSERT_COLUMNS_EQ_UR(expected_cols, buildAndExecuteMPPTasks(request)); + } + + { + auto request = context + .scan("test_db", "test_table_2") + .aggregation({Max(col(max_cols[i]))}, {col("i2")}); + auto expected_cols = { + toNullableVec({res[i]}), + toNullableVec({{1}})}; + ASSERT_COLUMNS_EQ_UR(expected_cols, buildAndExecuteMPPTasks(request)); + } + + { + auto request = context + .scan("test_db", "test_table_2") + .aggregation({Max(col(max_cols[i]))}, {col("i1"), col("i2")}); + auto expected_cols = { + toNullableVec({res[i]}), + toNullableVec({{0}}), + toNullableVec({{1}})}; + ASSERT_COLUMNS_EQ_UR(expected_cols, buildAndExecuteMPPTasks(request)); + } + } +} +CATCH + TEST_F(ComputeServerRunner, cancelAggTasks) try { diff --git a/dbms/src/Server/FlashGrpcServerHolder.cpp b/dbms/src/Server/FlashGrpcServerHolder.cpp index 1aeff0a49c6..0ab13e2dd85 100644 --- a/dbms/src/Server/FlashGrpcServerHolder.cpp +++ b/dbms/src/Server/FlashGrpcServerHolder.cpp @@ -11,6 +11,7 @@ // 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 diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp index af8c8bed4ba..2c993ec91ea 100644 --- a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp @@ -13,13 +13,14 @@ // limitations under the License. #include +#include #include #include #include +#include #include #include - namespace DB { namespace tests diff --git a/dbms/src/TestUtils/MPPTaskTestUtils.cpp b/dbms/src/TestUtils/MPPTaskTestUtils.cpp index 6d7275a19a0..d33ae8e5910 100644 --- a/dbms/src/TestUtils/MPPTaskTestUtils.cpp +++ b/dbms/src/TestUtils/MPPTaskTestUtils.cpp @@ -11,7 +11,12 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include +#include #include +#include #include namespace DB::tests @@ -80,7 +85,7 @@ std::tuple> MPPTaskTestUtils::prepa return {MPPQueryId(properties.query_ts, properties.local_query_id, properties.server_id, properties.start_ts), res}; } -ColumnsWithTypeAndName MPPTaskTestUtils::exeucteMPPTasks(QueryTasks & tasks, const DAGProperties & properties, std::unordered_map & server_config_map) +ColumnsWithTypeAndName MPPTaskTestUtils::executeMPPTasks(QueryTasks & tasks, const DAGProperties & properties, std::unordered_map & server_config_map) { auto res = executeMPPQueryWithMultipleContext(properties, tasks, server_config_map); return readBlocks(res); @@ -145,8 +150,8 @@ ::testing::AssertionResult MPPTaskTestUtils::assertQueryCancelled(const MPPQuery { std::this_thread::sleep_for(seconds); retry_times++; - // Currenly we wait for 10 times to ensure all tasks are cancelled. - if (retry_times > 10) + // Currenly we wait for 20 times to ensure all tasks are cancelled. + if (retry_times > 20) { return ::testing::AssertionFailure() << "Query not cancelled, " << queryInfo(i) << std::endl; } @@ -166,4 +171,15 @@ ::testing::AssertionResult MPPTaskTestUtils::assertQueryActive(const MPPQueryId } return ::testing::AssertionSuccess(); } + +ColumnsWithTypeAndName MPPTaskTestUtils::buildAndExecuteMPPTasks(DAGRequestBuilder builder) +{ + auto properties = DB::tests::getDAGPropertiesForTest(serverNum()); + for (int i = 0; i < TiFlashTestEnv::globalContextSize(); ++i) + TiFlashTestEnv::getGlobalContext(i).setMPPTest(); + auto tasks = (builder).buildMPPTasks(context, properties); + MockComputeServerManager::instance().resetMockMPPServerInfo(serverNum()); + MockComputeServerManager::instance().setMockStorage(context.mockStorage()); + return executeMPPTasks(tasks, properties, MockComputeServerManager::instance().getServerConfigMap()); +} } // namespace DB::tests diff --git a/dbms/src/TestUtils/MPPTaskTestUtils.h b/dbms/src/TestUtils/MPPTaskTestUtils.h index cb0e84a2a14..75330ed0c6d 100644 --- a/dbms/src/TestUtils/MPPTaskTestUtils.h +++ b/dbms/src/TestUtils/MPPTaskTestUtils.h @@ -15,11 +15,7 @@ #pragma once #include -#include -#include -#include #include -#include namespace DB::tests { @@ -50,12 +46,12 @@ class MockServerAddrGenerator : public ext::Singleton void reset() { - port = 3931; + port = 4931; } private: const Int64 port_upper_bound = 65536; - std::atomic port = 3931; + std::atomic port = 4931; }; // Hold MPP test related infomation: @@ -82,12 +78,14 @@ class MPPTaskTestUtils : public ExecutorTest // run mpp tasks which are ready to cancel, the return value is the start_ts of query. std::tuple> prepareMPPStreams(DAGRequestBuilder builder); - ColumnsWithTypeAndName exeucteMPPTasks(QueryTasks & tasks, const DAGProperties & properties, std::unordered_map & server_config_map); + static ColumnsWithTypeAndName executeMPPTasks(QueryTasks & tasks, const DAGProperties & properties, std::unordered_map & server_config_map); + ColumnsWithTypeAndName buildAndExecuteMPPTasks(DAGRequestBuilder builder); ColumnsWithTypeAndName executeCoprocessorTask(std::shared_ptr & dag_request); static ::testing::AssertionResult assertQueryCancelled(const MPPQueryId & query_id); static ::testing::AssertionResult assertQueryActive(const MPPQueryId & query_id); + static String queryInfo(size_t server_id); protected: @@ -101,7 +99,7 @@ class MPPTaskTestUtils : public ExecutorTest { \ TiFlashTestEnv::getGlobalContext().setMPPTest(); \ MockComputeServerManager::instance().setMockStorage(context.mockStorage()); \ - ASSERT_COLUMNS_EQ_UR(exeucteMPPTasks(tasks, properties, MockComputeServerManager::instance().getServerConfigMap()), expected_cols); \ + ASSERT_COLUMNS_EQ_UR(expected_cols, executeMPPTasks(tasks, properties, MockComputeServerManager::instance().getServerConfigMap())); \ } while (0) diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index e6c9a82a231..59000185cdf 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -13,6 +13,17 @@ // limitations under the License. #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -261,18 +272,19 @@ DAGRequestBuilder & DAGRequestBuilder::exchangeSender(tipb::ExchangeType exchang return *this; } -DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, - tipb::JoinType tp, - MockAstVec join_cols, - MockAstVec left_conds, - MockAstVec right_conds, - MockAstVec other_conds, - MockAstVec other_eq_conds_from_in) +DAGRequestBuilder & DAGRequestBuilder::join( + const DAGRequestBuilder & right, + tipb::JoinType tp, + MockAstVec join_col_exprs, + MockAstVec left_conds, + MockAstVec right_conds, + MockAstVec other_conds, + MockAstVec other_eq_conds_from_in) { assert(root); assert(right.root); - root = mock::compileJoin(getExecutorIndex(), root, right.root, tp, join_cols, left_conds, right_conds, other_conds, other_eq_conds_from_in); + root = mock::compileJoin(getExecutorIndex(), root, right.root, tp, join_col_exprs, left_conds, right_conds, other_conds, other_eq_conds_from_in); return *this; } @@ -385,6 +397,7 @@ void MockDAGRequestContext::addMockTable(const String & db, const String & table void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns) { + assert(columnInfos.size() == columns.size()); addMockTable(name, columnInfos); addMockTableColumnData(name, columns); } diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 11c09caf4cf..14b314d9c20 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include From 08660589aeb3826ac9b3922adb72534edd619a20 Mon Sep 17 00:00:00 2001 From: hehechen Date: Thu, 22 Dec 2022 12:00:55 +0800 Subject: [PATCH 34/42] Fix stale read in testing scenario (#6526) close pingcap/tiflash#6525 --- dbms/src/Storages/Transaction/LearnerRead.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/LearnerRead.cpp b/dbms/src/Storages/Transaction/LearnerRead.cpp index bc9cde099fb..7a05b8d4508 100644 --- a/dbms/src/Storages/Transaction/LearnerRead.cpp +++ b/dbms/src/Storages/Transaction/LearnerRead.cpp @@ -219,7 +219,7 @@ LearnerReadSnapshot doLearnerRead( const auto & region_to_query = regions_info[region_idx]; const RegionID region_id = region_to_query.region_id; UInt64 physical_tso = read_index_tso >> TsoPhysicalShiftBits; - bool can_stale_read = physical_tso < region_table.getSelfSafeTS(region_id); + bool can_stale_read = mvcc_query_info->read_tso != std::numeric_limits::max() && physical_tso < region_table.getSelfSafeTS(region_id); if (!can_stale_read) { if (auto ori_read_index = mvcc_query_info.getReadIndexRes(region_id); ori_read_index) From ba882a33c46332e6bb190ce1a3d24c2b99ea7111 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 22 Dec 2022 15:50:55 +0800 Subject: [PATCH 35/42] refine remote execution summary (#6349) ref pingcap/tiflash#5900 --- .../DataStreams/TiRemoteBlockInputStream.h | 88 +-------- dbms/src/Flash/Coprocessor/DAGContext.h | 1 + .../Flash/Coprocessor/ExecutionSummary.cpp | 40 ++-- dbms/src/Flash/Coprocessor/ExecutionSummary.h | 7 +- .../Coprocessor/ExecutionSummaryCollector.cpp | 175 +++++++----------- .../Coprocessor/ExecutionSummaryCollector.h | 19 +- .../Coprocessor/RemoteExecutionSummary.cpp | 59 ++++++ .../Coprocessor/RemoteExecutionSummary.h | 33 ++++ .../gtest_ti_remote_block_inputstream.cpp | 9 +- .../Flash/tests/gtest_execution_summary.cpp | 140 ++++++++++++++ dbms/src/Storages/DeltaMerge/ScanContext.h | 11 ++ dbms/src/TestUtils/ExecutorTestUtils.cpp | 11 +- dbms/src/TestUtils/ExecutorTestUtils.h | 2 + 13 files changed, 379 insertions(+), 216 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/RemoteExecutionSummary.cpp create mode 100644 dbms/src/Flash/Coprocessor/RemoteExecutionSummary.h create mode 100644 dbms/src/Flash/tests/gtest_execution_summary.cpp diff --git a/dbms/src/DataStreams/TiRemoteBlockInputStream.h b/dbms/src/DataStreams/TiRemoteBlockInputStream.h index a0b90464dff..124f08d65c4 100644 --- a/dbms/src/DataStreams/TiRemoteBlockInputStream.h +++ b/dbms/src/DataStreams/TiRemoteBlockInputStream.h @@ -19,8 +19,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -50,15 +50,10 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream String name; - /// this atomic variable is kind of a lock for the struct of execution_summaries: - /// if execution_summaries_inited[index] = true, the map execution_summaries[index] - /// itself will not be modified, so ExecutionSummaryCollector can read it safely, otherwise, - /// ExecutionSummaryCollector will just skip execution_summaries[index] - std::vector> execution_summaries_inited; - std::vector> execution_summaries; - const LoggerPtr log; + RemoteExecutionSummary remote_execution_summary; + uint64_t total_rows; // For fine grained shuffle, sender will partition data into muiltiple streams by hashing. @@ -68,64 +63,6 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream std::unique_ptr decoder_ptr; - void initRemoteExecutionSummaries(tipb::SelectResponse & resp, size_t index) - { - for (const auto & execution_summary : resp.execution_summaries()) - { - if (likely(execution_summary.has_executor_id())) - { - auto & remote_execution_summary = execution_summaries[index][execution_summary.executor_id()]; - remote_execution_summary.time_processed_ns = execution_summary.time_processed_ns(); - remote_execution_summary.num_produced_rows = execution_summary.num_produced_rows(); - remote_execution_summary.num_iterations = execution_summary.num_iterations(); - remote_execution_summary.concurrency = execution_summary.concurrency(); - DM::ScanContext scan_context; - scan_context.deserialize(execution_summary.tiflash_scan_context()); - remote_execution_summary.scan_context->merge(scan_context); - } - } - execution_summaries_inited[index].store(true); - } - - void addRemoteExecutionSummaries(tipb::SelectResponse & resp, size_t index) - { - if (unlikely(resp.execution_summaries_size() == 0)) - return; - - if (!execution_summaries_inited[index].load()) - { - initRemoteExecutionSummaries(resp, index); - return; - } - if constexpr (is_streaming_reader) - throw Exception( - fmt::format( - "There are more than one execution summary packet of index {} in streaming reader, " - "this should not happen", - index)); - auto & execution_summaries_map = execution_summaries[index]; - for (const auto & execution_summary : resp.execution_summaries()) - { - if (likely(execution_summary.has_executor_id())) - { - const auto & executor_id = execution_summary.executor_id(); - if (unlikely(execution_summaries_map.find(executor_id) == execution_summaries_map.end())) - { - LOG_WARNING(log, "execution {} not found in execution_summaries, this should not happen", executor_id); - continue; - } - auto & remote_execution_summary = execution_summaries_map[executor_id]; - remote_execution_summary.time_processed_ns = std::max(remote_execution_summary.time_processed_ns, execution_summary.time_processed_ns()); - remote_execution_summary.num_produced_rows += execution_summary.num_produced_rows(); - remote_execution_summary.num_iterations += execution_summary.num_iterations(); - remote_execution_summary.concurrency += execution_summary.concurrency(); - DM::ScanContext scan_context; - scan_context.deserialize(execution_summary.tiflash_scan_context()); - remote_execution_summary.scan_context->merge(scan_context); - } - } - } - bool fetchRemoteResult() { while (true) @@ -147,14 +84,13 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream throw Exception(result.resp->error().DebugString()); } - size_t index = 0; - if constexpr (is_streaming_reader) - index = result.call_index; - /// only the last response contains execution summaries if (result.resp != nullptr) - addRemoteExecutionSummaries(*result.resp, index); + remote_execution_summary.add(*result.resp); + size_t index = 0; + if constexpr (is_streaming_reader) + index = result.call_index; const auto & decode_detail = result.decode_detail; auto & connection_profile_info = connection_profile_infos[index]; connection_profile_info.packets += decode_detail.packets; @@ -179,16 +115,10 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream : remote_reader(remote_reader_) , source_num(remote_reader->getSourceNum()) , name(fmt::format("TiRemote({})", RemoteReader::name)) - , execution_summaries_inited(source_num) , log(Logger::get(name, req_id, executor_id)) , total_rows(0) , stream_id(stream_id_) { - 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(getColumnWithTypeAndName(toNamesAndTypes(remote_reader->getOutputSchema()))); static constexpr size_t squash_rows_limit = 8192; @@ -228,9 +158,9 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream return block; } - const std::unordered_map * getRemoteExecutionSummaries(size_t index) + const RemoteExecutionSummary & getRemoteExecutionSummary() { - return execution_summaries_inited[index].load() ? &execution_summaries[index] : nullptr; + return remote_execution_summary; } size_t getTotalRows() const { return total_rows; } diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index aaf218ba24e..ce8b93e92d2 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -193,6 +193,7 @@ class DAGContext , dummy_query_string(dag_request->DebugString()) , dummy_ast(makeDummyQuery()) , initialize_concurrency(concurrency) + , collect_execution_summaries(dag_request->has_collect_execution_summaries() && dag_request->collect_execution_summaries()) , is_mpp_task(true) , is_root_mpp_task(false) , log(Logger::get(log_identifier)) diff --git a/dbms/src/Flash/Coprocessor/ExecutionSummary.cpp b/dbms/src/Flash/Coprocessor/ExecutionSummary.cpp index a62693f1aec..818d0edfbea 100644 --- a/dbms/src/Flash/Coprocessor/ExecutionSummary.cpp +++ b/dbms/src/Flash/Coprocessor/ExecutionSummary.cpp @@ -16,24 +16,30 @@ namespace DB { +void ExecutionSummary::merge(const ExecutionSummary & other) +{ + time_processed_ns = std::max(time_processed_ns, other.time_processed_ns); + num_produced_rows += other.num_produced_rows; + num_iterations += other.num_iterations; + concurrency += other.concurrency; + scan_context->merge(*other.scan_context); +} + +void ExecutionSummary::merge(const tipb::ExecutorExecutionSummary & other) +{ + time_processed_ns = std::max(time_processed_ns, other.time_processed_ns()); + num_produced_rows += other.num_produced_rows(); + num_iterations += other.num_iterations(); + concurrency += other.concurrency(); + scan_context->merge(other.tiflash_scan_context()); +} -void ExecutionSummary::merge(const ExecutionSummary & other, bool streaming_call) +void ExecutionSummary::init(const tipb::ExecutorExecutionSummary & other) { - if (streaming_call) - { - time_processed_ns = std::max(time_processed_ns, other.time_processed_ns); - num_produced_rows = std::max(num_produced_rows, other.num_produced_rows); - num_iterations = std::max(num_iterations, other.num_iterations); - concurrency = std::max(concurrency, other.concurrency); - scan_context->merge(*other.scan_context); - } - else - { - time_processed_ns = std::max(time_processed_ns, other.time_processed_ns); - num_produced_rows += other.num_produced_rows; - num_iterations += other.num_iterations; - concurrency += other.concurrency; - scan_context->merge(*other.scan_context); - } + time_processed_ns = other.time_processed_ns(); + num_produced_rows = other.num_produced_rows(); + num_iterations = other.num_iterations(); + concurrency = other.concurrency(); + scan_context->deserialize(other.tiflash_scan_context()); } } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/ExecutionSummary.h b/dbms/src/Flash/Coprocessor/ExecutionSummary.h index eafeaeed292..5a8ce579a6e 100644 --- a/dbms/src/Flash/Coprocessor/ExecutionSummary.h +++ b/dbms/src/Flash/Coprocessor/ExecutionSummary.h @@ -16,6 +16,7 @@ #include #include +#include #include @@ -29,11 +30,13 @@ struct ExecutionSummary UInt64 num_iterations = 0; UInt64 concurrency = 0; - std::unique_ptr scan_context = std::make_unique(); + DM::ScanContextPtr scan_context = std::make_shared(); ExecutionSummary() = default; - void merge(const ExecutionSummary & other, bool streaming_call); + void merge(const ExecutionSummary & other); + void merge(const tipb::ExecutorExecutionSummary & other); + void init(const tipb::ExecutorExecutionSummary & other); }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/ExecutionSummaryCollector.cpp b/dbms/src/Flash/Coprocessor/ExecutionSummaryCollector.cpp index c21c839760c..86122e400b2 100644 --- a/dbms/src/Flash/Coprocessor/ExecutionSummaryCollector.cpp +++ b/dbms/src/Flash/Coprocessor/ExecutionSummaryCollector.cpp @@ -14,14 +14,31 @@ #include #include +#include #include -#include -#include - -#include +#include namespace DB { +namespace +{ +RemoteExecutionSummary getRemoteExecutionSummariesFromExchange(DAGContext & dag_context) +{ + RemoteExecutionSummary exchange_execution_summary; + for (const auto & map_entry : dag_context.getInBoundIOInputStreamsMap()) + { + for (const auto & stream_ptr : map_entry.second) + { + if (auto * exchange_receiver_stream_ptr = dynamic_cast(stream_ptr.get()); exchange_receiver_stream_ptr) + { + exchange_execution_summary.merge(exchange_receiver_stream_ptr->getRemoteExecutionSummary()); + } + } + } + return exchange_execution_summary; +} +} // namespace + void ExecutionSummaryCollector::fillTiExecutionSummary( tipb::ExecutorExecutionSummary * execution_summary, ExecutionSummary & current, @@ -37,29 +54,6 @@ void ExecutionSummaryCollector::fillTiExecutionSummary( execution_summary->set_executor_id(executor_id); } -template -void mergeRemoteExecuteSummaries( - RemoteBlockInputStream * input_stream, - std::unordered_map> & execution_summaries) -{ - size_t source_num = input_stream->getSourceNum(); - for (size_t s_index = 0; s_index < source_num; ++s_index) - { - auto remote_execution_summaries = input_stream->getRemoteExecutionSummaries(s_index); - if (remote_execution_summaries == nullptr) - continue; - bool is_streaming_call = input_stream->isStreamingCall(); - for (auto & p : *remote_execution_summaries) - { - if (execution_summaries[p.first].size() < source_num) - { - execution_summaries[p.first].resize(source_num); - } - execution_summaries[p.first][s_index].merge(p.second, is_streaming_call); - } - } -} - tipb::SelectResponse ExecutionSummaryCollector::genExecutionSummaryResponse() { tipb::SelectResponse response; @@ -67,89 +61,67 @@ tipb::SelectResponse ExecutionSummaryCollector::genExecutionSummaryResponse() return response; } -void ExecutionSummaryCollector::addExecuteSummaries(tipb::SelectResponse & response) +void ExecutionSummaryCollector::fillLocalExecutionSummary( + tipb::SelectResponse & response, + const String & executor_id, + const BlockInputStreams & streams, + const std::unordered_map & scan_context_map) const { - if (!dag_context.collect_execution_summaries) - return; - /// get executionSummary info from remote input streams - std::unordered_map> merged_remote_execution_summaries; - for (const auto & map_entry : dag_context.getInBoundIOInputStreamsMap()) + ExecutionSummary current; + /// part 1: local execution info + // get execution info from streams + for (const auto & stream_ptr : streams) { - for (const auto & stream_ptr : map_entry.second) + if (auto * p_stream = dynamic_cast(stream_ptr.get())) { - if (auto * exchange_receiver_stream_ptr = dynamic_cast(stream_ptr.get())) - { - mergeRemoteExecuteSummaries(exchange_receiver_stream_ptr, merged_remote_execution_summaries); - } - else if (auto * cop_stream_ptr = dynamic_cast(stream_ptr.get())) - { - mergeRemoteExecuteSummaries(cop_stream_ptr, merged_remote_execution_summaries); - } - else - { - /// local read input stream - } + current.time_processed_ns = std::max(current.time_processed_ns, p_stream->getProfileInfo().execution_time); + current.num_produced_rows += p_stream->getProfileInfo().rows; + current.num_iterations += p_stream->getProfileInfo().blocks; } + ++current.concurrency; } - - auto fill_execution_summary = [&](const String & executor_id, const BlockInputStreams & streams, const std::unordered_map & scan_context_map) { - ExecutionSummary current; - /// part 1: local execution info - // get execution info from streams - for (const auto & stream_ptr : streams) - { - if (auto * p_stream = dynamic_cast(stream_ptr.get())) - { - current.time_processed_ns = std::max(current.time_processed_ns, p_stream->getProfileInfo().execution_time); - current.num_produced_rows += p_stream->getProfileInfo().rows; - current.num_iterations += p_stream->getProfileInfo().blocks; - } - current.concurrency++; - } - // get execution info from scan_context - if (const auto & iter = scan_context_map.find(executor_id); iter != scan_context_map.end()) - { - current.scan_context->merge(*(iter->second)); - } - - /// part 2: remote execution info - if (merged_remote_execution_summaries.find(executor_id) != merged_remote_execution_summaries.end()) - { - for (auto & remote : merged_remote_execution_summaries[executor_id]) - current.merge(remote, false); - } - /// part 3: for join need to add the build time - /// In TiFlash, a hash join's build side is finished before probe side starts, - /// so the join probe side's running time does not include hash table's build time, - /// when construct ExecSummaries, we need add the build cost to probe executor - auto all_join_id_it = dag_context.getExecutorIdToJoinIdMap().find(executor_id); - if (all_join_id_it != dag_context.getExecutorIdToJoinIdMap().end()) + // get execution info from scan_context + if (const auto & iter = scan_context_map.find(executor_id); iter != scan_context_map.end()) + { + current.scan_context->merge(*(iter->second)); + } + /// part 2: for join need to add the build time + /// In TiFlash, a hash join's build side is finished before probe side starts, + /// so the join probe side's running time does not include hash table's build time, + /// when construct ExecSummaries, we need add the build cost to probe executor + auto all_join_id_it = dag_context.getExecutorIdToJoinIdMap().find(executor_id); + if (all_join_id_it != dag_context.getExecutorIdToJoinIdMap().end()) + { + for (const auto & join_executor_id : all_join_id_it->second) { - for (const auto & join_executor_id : all_join_id_it->second) + auto it = dag_context.getJoinExecuteInfoMap().find(join_executor_id); + if (it != dag_context.getJoinExecuteInfoMap().end()) { - auto it = dag_context.getJoinExecuteInfoMap().find(join_executor_id); - if (it != dag_context.getJoinExecuteInfoMap().end()) + UInt64 process_time_for_build = 0; + for (const auto & join_build_stream : it->second.join_build_streams) { - UInt64 process_time_for_build = 0; - for (const auto & join_build_stream : it->second.join_build_streams) - { - if (auto * p_stream = dynamic_cast(join_build_stream.get()); p_stream) - process_time_for_build = std::max(process_time_for_build, p_stream->getProfileInfo().execution_time); - } - current.time_processed_ns += process_time_for_build; + if (auto * p_stream = dynamic_cast(join_build_stream.get()); p_stream) + process_time_for_build = std::max(process_time_for_build, p_stream->getProfileInfo().execution_time); } + current.time_processed_ns += process_time_for_build; } } + } - current.time_processed_ns += dag_context.compile_time_ns; - fillTiExecutionSummary(response.add_execution_summaries(), current, executor_id); - }; + current.time_processed_ns += dag_context.compile_time_ns; + fillTiExecutionSummary(response.add_execution_summaries(), current, executor_id); +} - /// add execution_summary for local executor +void ExecutionSummaryCollector::addExecuteSummaries(tipb::SelectResponse & response) +{ + if (!dag_context.collect_execution_summaries) + return; + + /// fill execution_summary for local executor if (dag_context.return_executor_id) { for (auto & p : dag_context.getProfileStreamsMap()) - fill_execution_summary(p.first, p.second, dag_context.scan_context_map); + fillLocalExecutionSummary(response, p.first, p.second, dag_context.scan_context_map); } else { @@ -159,19 +131,16 @@ void ExecutionSummaryCollector::addExecuteSummaries(tipb::SelectResponse & respo { auto it = profile_streams_map.find(executor_id); assert(it != profile_streams_map.end()); - fill_execution_summary(executor_id, it->second, dag_context.scan_context_map); + fillLocalExecutionSummary(response, executor_id, it->second, dag_context.scan_context_map); } } - for (auto & p : merged_remote_execution_summaries) + // TODO support cop remote read and disaggregated mode. + auto exchange_execution_summary = getRemoteExecutionSummariesFromExchange(dag_context); + // fill execution_summary to reponse for remote executor received by exchange. + for (auto & p : exchange_execution_summary.execution_summaries) { - if (local_executors.find(p.first) == local_executors.end()) - { - ExecutionSummary merged; - for (auto & remote : p.second) - merged.merge(remote, false); - fillTiExecutionSummary(response.add_execution_summaries(), merged, p.first); - } + fillTiExecutionSummary(response.add_execution_summaries(), p.second, p.first); } } } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/ExecutionSummaryCollector.h b/dbms/src/Flash/Coprocessor/ExecutionSummaryCollector.h index dedd488d125..dc5a64e723b 100644 --- a/dbms/src/Flash/Coprocessor/ExecutionSummaryCollector.h +++ b/dbms/src/Flash/Coprocessor/ExecutionSummaryCollector.h @@ -14,23 +14,21 @@ #pragma once -#include +#include #include +#include namespace DB { +class DAGContext; + class ExecutionSummaryCollector { public: explicit ExecutionSummaryCollector( DAGContext & dag_context_) : dag_context(dag_context_) - { - for (auto & p : dag_context.getProfileStreamsMap()) - { - local_executors.insert(p.first); - } - } + {} void addExecuteSummaries(tipb::SelectResponse & response); @@ -42,8 +40,13 @@ class ExecutionSummaryCollector ExecutionSummary & current, const String & executor_id) const; + void fillLocalExecutionSummary( + tipb::SelectResponse & response, + const String & executor_id, + const BlockInputStreams & streams, + const std::unordered_map & scan_context_map) const; + private: DAGContext & dag_context; - std::unordered_set local_executors; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/RemoteExecutionSummary.cpp b/dbms/src/Flash/Coprocessor/RemoteExecutionSummary.cpp new file mode 100644 index 00000000000..fc88afcf700 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/RemoteExecutionSummary.cpp @@ -0,0 +1,59 @@ +// 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 +{ +void RemoteExecutionSummary::merge(const RemoteExecutionSummary & other) +{ + for (const auto & p : other.execution_summaries) + { + const auto & executor_id = p.first; + auto it = execution_summaries.find(executor_id); + if (unlikely(it == execution_summaries.end())) + { + execution_summaries[executor_id] = p.second; + } + else + { + it->second.merge(p.second); + } + } +} + +void RemoteExecutionSummary::add(tipb::SelectResponse & resp) +{ + if (unlikely(resp.execution_summaries_size() == 0)) + return; + + for (const auto & execution_summary : resp.execution_summaries()) + { + if (likely(execution_summary.has_executor_id())) + { + const auto & executor_id = execution_summary.executor_id(); + auto it = execution_summaries.find(executor_id); + if (unlikely(it == execution_summaries.end())) + { + execution_summaries[executor_id].init(execution_summary); + } + else + { + it->second.merge(execution_summary); + } + } + } +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/RemoteExecutionSummary.h b/dbms/src/Flash/Coprocessor/RemoteExecutionSummary.h new file mode 100644 index 00000000000..dd2a9d0b5bf --- /dev/null +++ b/dbms/src/Flash/Coprocessor/RemoteExecutionSummary.h @@ -0,0 +1,33 @@ +// 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. + +#pragma once + +#include +#include + +#include + +namespace DB +{ +struct RemoteExecutionSummary +{ + void merge(const RemoteExecutionSummary & other); + + void add(tipb::SelectResponse & resp); + + // + std::unordered_map execution_summaries; +}; +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp index 2434feeba26..0162b940ce4 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp @@ -398,11 +398,10 @@ class TestTiRemoteBlockInputStream : public testing::Test { assert(receiver_stream); /// Check Execution Summary - const auto * summary = receiver_stream->getRemoteExecutionSummaries(0); - ASSERT_TRUE(summary != nullptr); - ASSERT_EQ(summary->size(), 1); - ASSERT_EQ(summary->begin()->first, "Executor_0"); - ASSERT_TRUE(equalSummaries(writer->mockExecutionSummary(), summary->begin()->second)); + const auto & summary = receiver_stream->getRemoteExecutionSummary(); + ASSERT_EQ(summary.execution_summaries.size(), 1); + ASSERT_EQ(summary.execution_summaries.begin()->first, "Executor_0"); + ASSERT_TRUE(equalSummaries(writer->mockExecutionSummary(), summary.execution_summaries.begin()->second)); /// Check Connection Info auto infos = receiver_stream->getConnectionProfileInfos(); diff --git a/dbms/src/Flash/tests/gtest_execution_summary.cpp b/dbms/src/Flash/tests/gtest_execution_summary.cpp new file mode 100644 index 00000000000..e010cfb5a53 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_execution_summary.cpp @@ -0,0 +1,140 @@ +// 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 + +namespace DB +{ +namespace tests +{ +class ExecutionSummaryTestRunner : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + context.addMockTable({"test_db", "test_table"}, + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana", "banana", {}, "banana", "banana", {}, "banana", "banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana", "apple", {}, "banana", "apple", {}, "banana", "apple", {}, "banana"})}); + context.addExchangeReceiver("test_exchange", + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana", "banana", {}, "banana", "banana", {}, "banana", "banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana", "apple", {}, "banana", "apple", {}, "banana", "apple", {}, "banana"})}); + } + + static constexpr size_t concurrency = 10; + static constexpr int not_check_rows = -1; + // + using ProfileInfo = std::pair; + using Expect = std::unordered_map; + void testForExecutionSummary( + const std::shared_ptr & request, + const Expect & expect) + { + request->set_collect_execution_summaries(true); + DAGContext dag_context(*request, "test_execution_summary", concurrency); + executeStreams(&dag_context); + ASSERT_EQ(dag_context.getProfileStreamsMap().size(), expect.size()); + ASSERT_TRUE(dag_context.collect_execution_summaries); + ExecutionSummaryCollector summary_collector(dag_context); + auto summaries = summary_collector.genExecutionSummaryResponse().execution_summaries(); + ASSERT_EQ(summaries.size(), expect.size()); + for (const auto & summary : summaries) + { + ASSERT_TRUE(summary.has_executor_id()); + auto it = expect.find(summary.executor_id()); + ASSERT_TRUE(it != expect.end()) << fmt::format("unknown executor_id: {}", summary.executor_id()); + if (it->second.first != not_check_rows) + ASSERT_EQ(summary.num_produced_rows(), it->second.first) << fmt::format("executor_id: {}", summary.executor_id()); + ASSERT_EQ(summary.concurrency(), it->second.second) << fmt::format("executor_id: {}", summary.executor_id()); + // time_processed_ns, num_iterations and tiflash_scan_context are not checked here. + } + } +}; + +TEST_F(ExecutionSummaryTestRunner, test) +try +{ + { + auto request = context + .scan("test_db", "test_table") + .filter(eq(col("s1"), col("s2"))) + .build(context); + Expect expect{{"table_scan_0", {12, concurrency}}, {"selection_1", {4, concurrency}}}; + testForExecutionSummary(request, expect); + } + { + auto request = context + .scan("test_db", "test_table") + .limit(5) + .build(context); + Expect expect{{"table_scan_0", {not_check_rows, concurrency}}, {"limit_1", {5, 1}}}; + testForExecutionSummary(request, expect); + } + { + auto request = context + .scan("test_db", "test_table") + .topN("s1", true, 5) + .build(context); + Expect expect{{"table_scan_0", {not_check_rows, concurrency}}, {"topn_1", {5, 1}}}; + testForExecutionSummary(request, expect); + } + { + auto request = context + .scan("test_db", "test_table") + .project({col("s2")}) + .build(context); + Expect expect{{"table_scan_0", {12, concurrency}}, {"project_1", {12, concurrency}}}; + testForExecutionSummary(request, expect); + } + { + auto request = context + .scan("test_db", "test_table") + .aggregation({col("s2")}, {col("s2")}) + .build(context); + Expect expect{{"table_scan_0", {12, concurrency}}, {"aggregation_1", {3, concurrency}}}; + testForExecutionSummary(request, expect); + } + { + auto t1 = context.scan("test_db", "test_table"); + auto t2 = context.scan("test_db", "test_table"); + auto request = t1.join(t2, tipb::JoinType::TypeInnerJoin, {col("s1")}).build(context); + Expect expect{{"table_scan_0", {12, concurrency}}, {"table_scan_1", {12, concurrency}}, {"Join_2", {64, concurrency}}}; + testForExecutionSummary(request, expect); + } + { + auto request = context + .receive("test_exchange") + .exchangeSender(tipb::Hash) + .build(context); + Expect expect{{"exchange_receiver_0", {12, concurrency}}, {"exchange_sender_1", {12, concurrency}}}; + testForExecutionSummary(request, expect); + } + { + auto request = context + .receive("test_exchange") + .sort({{"s1", false}, {"s2", false}, {"s1", false}, {"s2", false}}, true) + .window(RowNumber(), {"s1", false}, {"s2", false}, buildDefaultRowsFrame()) + .build(context); + Expect expect{{"exchange_receiver_0", {12, concurrency}}, {"sort_1", {12, 1}}, {"window_2", {12, 1}}}; + testForExecutionSummary(request, expect); + } +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/ScanContext.h b/dbms/src/Storages/DeltaMerge/ScanContext.h index 590223caef6..63d2081092c 100644 --- a/dbms/src/Storages/DeltaMerge/ScanContext.h +++ b/dbms/src/Storages/DeltaMerge/ScanContext.h @@ -81,6 +81,17 @@ class ScanContext total_dmfile_read_time_ms += other.total_dmfile_read_time_ms; total_create_snapshot_time_ms += other.total_create_snapshot_time_ms; } + + void merge(const tipb::TiFlashScanContext & other) + { + total_dmfile_scanned_packs += other.total_dmfile_scanned_packs(); + total_dmfile_skipped_packs += other.total_dmfile_skipped_packs(); + total_dmfile_scanned_rows += other.total_dmfile_scanned_rows(); + total_dmfile_skipped_rows += other.total_dmfile_skipped_rows(); + total_dmfile_rough_set_index_load_time_ms += other.total_dmfile_rough_set_index_load_time_ms(); + total_dmfile_read_time_ms += other.total_dmfile_read_time_ms(); + total_create_snapshot_time_ms += other.total_create_snapshot_time_ms(); + } }; using ScanContextPtr = std::shared_ptr; diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 90fa363fe4d..dc279f3ea7f 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -222,12 +222,19 @@ void ExecutorTest::enablePlanner(bool is_enable) context.context.setSetting("enable_planner", is_enable ? "true" : "false"); } -DB::ColumnsWithTypeAndName ExecutorTest::executeStreams(const std::shared_ptr & request, size_t concurrency) +DB::ColumnsWithTypeAndName ExecutorTest::executeStreams( + const std::shared_ptr & request, + size_t concurrency) { DAGContext dag_context(*request, "executor_test", concurrency); + return executeStreams(&dag_context); +} + +ColumnsWithTypeAndName ExecutorTest::executeStreams(DAGContext * dag_context) +{ context.context.setExecutorTest(); context.context.setMockStorage(context.mockStorage()); - context.context.setDAGContext(&dag_context); + context.context.setDAGContext(dag_context); // Currently, don't care about regions information in tests. Blocks blocks; queryExecute(context.context, /*internal=*/true)->execute([&blocks](const Block & block) { blocks.push_back(block); }).verify(); diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index acb239a96df..ee014e4b069 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -94,6 +94,8 @@ class ExecutorTest : public ::testing::Test } } + ColumnsWithTypeAndName executeStreams(DAGContext * dag_context); + ColumnsWithTypeAndName executeStreams( const std::shared_ptr & request, size_t concurrency = 1); From 87d7c9b1801c055e9f8789a4450111fd4a35f61f Mon Sep 17 00:00:00 2001 From: JaySon Date: Fri, 23 Dec 2022 12:06:14 +0800 Subject: [PATCH 36/42] *: Remove useless DataType widen property (#6530) ref pingcap/tiflash#6233 --- dbms/src/DataTypes/DataTypeNullable.cpp | 67 +++----------- dbms/src/DataTypes/DataTypeNullable.h | 16 ---- dbms/src/DataTypes/DataTypeNumberBase.cpp | 70 +------------- dbms/src/DataTypes/DataTypeNumberBase.h | 5 - dbms/src/DataTypes/DataTypesNumber.h | 8 -- dbms/src/DataTypes/IDataType.h | 74 --------------- dbms/src/Storages/MutableSupport.h | 9 -- dbms/src/Storages/Transaction/Datum.cpp | 19 ++-- dbms/src/Storages/Transaction/TiDB.cpp | 4 +- dbms/src/Storages/Transaction/TiDB.h | 62 ++++++------- dbms/src/Storages/Transaction/TypeMapping.cpp | 91 +++++++------------ .../Transaction/tests/gtest_type_mapping.cpp | 14 +-- 12 files changed, 99 insertions(+), 340 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index fbcda5065c3..c99ee2edf83 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -67,12 +67,12 @@ void DataTypeNullable::serializeBinaryBulkWithMultipleStreams( bool position_independent_encoding, SubstreamPath & path) const { - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); col.checkConsistency(); /// First serialize null map. path.push_back(Substream::NullMap); - if (auto stream = getter(path)) + if (auto * stream = getter(path)) DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit); /// Then serialize contents of arrays. @@ -89,10 +89,10 @@ void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( bool position_independent_encoding, SubstreamPath & path) const { - ColumnNullable & col = static_cast(column); + auto & col = static_cast(column); path.push_back(Substream::NullMap); - if (auto stream = getter(path)) + if (auto * stream = getter(path)) DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); path.back() = Substream::NullableElements; @@ -100,50 +100,9 @@ void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( } -void DataTypeNullable::serializeWidenBinaryBulkWithMultipleStreams( - const IColumn & column, - const OutputStreamGetter & getter, - size_t offset, - size_t limit, - bool position_independent_encoding, - SubstreamPath & path) const -{ - const ColumnNullable & col = static_cast(column); - col.checkConsistency(); - - /// First serialize null map. - path.push_back(Substream::NullMap); - if (auto stream = getter(path)) - DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit); - - /// Then serialize contents of arrays. - path.back() = Substream::NullableElements; - nested_data_type->serializeWidenBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, offset, limit, position_independent_encoding, path); -} - - -void DataTypeNullable::deserializeWidenBinaryBulkWithMultipleStreams( - IColumn & column, - const InputStreamGetter & getter, - size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath & path) const -{ - ColumnNullable & col = static_cast(column); - - path.push_back(Substream::NullMap); - if (auto stream = getter(path)) - DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); - - path.back() = Substream::NullableElements; - nested_data_type->deserializeWidenBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, limit, avg_value_size_hint, position_independent_encoding, path); -} - - void DataTypeNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); bool is_null = col.isNullAt(row_num); writeBinary(is_null, ostr); @@ -159,7 +118,7 @@ static void safeDeserialize( CheckForNull && check_for_null, DeserializeNested && deserialize_nested) { - ColumnNullable & col = static_cast(column); + auto & col = static_cast(column); if (check_for_null()) { @@ -186,14 +145,14 @@ void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) co { safeDeserialize( column, - [&istr] { bool is_null = 0; readBinary(is_null, istr); return is_null; }, + [&istr] { bool is_null = false; readBinary(is_null, istr); return is_null; }, [this, &istr](IColumn & nested) { nested_data_type->deserializeBinary(nested, istr); }); } void DataTypeNullable::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); if (col.isNullAt(row_num)) writeCString("\\N", ostr); @@ -261,7 +220,7 @@ void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & ist void DataTypeNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); if (col.isNullAt(row_num)) writeCString("NULL", ostr); @@ -280,7 +239,7 @@ void DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); if (col.isNullAt(row_num)) writeCString("\\N", ostr); @@ -298,7 +257,7 @@ void DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); if (col.isNullAt(row_num)) writeCString("NULL", ostr); @@ -308,7 +267,7 @@ void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, Wri void DataTypeNullable::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettingsJSON & settings) const { - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); if (col.isNullAt(row_num)) writeCString("null", ostr); @@ -326,7 +285,7 @@ void DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr) void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); if (col.isNullAt(row_num)) writeCString("\\N", ostr); diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index 711aabfb905..1c7d5c9d2ab 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -50,22 +50,6 @@ class DataTypeNullable final : public IDataType bool position_independent_encoding, SubstreamPath & path) const override; - void serializeWidenBinaryBulkWithMultipleStreams( - const IColumn & column, - const OutputStreamGetter & getter, - size_t offset, - size_t limit, - bool position_independent_encoding, - SubstreamPath & path) const override; - - void deserializeWidenBinaryBulkWithMultipleStreams( - IColumn & column, - const InputStreamGetter & getter, - size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath & path) const override; - void serializeBinary(const Field & field, WriteBuffer & ostr) const override { nested_data_type->serializeBinary(field, ostr); } void deserializeBinary(Field & field, ReadBuffer & istr) const override { nested_data_type->deserializeBinary(field, istr); } void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; diff --git a/dbms/src/DataTypes/DataTypeNumberBase.cpp b/dbms/src/DataTypes/DataTypeNumberBase.cpp index 768e6aa7d1f..9b90a145852 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.cpp +++ b/dbms/src/DataTypes/DataTypeNumberBase.cpp @@ -42,7 +42,7 @@ void DataTypeNumberBase::serializeTextEscaped(const IColumn & column, size_t template static void deserializeText(IColumn & column, ReadBuffer & istr) { - T x; + T x{}; if constexpr (std::is_integral_v && std::is_arithmetic_v) readIntTextUnsafe(x, istr); @@ -198,7 +198,7 @@ template void DataTypeNumberBase::serializeBinary(const Field & field, WriteBuffer & ostr) const { /// ColumnVector::value_type is a narrower type. For example, UInt8, when the Field type is UInt64 - typename ColumnVector::value_type x = get::Type>(field); + auto x = get::Type>(field); writeBinary(x, ostr); } @@ -214,15 +214,6 @@ template void DataTypeNumberBase::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { writeBinary(static_cast &>(column).getData()[row_num], ostr); - // if (likely(widened)) - // { - // using WidestType = typename NearestFieldType::Type; - // writeBinary(static_cast(static_cast &>(column).getData()[row_num]), ostr); - // } - // else - // { - // writeBinary(static_cast &>(column).getData()[row_num], ostr); - // } } template @@ -231,18 +222,6 @@ void DataTypeNumberBase::deserializeBinary(IColumn & column, ReadBuffer & ist typename ColumnVector::value_type x; readBinary(x, istr); static_cast &>(column).getData().push_back(x); - // if (likely(widened)) - // { - // using WidestType = typename NearestFieldType::Type; - // typename ColumnVector::value_type y; - // readBinary(y, istr); - // x = static_cast(y); - // } - // else - // { - // readBinary(x, istr); - // } - // static_cast &>(column).getData().push_back(x); } template @@ -268,51 +247,6 @@ void DataTypeNumberBase::deserializeBinaryBulk(IColumn & column, ReadBuffer & x.resize(initial_size + size / sizeof(typename ColumnVector::value_type)); } -template -void DataTypeNumberBase::serializeWidenBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const -{ - if (!widened) - return serializeBinaryBulk(column, ostr, offset, limit); - - const typename ColumnVector::Container & x = typeid_cast &>(column).getData(); - - size_t size = x.size(); - - if (limit == 0 || offset + limit > size) - limit = size - offset; - - using WidestType = typename NearestFieldType::Type; - typename ColumnVector::Container y(limit); - for (size_t i = 0; i < limit; i++) - { - y[i] = static_cast(x[offset + i]); - } - - ostr.write(reinterpret_cast(&y[0]), sizeof(typename ColumnVector::value_type) * limit); -} - -template -void DataTypeNumberBase::deserializeWidenBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const -{ - if (!widened) - return deserializeBinaryBulk(column, istr, limit, avg_value_size_hint); - - typename ColumnVector::Container & x = typeid_cast &>(column).getData(); - size_t initial_size = x.size(); - x.resize(initial_size + limit); - - using WidestType = typename NearestFieldType::Type; - typename ColumnVector::Container y(limit); - size_t size = istr.readBig(reinterpret_cast(&y[0]), sizeof(typename ColumnVector::value_type) * limit); - size_t elem_size = size / sizeof(typename ColumnVector::value_type); - for (size_t i = 0; i < elem_size; i++) - { - x[initial_size + i] = static_cast(y[i]); - } - - x.resize(initial_size + elem_size); -} - template MutableColumnPtr DataTypeNumberBase::createColumn() const { diff --git a/dbms/src/DataTypes/DataTypeNumberBase.h b/dbms/src/DataTypes/DataTypeNumberBase.h index 6d8f3162981..4b75b954d0b 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.h +++ b/dbms/src/DataTypes/DataTypeNumberBase.h @@ -51,8 +51,6 @@ class DataTypeNumberBase : public IDataType void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - void serializeWidenBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; - void deserializeWidenBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; MutableColumnPtr createColumn() const override; @@ -67,9 +65,6 @@ class DataTypeNumberBase : public IDataType bool haveMaximumSizeOfValue() const override { return true; } size_t getSizeOfValueInMemory() const override { return sizeof(T); } bool isCategorial() const override { return isValueRepresentedByInteger(); } - -protected: - bool widened = false; }; } // namespace DB diff --git a/dbms/src/DataTypes/DataTypesNumber.h b/dbms/src/DataTypes/DataTypesNumber.h index 7ac1294473f..7021abea3ad 100644 --- a/dbms/src/DataTypes/DataTypesNumber.h +++ b/dbms/src/DataTypes/DataTypesNumber.h @@ -35,14 +35,6 @@ class DataTypeNumber final : public DataTypeNumberBase bool isInteger() const override { return std::is_integral_v; } bool isFloatingPoint() const override { return std::is_floating_point_v; } bool canBeInsideNullable() const override { return true; } - -public: - DataTypePtr widen() const override - { - auto t = std::make_shared>(); - t->widened = true; - return t; - } }; using DataTypeUInt8 = DataTypeNumber; diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 71fda0615e4..58f30600a72 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -179,71 +179,6 @@ class IDataType : private boost::noncopyable virtual void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const; virtual void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const; - /** Widen version for `serializeBinaryBulkWithMultipleStreams`. - */ - virtual void serializeWidenBinaryBulkWithMultipleStreams( - const IColumn & column, - const OutputStreamGetter & getter, - size_t offset, - size_t limit, - bool /*position_independent_encoding*/, - SubstreamPath & path) const - { - if (WriteBuffer * stream = getter(path)) - serializeWidenBinaryBulk(column, *stream, offset, limit); - } - - void serializeWidenBinaryBulkWithMultipleStreams( - const IColumn & column, - const OutputStreamGetter & getter, - size_t offset, - size_t limit, - bool position_independent_encoding, - SubstreamPath && path) const - { - serializeWidenBinaryBulkWithMultipleStreams(column, getter, offset, limit, position_independent_encoding, path); - } - - - /** Widen version for `deserializeBinaryBulkWithMultipleStreams`. - */ - virtual void deserializeWidenBinaryBulkWithMultipleStreams( - IColumn & column, - const InputStreamGetter & getter, - size_t limit, - double avg_value_size_hint, - bool /*position_independent_encoding*/, - SubstreamPath & path) const - { - if (ReadBuffer * stream = getter(path)) - deserializeWidenBinaryBulk(column, *stream, limit, avg_value_size_hint); - } - - void deserializeWidenBinaryBulkWithMultipleStreams( - IColumn & column, - const InputStreamGetter & getter, - size_t limit, - double avg_value_size_hint, - bool position_independent_encoding, - SubstreamPath && path) const - { - deserializeWidenBinaryBulkWithMultipleStreams(column, getter, limit, avg_value_size_hint, position_independent_encoding, path); - } - - /** Widen version for `serializeBinaryBulk`. - */ - virtual void serializeWidenBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const - { - serializeBinaryBulk(column, ostr, offset, limit); - } - - /** Widen version for `deserializeBinaryBulk`. - */ - virtual void deserializeWidenBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const - { - deserializeBinaryBulk(column, istr, limit, avg_value_size_hint); - } - /** Serialization/deserialization of individual values. * * These are helper methods for implementation of various formats to input/output for user (like CSV, JSON, etc.). @@ -479,15 +414,6 @@ class IDataType : private boost::noncopyable */ virtual bool canBeInsideNullable() const { return false; }; - /** Some specific data types are required to be widened for some specific storage for whatever reason, - * i.e. to avoid data rewriting upon type change, - * TMT will intentionally store narrow type (int8/16/32) to its widest possible type (int64) of the same family, - * meanwhile behaves as its original narrow type. - * Given that most data type objects on the fly are const (DataTypePtr), this function returns a new copy of the widened type. - */ - virtual DataTypePtr widen() const { return nullptr; } - - /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); diff --git a/dbms/src/Storages/MutableSupport.h b/dbms/src/Storages/MutableSupport.h index e3d3f896970..5ff61994dbf 100644 --- a/dbms/src/Storages/MutableSupport.h +++ b/dbms/src/Storages/MutableSupport.h @@ -52,15 +52,6 @@ class MutableSupport : public ext::Singleton block.erase(it); } - bool shouldWiden(const NameAndTypePair & column) - { - DataTypePtr t - = column.type->isNullable() ? dynamic_cast(column.type.get())->getNestedType() : column.type; - return (column.name != MutableSupport::version_column_name && column.name != MutableSupport::delmark_column_name - && column.name != MutableSupport::tidb_pk_column_name) - && t->isInteger() && !(typeid_cast(t.get()) || typeid_cast(t.get())); - } - static const String mmt_storage_name; static const String txn_storage_name; static const String delta_tree_storage_name; diff --git a/dbms/src/Storages/Transaction/Datum.cpp b/dbms/src/Storages/Transaction/Datum.cpp index b379a448db2..d0d43b60d57 100644 --- a/dbms/src/Storages/Transaction/Datum.cpp +++ b/dbms/src/Storages/Transaction/Datum.cpp @@ -79,7 +79,8 @@ struct DatumOp::type> static bool overflow(const Field &, const ColumnInfo &) { return false; } }; -DatumFlat::DatumFlat(const DB::Field & field, TP tp) : DatumBase(field, tp) +DatumFlat::DatumFlat(const DB::Field & field, TP tp) + : DatumBase(field, tp) { if (orig.isNull()) return; @@ -89,7 +90,7 @@ DatumFlat::DatumFlat(const DB::Field & field, TP tp) : DatumBase(field, tp) #ifdef M #error "Please undefine macro M first." #endif -#define M(tt, v, cf, ct, w) \ +#define M(tt, v, cf, ct) \ case Type##tt: \ DatumOp::unflatten(orig, copy); \ break; @@ -98,7 +99,10 @@ DatumFlat::DatumFlat(const DB::Field & field, TP tp) : DatumBase(field, tp) } } -bool DatumFlat::invalidNull(const ColumnInfo & column_info) { return column_info.hasNotNullFlag() && orig.isNull(); } +bool DatumFlat::invalidNull(const ColumnInfo & column_info) +{ + return column_info.hasNotNullFlag() && orig.isNull(); +} bool DatumFlat::overflow(const ColumnInfo & column_info) { @@ -110,8 +114,8 @@ bool DatumFlat::overflow(const ColumnInfo & column_info) #ifdef M #error "Please undefine macro M first." #endif -#define M(tt, v, cf, ct, w) \ - case Type##tt: \ +#define M(tt, v, cf, ct) \ + case Type##tt: \ return DatumOp::overflow(field(), column_info); COLUMN_TYPES(M) #undef M @@ -120,7 +124,8 @@ bool DatumFlat::overflow(const ColumnInfo & column_info) throw DB::Exception("Shouldn't reach here", DB::ErrorCodes::LOGICAL_ERROR); } -DatumBumpy::DatumBumpy(const DB::Field & field, TP tp) : DatumBase(field, tp) +DatumBumpy::DatumBumpy(const DB::Field & field, TP tp) + : DatumBase(field, tp) { if (orig.isNull()) return; @@ -130,7 +135,7 @@ DatumBumpy::DatumBumpy(const DB::Field & field, TP tp) : DatumBase(field, tp) #ifdef M #error "Please undefine macro M first." #endif -#define M(tt, v, cf, ct, w) \ +#define M(tt, v, cf, ct) \ case Type##tt: \ DatumOp::flatten(orig, copy); \ break; diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index 745839a2476..40f189d9009 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -989,8 +989,8 @@ CodecFlag ColumnInfo::getCodecFlag() const #ifdef M #error "Please undefine macro M first." #endif -#define M(tt, v, cf, ct, w) \ - case Type##tt: \ +#define M(tt, v, cf, ct) \ + case Type##tt: \ return getCodecFlagBase(hasUnsignedFlag()); COLUMN_TYPES(M) #undef M diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index cd428e57e6e..c41a2f5157e 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -53,46 +53,46 @@ using DB::Timestamp; // Column types. // In format: -// TiDB type, int value, codec flag, CH type, should widen. +// TiDB type, int value, codec flag, CH type. #ifdef M #error "Please undefine macro M first." #endif -#define COLUMN_TYPES(M) \ - M(Decimal, 0, Decimal, Decimal32, false) \ - M(Tiny, 1, VarInt, Int8, true) \ - M(Short, 2, VarInt, Int16, true) \ - M(Long, 3, VarInt, Int32, true) \ - M(Float, 4, Float, Float32, false) \ - M(Double, 5, Float, Float64, false) \ - M(Null, 6, Nil, Nothing, false) \ - M(Timestamp, 7, UInt, MyDateTime, false) \ - M(LongLong, 8, Int, Int64, false) \ - M(Int24, 9, VarInt, Int32, true) \ - M(Date, 10, UInt, MyDate, false) \ - M(Time, 11, Duration, Int64, false) \ - M(Datetime, 12, UInt, MyDateTime, false) \ - M(Year, 13, Int, Int16, false) \ - M(NewDate, 14, Int, MyDate, false) \ - M(Varchar, 15, CompactBytes, String, false) \ - M(Bit, 16, VarInt, UInt64, false) \ - M(JSON, 0xf5, Json, String, false) \ - M(NewDecimal, 0xf6, Decimal, Decimal32, false) \ - M(Enum, 0xf7, VarUInt, Enum16, false) \ - M(Set, 0xf8, VarUInt, UInt64, false) \ - M(TinyBlob, 0xf9, CompactBytes, String, false) \ - M(MediumBlob, 0xfa, CompactBytes, String, false) \ - M(LongBlob, 0xfb, CompactBytes, String, false) \ - M(Blob, 0xfc, CompactBytes, String, false) \ - M(VarString, 0xfd, CompactBytes, String, false) \ - M(String, 0xfe, CompactBytes, String, false) \ - M(Geometry, 0xff, CompactBytes, String, false) +#define COLUMN_TYPES(M) \ + M(Decimal, 0, Decimal, Decimal32) \ + M(Tiny, 1, VarInt, Int8) \ + M(Short, 2, VarInt, Int16) \ + M(Long, 3, VarInt, Int32) \ + M(Float, 4, Float, Float32) \ + M(Double, 5, Float, Float64) \ + M(Null, 6, Nil, Nothing) \ + M(Timestamp, 7, UInt, MyDateTime) \ + M(LongLong, 8, Int, Int64) \ + M(Int24, 9, VarInt, Int32) \ + M(Date, 10, UInt, MyDate) \ + M(Time, 11, Duration, Int64) \ + M(Datetime, 12, UInt, MyDateTime) \ + M(Year, 13, Int, Int16) \ + M(NewDate, 14, Int, MyDate) \ + M(Varchar, 15, CompactBytes, String) \ + M(Bit, 16, VarInt, UInt64) \ + M(JSON, 0xf5, Json, String) \ + M(NewDecimal, 0xf6, Decimal, Decimal32) \ + M(Enum, 0xf7, VarUInt, Enum16) \ + M(Set, 0xf8, VarUInt, UInt64) \ + M(TinyBlob, 0xf9, CompactBytes, String) \ + M(MediumBlob, 0xfa, CompactBytes, String) \ + M(LongBlob, 0xfb, CompactBytes, String) \ + M(Blob, 0xfc, CompactBytes, String) \ + M(VarString, 0xfd, CompactBytes, String) \ + M(String, 0xfe, CompactBytes, String) \ + M(Geometry, 0xff, CompactBytes, String) enum TP { #ifdef M #error "Please undefine macro M first." #endif -#define M(tt, v, cf, ct, w) Type##tt = (v), +#define M(tt, v, cf, ct) Type##tt = (v), COLUMN_TYPES(M) #undef M }; diff --git a/dbms/src/Storages/Transaction/TypeMapping.cpp b/dbms/src/Storages/Transaction/TypeMapping.cpp index 15f11f4f7d6..256855f17b4 100644 --- a/dbms/src/Storages/Transaction/TypeMapping.cpp +++ b/dbms/src/Storages/Transaction/TypeMapping.cpp @@ -104,22 +104,14 @@ struct EnumType : public std::true_type template inline constexpr bool IsEnumType = EnumType::value; -template +template std::enable_if_t && !IsDecimalType && !IsEnumType && !std::is_same_v, DataTypePtr> getDataTypeByColumnInfoBase(const ColumnInfo &, const T *) { - DataTypePtr t = std::make_shared(); - - if (should_widen) - { - auto widen = t->widen(); - t.swap(widen); - } - - return t; + return std::make_shared(); } -template +template std::enable_if_t, DataTypePtr> getDataTypeByColumnInfoBase(const ColumnInfo & column_info, const T *) { DataTypePtr t = nullptr; @@ -129,57 +121,27 @@ std::enable_if_t, DataTypePtr> getDataTypeByColumnInfoBase(const else t = std::make_shared(); - if (should_widen) - { - auto widen = t->widen(); - t.swap(widen); - } - return t; } -template +template std::enable_if_t, DataTypePtr> getDataTypeByColumnInfoBase(const ColumnInfo & column_info, const T *) { - DataTypePtr t = createDecimal(column_info.flen, column_info.decimal); - - if (should_widen) - { - auto widen = t->widen(); - t.swap(widen); - } - - return t; + return createDecimal(column_info.flen, column_info.decimal); } -template +template std::enable_if_t, DataTypePtr> getDataTypeByColumnInfoBase(const ColumnInfo & column_info, const T *) { // In some cases, TiDB will set the decimal to -1, change -1 to 6 to avoid error - DataTypePtr t = std::make_shared(column_info.decimal == -1 ? 6 : column_info.decimal); - - if (should_widen) - { - auto widen = t->widen(); - t.swap(widen); - } - - return t; + return std::make_shared(column_info.decimal == -1 ? 6 : column_info.decimal); } -template +template std::enable_if_t, DataTypePtr> getDataTypeByColumnInfoBase(const ColumnInfo & column_info, const T *) { - DataTypePtr t = std::make_shared(column_info.elems); - - if (should_widen) - { - auto widen = t->widen(); - t.swap(widen); - } - - return t; + return std::make_shared(column_info.elems); } TypeMapping::TypeMapping() @@ -187,17 +149,24 @@ TypeMapping::TypeMapping() #ifdef M #error "Please undefine macro M first." #endif -#define M(tt, v, cf, ct, w) \ - type_map[TiDB::Type##tt] = std::bind(getDataTypeByColumnInfoBase, std::placeholders::_1, (DataType##ct *)nullptr); +#define M(tt, v, cf, ct) \ + type_map[TiDB::Type##tt] = [](const ColumnInfo & column_info) { \ + return getDataTypeByColumnInfoBase(column_info, (DataType##ct *)nullptr); \ + }; COLUMN_TYPES(M) #undef M } +// Get the basic data type according to column_info. +// This method ignores the nullable flag. DataTypePtr TypeMapping::getDataType(const ColumnInfo & column_info) { return type_map[column_info.tp](column_info); } +// Get the data type according to column_info, respecting +// the nullable flag. +// This does not support the "duration" type. DataTypePtr getDataTypeByColumnInfo(const ColumnInfo & column_info) { DataTypePtr base = TypeMapping::instance().getDataType(column_info); @@ -209,6 +178,8 @@ DataTypePtr getDataTypeByColumnInfo(const ColumnInfo & column_info) return base; } +// Get the data type according to column_info. +// This support the duration type that only will be generated when executing DataTypePtr getDataTypeByColumnInfoForComputingLayer(const ColumnInfo & column_info) { DataTypePtr base = TypeMapping::instance().getDataType(column_info); @@ -252,10 +223,10 @@ void setDecimalPrecScale(const T * decimal_type, ColumnInfo & column_info) void fillTiDBColumnInfo(const String & family_name, const ASTPtr & parameters, ColumnInfo & column_info); void fillTiDBColumnInfo(const ASTPtr & type, ColumnInfo & column_info) { - auto * func = typeid_cast(type.get()); + const auto * func = typeid_cast(type.get()); if (func != nullptr) return fillTiDBColumnInfo(func->name, func->arguments, column_info); - auto * ident = typeid_cast(type.get()); + const auto * ident = typeid_cast(type.get()); if (ident != nullptr) return fillTiDBColumnInfo(ident->name, {}, column_info); throw Exception("Failed to get TiDB data type"); @@ -374,7 +345,7 @@ ColumnInfo reverseGetColumnInfo(const NameAndTypePair & column, ColumnID id, con } else { - auto nullable_type = checkAndGetDataType(nested_type); + const auto * nullable_type = checkAndGetDataType(nested_type); nested_type = nullable_type->getNestedType().get(); } @@ -443,28 +414,28 @@ ColumnInfo reverseGetColumnInfo(const NameAndTypePair & column, ColumnID id, con column_info.setUnsignedFlag(); // Fill flen and decimal for decimal. - if (auto decimal_type32 = checkAndGetDataType>(nested_type)) + if (const auto * decimal_type32 = checkAndGetDataType>(nested_type)) setDecimalPrecScale(decimal_type32, column_info); - else if (auto decimal_type64 = checkAndGetDataType>(nested_type)) + else if (const auto * decimal_type64 = checkAndGetDataType>(nested_type)) setDecimalPrecScale(decimal_type64, column_info); - else if (auto decimal_type128 = checkAndGetDataType>(nested_type)) + else if (const auto * decimal_type128 = checkAndGetDataType>(nested_type)) setDecimalPrecScale(decimal_type128, column_info); - else if (auto decimal_type256 = checkAndGetDataType>(nested_type)) + else if (const auto * decimal_type256 = checkAndGetDataType>(nested_type)) setDecimalPrecScale(decimal_type256, column_info); // Fill decimal for date time. - if (auto type = checkAndGetDataType(nested_type)) + if (const auto * type = checkAndGetDataType(nested_type)) column_info.decimal = type->getFraction(); // Fill decimal for duration. - if (auto type = checkAndGetDataType(nested_type)) + if (const auto * type = checkAndGetDataType(nested_type)) column_info.decimal = type->getFsp(); // Fill elems for enum. if (checkDataType(nested_type)) { - auto enum16_type = checkAndGetDataType(nested_type); - for (auto & element : enum16_type->getValues()) + const auto * enum16_type = checkAndGetDataType(nested_type); + for (const auto & element : enum16_type->getValues()) { column_info.elems.emplace_back(element.first, element.second); } diff --git a/dbms/src/Storages/Transaction/tests/gtest_type_mapping.cpp b/dbms/src/Storages/Transaction/tests/gtest_type_mapping.cpp index 4fda5b51f35..1f5dd7ac2f9 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_type_mapping.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_type_mapping.cpp @@ -22,12 +22,7 @@ namespace DB namespace tests { -TEST(TypeMapping_test, ColumnInfoToDataType) -{ - // TODO fill this test -} - -TEST(TypeMapping_test, DataTypeToColumnInfo) +TEST(TypeMappingTest, DataTypeToColumnInfo) try { String name = "col"; @@ -67,12 +62,19 @@ try { ASSERT_EQ(column_info.tp, TiDB::TypeLongLong) << actual_test_type; } + + auto data_type = getDataTypeByColumnInfo(column_info); + ASSERT_EQ(data_type->getName(), actual_test_type); } } } column_info = reverseGetColumnInfo(NameAndTypePair{name, typeFromString("String")}, 1, default_field, true); ASSERT_EQ(column_info.tp, TiDB::TypeString); + auto data_type = getDataTypeByColumnInfo(column_info); + ASSERT_EQ(data_type->getName(), "String"); + + // TODO: test decimal, datetime, enum } CATCH From 1e7edb85e61bffdda4286cb73a3ee20b5a813ba4 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 23 Dec 2022 12:48:14 +0800 Subject: [PATCH 37/42] move MergingAndConvertingBlockInputStream out of Aggregate.cpp (#6531) ref pingcap/tiflash#5900 --- dbms/src/Interpreters/Aggregator.cpp | 225 +--------------- .../MergingAndConvertingBlockInputStream.h | 252 ++++++++++++++++++ 2 files changed, 259 insertions(+), 218 deletions(-) create mode 100644 dbms/src/Interpreters/MergingAndConvertingBlockInputStream.h diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index b7193833031..4a9ffc1c993 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include @@ -56,7 +57,6 @@ extern const int LOGICAL_ERROR; namespace FailPoints { extern const char random_aggregate_create_state_failpoint[]; -extern const char random_aggregate_merge_failpoint[]; } // namespace FailPoints #define AggregationMethodName(NAME) AggregatedDataVariants::AggregationMethod_##NAME @@ -1773,223 +1773,6 @@ void NO_INLINE Aggregator::mergeBucketImpl( } -/** Combines aggregation states together, turns them into blocks, and outputs streams. - * If the aggregation states are two-level, then it produces blocks strictly in order of 'bucket_num'. - * (This is important for distributed processing.) - * In doing so, it can handle different buckets in parallel, using up to `threads` threads. - */ -class MergingAndConvertingBlockInputStream : public IProfilingBlockInputStream -{ -public: - /** The input is a set of non-empty sets of partially aggregated data, - * which are all either single-level, or are two-level. - */ - MergingAndConvertingBlockInputStream(const Aggregator & aggregator_, ManyAggregatedDataVariants & data_, bool final_, size_t threads_) - : log(Logger::get(aggregator_.log ? aggregator_.log->identifier() : "")) - , aggregator(aggregator_) - , data(data_) - , final(final_) - , threads(threads_) - { - /// At least we need one arena in first data item per thread - if (!data.empty() && threads > data[0]->aggregates_pools.size()) - { - Arenas & first_pool = data[0]->aggregates_pools; - for (size_t j = first_pool.size(); j < threads; ++j) - first_pool.emplace_back(std::make_shared()); - } - } - - String getName() const override { return "MergingAndConverting"; } - - Block getHeader() const override { return aggregator.getHeader(final); } - - ~MergingAndConvertingBlockInputStream() override - { - LOG_TRACE(&Poco::Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); - - /// We need to wait for threads to finish before destructor of 'parallel_merge_data', - /// because the threads access 'parallel_merge_data'. - if (parallel_merge_data && parallel_merge_data->thread_pool) - parallel_merge_data->thread_pool->wait(); - } - -protected: - Block readImpl() override - { - if (data.empty()) - return {}; - - if (current_bucket_num >= NUM_BUCKETS) - return {}; - - FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_aggregate_merge_failpoint); - - AggregatedDataVariantsPtr & first = data[0]; - - if (current_bucket_num == -1) - { - ++current_bucket_num; - - if (first->type == AggregatedDataVariants::Type::without_key || aggregator.params.overflow_row) - { - aggregator.mergeWithoutKeyDataImpl(data); - return aggregator.prepareBlockAndFillWithoutKey( - *first, - final, - first->type != AggregatedDataVariants::Type::without_key); - } - } - - if (!first->isTwoLevel()) - { - if (current_bucket_num > 0) - return {}; - - if (first->type == AggregatedDataVariants::Type::without_key) - return {}; - - ++current_bucket_num; - -#define M(NAME) \ - case AggregationMethodType(NAME): \ - { \ - aggregator.mergeSingleLevelDataImpl(data); \ - break; \ - } - switch (first->type) - { - APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) - default: - throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); - } -#undef M - return aggregator.prepareBlockAndFillSingleLevel(*first, final); - } - else - { - if (!parallel_merge_data) - { - parallel_merge_data = std::make_unique(threads); - for (size_t i = 0; i < threads; ++i) - scheduleThreadForNextBucket(); - } - - Block res; - - while (true) - { - std::unique_lock lock(parallel_merge_data->mutex); - - if (parallel_merge_data->exception) - std::rethrow_exception(parallel_merge_data->exception); - - auto it = parallel_merge_data->ready_blocks.find(current_bucket_num); - if (it != parallel_merge_data->ready_blocks.end()) - { - ++current_bucket_num; - scheduleThreadForNextBucket(); - - if (it->second) - { - res.swap(it->second); - break; - } - else if (current_bucket_num >= NUM_BUCKETS) - break; - } - - parallel_merge_data->condvar.wait(lock); - } - - return res; - } - } - -private: - const LoggerPtr log; - const Aggregator & aggregator; - ManyAggregatedDataVariants data; - bool final; - size_t threads; - - std::atomic current_bucket_num = -1; - std::atomic max_scheduled_bucket_num = -1; - static constexpr Int32 NUM_BUCKETS = 256; - - struct ParallelMergeData - { - std::map ready_blocks; - std::exception_ptr exception; - std::mutex mutex; - std::condition_variable condvar; - std::shared_ptr thread_pool; - - explicit ParallelMergeData(size_t threads) - : thread_pool(newThreadPoolManager(threads)) - {} - }; - - std::unique_ptr parallel_merge_data; - - void scheduleThreadForNextBucket() - { - int num = max_scheduled_bucket_num.fetch_add(1) + 1; - if (num >= NUM_BUCKETS) - return; - - parallel_merge_data->thread_pool->schedule(true, [this, num] { thread(num); }); - } - - void thread(Int32 bucket_num) - { - try - { - /// TODO: add no_more_keys support maybe - - auto & merged_data = *data[0]; - auto method = merged_data.type; - Block block; - - /// Select Arena to avoid race conditions - size_t thread_number = static_cast(bucket_num) % threads; - Arena * arena = merged_data.aggregates_pools.at(thread_number).get(); - -#define M(NAME) \ - case AggregationMethodType(NAME): \ - { \ - aggregator.mergeBucketImpl(data, bucket_num, arena); \ - block = aggregator.convertOneBucketToBlock( \ - merged_data, \ - *ToAggregationMethodPtr(NAME, merged_data.aggregation_method_impl), \ - arena, \ - final, \ - bucket_num); \ - break; \ - } - switch (method) - { - APPLY_FOR_VARIANTS_TWO_LEVEL(M) - default: - break; - } -#undef M - - std::lock_guard lock(parallel_merge_data->mutex); - parallel_merge_data->ready_blocks[bucket_num] = std::move(block); - } - catch (...) - { - std::lock_guard lock(parallel_merge_data->mutex); - if (!parallel_merge_data->exception) - parallel_merge_data->exception = std::current_exception(); - } - - parallel_merge_data->condvar.notify_all(); - } -}; - - std::unique_ptr Aggregator::mergeAndConvertToBlocks( ManyAggregatedDataVariants & data_variants, bool final, @@ -2751,5 +2534,11 @@ void Aggregator::setCancellationHook(CancellationHook cancellation_hook) is_cancelled = cancellation_hook; } +#undef AggregationMethodName +#undef AggregationMethodNameTwoLevel +#undef AggregationMethodType +#undef AggregationMethodTypeTwoLevel +#undef ToAggregationMethodPtr +#undef ToAggregationMethodPtrTwoLevel } // namespace DB diff --git a/dbms/src/Interpreters/MergingAndConvertingBlockInputStream.h b/dbms/src/Interpreters/MergingAndConvertingBlockInputStream.h new file mode 100644 index 00000000000..7e58eb8da81 --- /dev/null +++ b/dbms/src/Interpreters/MergingAndConvertingBlockInputStream.h @@ -0,0 +1,252 @@ +// 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. + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +namespace FailPoints +{ +extern const char random_aggregate_merge_failpoint[]; +} // namespace FailPoints + +#define AggregationMethodName(NAME) AggregatedDataVariants::AggregationMethod_##NAME +#define AggregationMethodType(NAME) AggregatedDataVariants::Type::NAME +#define ToAggregationMethodPtr(NAME, ptr) (reinterpret_cast(ptr)) + +/** Combines aggregation states together, turns them into blocks, and outputs streams. + * If the aggregation states are two-level, then it produces blocks strictly in order of 'bucket_num'. + * (This is important for distributed processing.) + * In doing so, it can handle different buckets in parallel, using up to `threads` threads. + */ +class MergingAndConvertingBlockInputStream : public IProfilingBlockInputStream +{ +public: + /** The input is a set of non-empty sets of partially aggregated data, + * which are all either single-level, or are two-level. + */ + MergingAndConvertingBlockInputStream(const Aggregator & aggregator_, ManyAggregatedDataVariants & data_, bool final_, size_t threads_) + : log(Logger::get(aggregator_.log ? aggregator_.log->identifier() : "")) + , aggregator(aggregator_) + , data(data_) + , final(final_) + , threads(threads_) + { + /// At least we need one arena in first data item per thread + if (!data.empty() && threads > data[0]->aggregates_pools.size()) + { + Arenas & first_pool = data[0]->aggregates_pools; + for (size_t j = first_pool.size(); j < threads; ++j) + first_pool.emplace_back(std::make_shared()); + } + } + + String getName() const override { return "MergingAndConverting"; } + + Block getHeader() const override { return aggregator.getHeader(final); } + + ~MergingAndConvertingBlockInputStream() override + { + LOG_TRACE(&Poco::Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); + + /// We need to wait for threads to finish before destructor of 'parallel_merge_data', + /// because the threads access 'parallel_merge_data'. + if (parallel_merge_data && parallel_merge_data->thread_pool) + parallel_merge_data->thread_pool->wait(); + } + +protected: + Block readImpl() override + { + if (data.empty()) + return {}; + + if (current_bucket_num >= NUM_BUCKETS) + return {}; + + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_aggregate_merge_failpoint); + + AggregatedDataVariantsPtr & first = data[0]; + + if (current_bucket_num == -1) + { + ++current_bucket_num; + + if (first->type == AggregatedDataVariants::Type::without_key || aggregator.params.overflow_row) + { + aggregator.mergeWithoutKeyDataImpl(data); + return aggregator.prepareBlockAndFillWithoutKey( + *first, + final, + first->type != AggregatedDataVariants::Type::without_key); + } + } + + if (!first->isTwoLevel()) + { + if (current_bucket_num > 0) + return {}; + + if (first->type == AggregatedDataVariants::Type::without_key) + return {}; + + ++current_bucket_num; + +#define M(NAME) \ + case AggregationMethodType(NAME): \ + { \ + aggregator.mergeSingleLevelDataImpl(data); \ + break; \ + } + switch (first->type) + { + APPLY_FOR_VARIANTS_SINGLE_LEVEL(M) + default: + throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); + } +#undef M + return aggregator.prepareBlockAndFillSingleLevel(*first, final); + } + else + { + if (!parallel_merge_data) + { + parallel_merge_data = std::make_unique(threads); + for (size_t i = 0; i < threads; ++i) + scheduleThreadForNextBucket(); + } + + Block res; + + while (true) + { + std::unique_lock lock(parallel_merge_data->mutex); + + if (parallel_merge_data->exception) + std::rethrow_exception(parallel_merge_data->exception); + + auto it = parallel_merge_data->ready_blocks.find(current_bucket_num); + if (it != parallel_merge_data->ready_blocks.end()) + { + ++current_bucket_num; + scheduleThreadForNextBucket(); + + if (it->second) + { + res.swap(it->second); + break; + } + else if (current_bucket_num >= NUM_BUCKETS) + break; + } + + parallel_merge_data->condvar.wait(lock); + } + + return res; + } + } + +private: + const LoggerPtr log; + const Aggregator & aggregator; + ManyAggregatedDataVariants data; + bool final; + size_t threads; + + std::atomic current_bucket_num = -1; + std::atomic max_scheduled_bucket_num = -1; + static constexpr Int32 NUM_BUCKETS = 256; + + struct ParallelMergeData + { + std::map ready_blocks; + std::exception_ptr exception; + std::mutex mutex; + std::condition_variable condvar; + std::shared_ptr thread_pool; + + explicit ParallelMergeData(size_t threads) + : thread_pool(newThreadPoolManager(threads)) + {} + }; + + std::unique_ptr parallel_merge_data; + + void scheduleThreadForNextBucket() + { + int num = max_scheduled_bucket_num.fetch_add(1) + 1; + if (num >= NUM_BUCKETS) + return; + + parallel_merge_data->thread_pool->schedule(true, [this, num] { thread(num); }); + } + + void thread(Int32 bucket_num) + { + try + { + /// TODO: add no_more_keys support maybe + + auto & merged_data = *data[0]; + auto method = merged_data.type; + Block block; + + /// Select Arena to avoid race conditions + size_t thread_number = static_cast(bucket_num) % threads; + Arena * arena = merged_data.aggregates_pools.at(thread_number).get(); + +#define M(NAME) \ + case AggregationMethodType(NAME): \ + { \ + aggregator.mergeBucketImpl(data, bucket_num, arena); \ + block = aggregator.convertOneBucketToBlock( \ + merged_data, \ + *ToAggregationMethodPtr(NAME, merged_data.aggregation_method_impl), \ + arena, \ + final, \ + bucket_num); \ + break; \ + } + switch (method) + { + APPLY_FOR_VARIANTS_TWO_LEVEL(M) + default: + break; + } +#undef M + + std::lock_guard lock(parallel_merge_data->mutex); + parallel_merge_data->ready_blocks[bucket_num] = std::move(block); + } + catch (...) + { + std::lock_guard lock(parallel_merge_data->mutex); + if (!parallel_merge_data->exception) + parallel_merge_data->exception = std::current_exception(); + } + + parallel_merge_data->condvar.notify_all(); + } +}; + +#undef AggregationMethodName +#undef AggregationMethodType +#undef ToAggregationMethodPtr +} // namespace DB From f16f037a6b431504484aede8e168afe6db074de0 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Fri, 23 Dec 2022 13:42:14 +0800 Subject: [PATCH 38/42] doc: add doc for auto-reload-cert (#6527) close pingcap/tiflash#5503 --- .../2022-12-21-auto-reload-tls-certificate.md | 75 +++++++++++++++++++ 1 file changed, 75 insertions(+) create mode 100644 docs/design/2022-12-21-auto-reload-tls-certificate.md diff --git a/docs/design/2022-12-21-auto-reload-tls-certificate.md b/docs/design/2022-12-21-auto-reload-tls-certificate.md new file mode 100644 index 00000000000..e96705cb7ae --- /dev/null +++ b/docs/design/2022-12-21-auto-reload-tls-certificate.md @@ -0,0 +1,75 @@ +# Auto reload TLS certificate for TiFlash + +- Author: [Weiqi Yan](https://github.com/ywqzzy) + +## Introduction + +In TiFlash config, we can set certificate as follows: +```YAML +[security] +ca_path = "/path/to/tls/ca.crt" +cert_path = "/path/to/tls/tiflash.crt" +key_path = "/path/to/tls/tiflash.pem" +``` +Then the TiFlash Server can use TLS certificates to enable secure transmission. + +Since the TLS certificate has a valid period, in order not to affect the normal operation of online business, the TiFlash node should not be restarted manually when replacing the certificate, so it needs to support automatic rotation of TLS certificates. + +By modifying the TiFlash configuration file(tiflash.toml) or the certificate content, TiFlash can dynamically load new TLS certificates. + +## Desgin + +### Overview + +TiFlash uses TLS certificates in GRPC Server, TCP Server, HTTP Server, MetricsPrometheus HTTP Server, client-c (rpcClient, pdClient). + +There are two ways to modify the certificate: + +1. Modify the path of the certificate in the config file. + +2. Directly overwrite the content of the certificate at the specified path. + +For 1, we use `main_config_reloader` to monitor the change of the certificate file path, then update the certificate path information in `Context` after the change of the file path, then update the certificate path maintained in the `TiFlashSecurity` class. + +For 2, the certificate can be dynamically loaded in the form of callback for various servers, and the client connection can be rebuilt for the client. + +### Detailed Design + +#### GRPC Server + +The certificate used by GRPC server should change with the change of the certificate path in `TiFlashSecurity`. +At the same time, each time a new SSL connection is created, a new certificate can be loaded according to the certificate path. + +We need to set one `ConfigFetcher `when building GRPC server in order to dynamically read new certificates when establishing a new SSL connection. + +The `ConfigFetcher` will be set as follows: + +```C++ +builder.AddListeningPort( + raft_config.flash_server_addr, + sslServerCredentialsWithFetcher(context)); +``` + +The `sslServerCredentialsWithFetcher` method will set the `ConfigFetcher` for `grpc::ServerBuilder`. As a callback function, `ConfigFetcher` obtains the certificate path from TiFlashSecurity and sets the certificate for each SSL connection. + +#### HTTP/TCP Server/MetricPrometheus + +These server use `Poco::Net`. To reload certificate dynamically, we can call `SSL_CTX_set_cert_cb` for `Poco::Net::Context::sslContext`. Then these servers can dynamically read new certificates and set new SSL certs when establishing SSL connections with others. + +The setting process is as follows: + +```C++ +SSL_CTX_set_cert_cb(context->sslContext(), + callSetCertificate, + reinterpret_cast(global_context)); +``` + +`callSetCertificate` will read the certificate path from `TiFlashSecurity`, then read the new certificate from certificate path in order to set the new certificate. + +#### client-c + +Judge whether the certificate file has changed (or whether the certificate path has changed) in `main_config_reloader`. When the certificate has changed, read `TiFlashSecurity` to get new certificate paths, and clear the existing client conn array (including `pdClient` and `rpcClient`), so that the new certificate can be read later to create a new connection. + +#### ConfigReloader + +The `ConfigReloader` should monitor whether the certificate file changed or the certificate file paths changed. When changes occur, the `ConfigReloader` should call `reload` to reload some of the configs that need to refresh. From c361d3aa366fe2cfbf2dd3604fb6fa525d543134 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Fri, 23 Dec 2022 16:02:14 +0800 Subject: [PATCH 39/42] *: remove limitByStream (#6539) ref pingcap/tiflash#5900 --- .../DataStreams/LimitByBlockInputStream.cpp | 92 ------------------- .../src/DataStreams/LimitByBlockInputStream.h | 52 ----------- .../Interpreters/InterpreterSelectQuery.cpp | 24 ----- 3 files changed, 168 deletions(-) delete mode 100644 dbms/src/DataStreams/LimitByBlockInputStream.cpp delete mode 100644 dbms/src/DataStreams/LimitByBlockInputStream.h diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.cpp b/dbms/src/DataStreams/LimitByBlockInputStream.cpp deleted file mode 100644 index 83e93041c34..00000000000 --- a/dbms/src/DataStreams/LimitByBlockInputStream.cpp +++ /dev/null @@ -1,92 +0,0 @@ -// 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 -{ - -LimitByBlockInputStream::LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, const Names & columns) - : columns_names(columns) - , group_size(group_size_) -{ - children.push_back(input); -} - -Block LimitByBlockInputStream::readImpl() -{ - /// Execute until end of stream or until - /// a block with some new records will be gotten. - while (true) - { - Block block = children[0]->read(); - if (!block) - return Block(); - - const ColumnRawPtrs column_ptrs(getKeyColumns(block)); - const size_t rows = block.rows(); - IColumn::Filter filter(rows); - size_t inserted_count = 0; - - for (size_t i = 0; i < rows; ++i) - { - UInt128 key; - SipHash hash; - - for (auto & column : column_ptrs) - column->updateHashWithValue(i, hash); - - hash.get128(key); - - if (keys_counts[key]++ < group_size) - { - inserted_count++; - filter[i] = 1; - } - else - filter[i] = 0; - } - - /// Just go to the next block if there isn't any new records in the current one. - if (!inserted_count) - continue; - - size_t all_columns = block.columns(); - for (size_t i = 0; i < all_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(filter, inserted_count); - - return block; - } -} - -ColumnRawPtrs LimitByBlockInputStream::getKeyColumns(Block & block) const -{ - ColumnRawPtrs column_ptrs; - column_ptrs.reserve(columns_names.size()); - - for (const auto & name : columns_names) - { - auto & column = block.getByName(name).column; - - /// Ignore all constant columns. - if (!column->isColumnConst()) - column_ptrs.emplace_back(column.get()); - } - - return column_ptrs; -} - -} diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.h b/dbms/src/DataStreams/LimitByBlockInputStream.h deleted file mode 100644 index 4a91f0ca9cc..00000000000 --- a/dbms/src/DataStreams/LimitByBlockInputStream.h +++ /dev/null @@ -1,52 +0,0 @@ -// 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. - -#pragma once - -#include -#include - -namespace DB -{ - -/** Implements LIMIT BY clause witch can be used to obtain a "top N by subgroup". - * - * For example, if you have table T like this (Num: 1 1 3 3 3 4 4 5 7 7 7 7), - * the query SELECT Num FROM T LIMIT 2 BY Num - * will give you the following result: (Num: 1 1 3 3 4 4 5 7 7). - */ -class LimitByBlockInputStream : public IProfilingBlockInputStream -{ -public: - LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, const Names & columns); - - String getName() const override { return "LimitBy"; } - - Block getHeader() const override { return children.at(0)->getHeader(); } - -protected: - Block readImpl() override; - -private: - ColumnRawPtrs getKeyColumns(Block & block) const; - -private: - using MapHashed = HashMap; - - const Names columns_names; - const size_t group_size; - MapHashed keys_counts; -}; - -} diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 47571a6f860..f1a39672a99 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -27,7 +27,6 @@ #include #include #include -#include #include #include #include @@ -616,12 +615,6 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (need_second_distinct_pass) executeDistinct(pipeline, false, expressions.selected_columns); - if (expressions.has_limit_by) - { - executeExpression(pipeline, expressions.before_limit_by); - executeLimitBy(pipeline); - } - /** We must do projection after DISTINCT because projection may remove some columns. */ executeProjection(pipeline, expressions.final_projection); @@ -1291,23 +1284,6 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) } -void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) // NOLINT -{ - if (!query.limit_by_value || !query.limit_by_expression_list) - return; - - Names columns; - for (const auto & elem : query.limit_by_expression_list->children) - columns.emplace_back(elem->getColumnName()); - - auto value = safeGet(typeid_cast(*query.limit_by_value).value); - - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, value, columns); - }); -} - - bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query) { if (query.group_by_with_totals) From 957d2d41c07d538edab9e47bdbb7eaede753c60b Mon Sep 17 00:00:00 2001 From: yibin Date: Fri, 23 Dec 2022 16:50:14 +0800 Subject: [PATCH 40/42] Add unit tests for fine-grained join & agg (#6445) ref pingcap/tiflash#6157 --- .../Debug/MockExecutor/AggregationBinder.cpp | 9 +- .../Debug/MockExecutor/AggregationBinder.h | 6 +- dbms/src/Debug/MockExecutor/JoinBinder.cpp | 6 +- dbms/src/Debug/MockExecutor/JoinBinder.h | 6 +- dbms/src/Flash/tests/gtest_compute_server.cpp | 172 ++++++++++++++++++ dbms/src/Flash/tests/gtest_interpreter.cpp | 100 +++++++++- .../Flash/tests/gtest_planner_interpreter.cpp | 102 +++++++++++ dbms/src/Interpreters/Join.cpp | 5 +- dbms/src/TestUtils/mockExecutor.cpp | 18 +- dbms/src/TestUtils/mockExecutor.h | 13 +- 10 files changed, 411 insertions(+), 26 deletions(-) diff --git a/dbms/src/Debug/MockExecutor/AggregationBinder.cpp b/dbms/src/Debug/MockExecutor/AggregationBinder.cpp index 7b5d9b9d134..e95346af901 100644 --- a/dbms/src/Debug/MockExecutor/AggregationBinder.cpp +++ b/dbms/src/Debug/MockExecutor/AggregationBinder.cpp @@ -28,6 +28,7 @@ bool AggregationBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t c { tipb_executor->set_tp(tipb::ExecType::TypeAggregation); tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); auto * agg = tipb_executor->mutable_aggregation(); buildAggExpr(agg, collator_id, context); buildGroupBy(agg, collator_id, context); @@ -80,7 +81,8 @@ void AggregationBinder::toMPPSubPlan(size_t & executor_index, const DAGPropertie false, std::move(agg_exprs), std::move(gby_exprs), - false); + false, + fine_grained_shuffle_stream_count); partial_agg->children.push_back(children[0]); std::vector partition_keys; size_t agg_func_num = partial_agg->agg_exprs.size(); @@ -206,7 +208,7 @@ void AggregationBinder::buildAggFunc(tipb::Expr * agg_func, const ASTFunction * agg_func->set_aggfuncmode(tipb::AggFunctionMode::Partial1Mode); } -ExecutorBinderPtr compileAggregation(ExecutorBinderPtr input, size_t & executor_index, ASTPtr agg_funcs, ASTPtr group_by_exprs) +ExecutorBinderPtr compileAggregation(ExecutorBinderPtr input, size_t & executor_index, ASTPtr agg_funcs, ASTPtr group_by_exprs, uint64_t fine_grained_shuffle_stream_count) { std::vector agg_exprs; std::vector gby_exprs; @@ -276,7 +278,8 @@ ExecutorBinderPtr compileAggregation(ExecutorBinderPtr input, size_t & executor_ need_append_project, std::move(agg_exprs), std::move(gby_exprs), - true); + true, + fine_grained_shuffle_stream_count); aggregation->children.push_back(input); return aggregation; } diff --git a/dbms/src/Debug/MockExecutor/AggregationBinder.h b/dbms/src/Debug/MockExecutor/AggregationBinder.h index 84821594988..005549e6f0b 100644 --- a/dbms/src/Debug/MockExecutor/AggregationBinder.h +++ b/dbms/src/Debug/MockExecutor/AggregationBinder.h @@ -25,13 +25,14 @@ class ExchangeReceiverBinder; class AggregationBinder : public ExecutorBinder { public: - AggregationBinder(size_t & index_, const DAGSchema & output_schema_, bool has_uniq_raw_res_, bool need_append_project_, ASTs && agg_exprs_, ASTs && gby_exprs_, bool is_final_mode_) + AggregationBinder(size_t & index_, const DAGSchema & output_schema_, bool has_uniq_raw_res_, bool need_append_project_, ASTs && agg_exprs_, ASTs && gby_exprs_, bool is_final_mode_, uint64_t fine_grained_shuffle_stream_count_) : ExecutorBinder(index_, "aggregation_" + std::to_string(index_), output_schema_) , has_uniq_raw_res(has_uniq_raw_res_) , need_append_project(need_append_project_) , agg_exprs(std::move(agg_exprs_)) , gby_exprs(std::move(gby_exprs_)) , is_final_mode(is_final_mode_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) {} bool toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; @@ -53,6 +54,7 @@ class AggregationBinder : public ExecutorBinder std::vector gby_exprs; bool is_final_mode; DAGSchema output_schema_for_partial_agg; + uint64_t fine_grained_shuffle_stream_count; private: void buildGroupBy(tipb::Aggregation * agg, int32_t collator_id, const Context & context) const; @@ -60,6 +62,6 @@ class AggregationBinder : public ExecutorBinder void buildAggFunc(tipb::Expr * agg_func, const ASTFunction * func, int32_t collator_id) const; }; -ExecutorBinderPtr compileAggregation(ExecutorBinderPtr input, size_t & executor_index, ASTPtr agg_funcs, ASTPtr group_by_exprs); +ExecutorBinderPtr compileAggregation(ExecutorBinderPtr input, size_t & executor_index, ASTPtr agg_funcs, ASTPtr group_by_exprs, uint64_t fine_grained_shuffle_stream_count = 0); } // namespace DB::mock diff --git a/dbms/src/Debug/MockExecutor/JoinBinder.cpp b/dbms/src/Debug/MockExecutor/JoinBinder.cpp index df0f11c2133..e9bc36bc5d0 100644 --- a/dbms/src/Debug/MockExecutor/JoinBinder.cpp +++ b/dbms/src/Debug/MockExecutor/JoinBinder.cpp @@ -140,6 +140,7 @@ bool JoinBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t collator { tipb_executor->set_tp(tipb::ExecType::TypeJoin); tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); tipb::Join * join = tipb_executor->mutable_join(); @@ -288,14 +289,15 @@ ExecutorBinderPtr compileJoin(size_t & executor_index, const ASTs & left_conds, const ASTs & right_conds, const ASTs & other_conds, - const ASTs & other_eq_conds_from_in) + const ASTs & other_eq_conds_from_in, + uint64_t fine_grained_shuffle_stream_count) { DAGSchema output_schema; buildLeftSideJoinSchema(output_schema, left->output_schema, tp); buildRightSideJoinSchema(output_schema, right->output_schema, tp); - auto join = std::make_shared(executor_index, output_schema, tp, join_cols, left_conds, right_conds, other_conds, other_eq_conds_from_in); + auto join = std::make_shared(executor_index, output_schema, tp, join_cols, left_conds, right_conds, other_conds, other_eq_conds_from_in, fine_grained_shuffle_stream_count); join->children.push_back(left); join->children.push_back(right); diff --git a/dbms/src/Debug/MockExecutor/JoinBinder.h b/dbms/src/Debug/MockExecutor/JoinBinder.h index c649420b8a9..cbdcd9d25b9 100644 --- a/dbms/src/Debug/MockExecutor/JoinBinder.h +++ b/dbms/src/Debug/MockExecutor/JoinBinder.h @@ -23,7 +23,7 @@ class ExchangeReceiverBinder; class JoinBinder : public ExecutorBinder { public: - JoinBinder(size_t & index_, const DAGSchema & output_schema_, tipb::JoinType tp_, const ASTs & join_cols_, const ASTs & l_conds, const ASTs & r_conds, const ASTs & o_conds, const ASTs & o_eq_conds) + JoinBinder(size_t & index_, const DAGSchema & output_schema_, tipb::JoinType tp_, const ASTs & join_cols_, const ASTs & l_conds, const ASTs & r_conds, const ASTs & o_conds, const ASTs & o_eq_conds, uint64_t fine_grained_shuffle_stream_count_) : ExecutorBinder(index_, "Join_" + std::to_string(index_), output_schema_) , tp(tp_) , join_cols(join_cols_) @@ -31,6 +31,7 @@ class JoinBinder : public ExecutorBinder , right_conds(r_conds) , other_conds(o_conds) , other_eq_conds_from_in(o_eq_conds) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { if (!(join_cols.size() + left_conds.size() + right_conds.size() + other_conds.size() + other_eq_conds_from_in.size())) throw Exception("No join condition found."); @@ -57,9 +58,10 @@ class JoinBinder : public ExecutorBinder const ASTs right_conds{}; const ASTs other_conds{}; const ASTs other_eq_conds_from_in{}; + uint64_t fine_grained_shuffle_stream_count; }; // compileJoin constructs a mocked Join executor node, note that all conditional expression params can be default -ExecutorBinderPtr compileJoin(size_t & executor_index, ExecutorBinderPtr left, ExecutorBinderPtr right, tipb::JoinType tp, const ASTs & join_cols, const ASTs & left_conds = {}, const ASTs & right_conds = {}, const ASTs & other_conds = {}, const ASTs & other_eq_conds_from_in = {}); +ExecutorBinderPtr compileJoin(size_t & executor_index, ExecutorBinderPtr left, ExecutorBinderPtr right, tipb::JoinType tp, const ASTs & join_cols, const ASTs & left_conds = {}, const ASTs & right_conds = {}, const ASTs & other_conds = {}, const ASTs & other_eq_conds_from_in = {}, uint64_t fine_grained_shuffle_stream_count = 0); /// Note: this api is only used by legacy test framework for compatibility purpose, which will be depracated soon, diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index e90e88f2289..ab53fe00392 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include namespace DB @@ -34,6 +35,24 @@ class ComputeServerRunner : public DB::tests::MPPTaskTestUtils {{"s1", TiDB::TP::TypeLong}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}, {toNullableVec("s1", {1, {}, 10000000, 10000000}), toNullableVec("s2", {"apple", {}, "banana", "test"}), toNullableVec("s3", {"apple", {}, "banana", "test"})}); + /// agg table with 200 rows + std::vector::FieldType>> agg_s1(200); + std::vector> agg_s2(200); + std::vector> agg_s3(200); + for (size_t i = 0; i < 200; ++i) + { + if (i % 30 != 0) + { + agg_s1[i] = i % 20; + agg_s2[i] = {fmt::format("val_{}", i % 10)}; + agg_s3[i] = {fmt::format("val_{}", i)}; + } + } + context.addMockTable( + {"test_db", "test_table_2"}, + {{"s1", TiDB::TP::TypeLong}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}, + {toNullableVec("s1", agg_s1), toNullableVec("s2", agg_s2), toNullableVec("s3", agg_s3)}); + /// for join context.addMockTable( {"test_db", "l_table"}, @@ -43,9 +62,46 @@ class ComputeServerRunner : public DB::tests::MPPTaskTestUtils {"test_db", "r_table"}, {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, {toNullableVec("s", {"banana", {}, "banana"}), toNullableVec("join_c", {"apple", {}, "banana"})}); + + /// join left table with 200 rows + std::vector::FieldType>> join_s1(200); + std::vector> join_s2(200); + std::vector> join_s3(200); + for (size_t i = 0; i < 200; ++i) + { + if (i % 20 != 0) + { + agg_s1[i] = i % 5; + agg_s2[i] = {fmt::format("val_{}", i % 6)}; + agg_s3[i] = {fmt::format("val_{}", i)}; + } + } + context.addMockTable( + {"test_db", "l_table_2"}, + {{"s1", TiDB::TP::TypeLong}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}, + {toNullableVec("s1", agg_s1), toNullableVec("s2", agg_s2), toNullableVec("s3", agg_s3)}); + + /// join right table with 100 rows + std::vector::FieldType>> join_r_s1(100); + std::vector> join_r_s2(100); + std::vector> join_r_s3(100); + for (size_t i = 0; i < 100; ++i) + { + if (i % 20 != 0) + { + join_r_s1[i] = i % 6; + join_r_s2[i] = {fmt::format("val_{}", i % 7)}; + join_r_s3[i] = {fmt::format("val_{}", i)}; + } + } + context.addMockTable( + {"test_db", "r_table_2"}, + {{"s1", TiDB::TP::TypeLong}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}, + {toNullableVec("s1", join_r_s1), toNullableVec("s2", join_r_s2), toNullableVec("s3", join_r_s3)}); } }; + TEST_F(ComputeServerRunner, runAggTasks) try { @@ -445,5 +501,121 @@ try } } CATCH + +/// For FineGrainedShuffleJoin/Agg test usage, update internal exchange senders/receivers flag +/// Allow select,agg,join,tableScan,exchangeSender,exchangeReceiver,projection executors only +void setFineGrainedShuffleForExchange(tipb::Executor & root) +{ + tipb::Executor * current = &root; + while (current) + { + switch (current->tp()) + { + case tipb::ExecType::TypeSelection: + current = const_cast(¤t->selection().child()); + break; + case tipb::ExecType::TypeAggregation: + current = const_cast(¤t->aggregation().child()); + break; + case tipb::ExecType::TypeProjection: + current = const_cast(¤t->projection().child()); + break; + case tipb::ExecType::TypeJoin: + { + /// update build side path + JoinInterpreterHelper::TiFlashJoin tiflash_join{current->join()}; + current = const_cast(¤t->join().children()[tiflash_join.build_side_index]); + break; + } + case tipb::ExecType::TypeExchangeSender: + if (current->exchange_sender().tp() == tipb::Hash) + current->set_fine_grained_shuffle_stream_count(8); + current = const_cast(¤t->exchange_sender().child()); + break; + case tipb::ExecType::TypeExchangeReceiver: + current->set_fine_grained_shuffle_stream_count(8); + current = nullptr; + break; + case tipb::ExecType::TypeTableScan: + current = nullptr; + break; + default: + throw TiFlashException("Should not reach here", Errors::Coprocessor::Internal); + } + } +} + +TEST_F(ComputeServerRunner, runFineGrainedShuffleJoinTest) +try +{ + startServers(3); + constexpr size_t join_type_num = 7; + constexpr tipb::JoinType join_types[join_type_num] = { + tipb::JoinType::TypeInnerJoin, + tipb::JoinType::TypeLeftOuterJoin, + tipb::JoinType::TypeRightOuterJoin, + tipb::JoinType::TypeSemiJoin, + tipb::JoinType::TypeAntiSemiJoin, + tipb::JoinType::TypeLeftOuterSemiJoin, + tipb::JoinType::TypeAntiLeftOuterSemiJoin, + }; + // fine-grained shuffle is enabled. + constexpr uint64_t enable = 8; + constexpr uint64_t disable = 0; + + for (auto join_type : join_types) + { + std::cout << "JoinType: " << static_cast(join_type) << std::endl; + auto properties = DB::tests::getDAGPropertiesForTest(serverNum()); + auto request = context + .scan("test_db", "l_table_2") + .join(context.scan("test_db", "r_table_2"), join_type, {col("s1"), col("s2")}, disable) + .project({col("l_table_2.s1"), col("l_table_2.s2"), col("l_table_2.s3")}); + const auto expected_cols = buildAndExecuteMPPTasks(request); + + auto request2 = context + .scan("test_db", "l_table_2") + .join(context.scan("test_db", "r_table_2"), join_type, {col("s1"), col("s2")}, enable) + .project({col("l_table_2.s1"), col("l_table_2.s2"), col("l_table_2.s3")}); + auto tasks = request2.buildMPPTasks(context, properties); + for (auto & task : tasks) + { + setFineGrainedShuffleForExchange(const_cast(task.dag_request->root_executor())); + } + const auto actual_cols = executeMPPTasks(tasks, properties, MockComputeServerManager::instance().getServerConfigMap()); + ASSERT_COLUMNS_EQ_UR(expected_cols, actual_cols); + } +} +CATCH + +TEST_F(ComputeServerRunner, runFineGrainedShuffleAggTest) +try +{ + startServers(3); + // fine-grained shuffle is enabled. + constexpr uint64_t enable = 8; + constexpr uint64_t disable = 0; + { + auto properties = DB::tests::getDAGPropertiesForTest(serverNum()); + auto request = context + .scan("test_db", "test_table_2") + .aggregation({Max(col("s3"))}, {col("s1"), col("s2")}, disable); + const auto expected_cols = buildAndExecuteMPPTasks(request); + + auto request2 = context + .scan("test_db", "test_table_2") + .aggregation({Max(col("s3"))}, {col("s1"), col("s2")}, enable); + auto tasks = request2.buildMPPTasks(context, properties); + for (auto & task : tasks) + { + setFineGrainedShuffleForExchange(const_cast(task.dag_request->root_executor())); + } + + const auto actual_cols = executeMPPTasks(tasks, properties, MockComputeServerManager::instance().getServerConfigMap()); + ASSERT_COLUMNS_EQ_UR(expected_cols, actual_cols); + } +} +CATCH + } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index 736166929bc..0afa65390ac 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -391,12 +391,110 @@ Union: } CATCH +TEST_F(InterpreterExecuteTest, FineGrainedShuffleJoin) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + { + // Join Source. + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r", enable); + + auto request = receiver1.join( + receiver2, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}, + enable) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + { + // Join Source. + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r", disable); + + auto request = receiver1.join( + receiver2, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}, + disable) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(InterpreterExecuteTest, FineGrainedShuffleAgg) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + { + DAGRequestBuilder receiver1 = context.receive("sender_1", enable); + auto request = receiver1 + .aggregation({Max(col("s1"))}, {col("s2")}, enable) + .build(context); + String expected = R"( +Union: + Expression x 10: + Aggregating: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + DAGRequestBuilder receiver1 = context.receive("sender_1", disable); + auto request = receiver1 + .aggregation({Max(col("s1"))}, {col("s2")}, disable) + .build(context); + String expected = R"( +Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + MockExchangeReceiver x 10)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + TEST_F(InterpreterExecuteTest, Join) try { // TODO: Find a way to write the request easier. { - // Join Source. + // join + ExchangeReceiver DAGRequestBuilder table1 = context.scan("test_db", "r_table"); DAGRequestBuilder table2 = context.scan("test_db", "l_table"); DAGRequestBuilder table3 = context.scan("test_db", "r_table"); diff --git a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp index e9f99891642..eb6de71ca4e 100644 --- a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -723,6 +723,108 @@ Union: } CATCH +TEST_F(PlannerInterpreterExecuteTest, FineGrainedShuffleJoin) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + { + // Join Source. + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r", enable); + + auto request = receiver1.join( + receiver2, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}, + enable) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + { + // Join Source. + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r", disable); + + auto request = receiver1.join( + receiver2, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}, + disable) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuild x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, FineGrainedShuffleAgg) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + { + DAGRequestBuilder receiver1 = context.receive("sender_1", enable); + auto request = receiver1 + .aggregation({Max(col("s1"))}, {col("s2")}, enable) + .build(context); + String expected = R"( +Union: + Expression x 10: + Expression: + Aggregating: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + DAGRequestBuilder receiver1 = context.receive("sender_1", disable); + auto request = receiver1 + .aggregation({Max(col("s1"))}, {col("s2")}, disable) + .build(context); + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + TEST_F(PlannerInterpreterExecuteTest, Join) try { diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 15d94e0c195..102fa05f9f6 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -1214,7 +1214,10 @@ void NO_INLINE joinBlockImplTypeCase( /// 2. In ExchangeReceiver, build_stream_id = packet_stream_id % build_stream_count; /// 3. In HashBuild, build_concurrency decides map's segment size, and build_steam_id decides the segment index auto packet_stream_id = shuffle_hash_data[i] % fine_grained_shuffle_count; - segment_index = packet_stream_id % segment_size; + if likely (fine_grained_shuffle_count == segment_size) + segment_index = packet_stream_id; + else + segment_index = packet_stream_id % segment_size; } else { diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 59000185cdf..66fabb59fbd 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -279,16 +279,16 @@ DAGRequestBuilder & DAGRequestBuilder::join( MockAstVec left_conds, MockAstVec right_conds, MockAstVec other_conds, - MockAstVec other_eq_conds_from_in) + MockAstVec other_eq_conds_from_in, + uint64_t fine_grained_shuffle_stream_count) { assert(root); assert(right.root); - - root = mock::compileJoin(getExecutorIndex(), root, right.root, tp, join_col_exprs, left_conds, right_conds, other_conds, other_eq_conds_from_in); + root = mock::compileJoin(getExecutorIndex(), root, right.root, tp, join_col_exprs, left_conds, right_conds, other_conds, other_eq_conds_from_in, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::aggregation(ASTPtr agg_func, ASTPtr group_by_expr) +DAGRequestBuilder & DAGRequestBuilder::aggregation(ASTPtr agg_func, ASTPtr group_by_expr, uint64_t fine_grained_shuffle_stream_count) { auto agg_funcs = std::make_shared(); auto group_by_exprs = std::make_shared(); @@ -296,10 +296,10 @@ DAGRequestBuilder & DAGRequestBuilder::aggregation(ASTPtr agg_func, ASTPtr group agg_funcs->children.push_back(agg_func); if (group_by_expr) group_by_exprs->children.push_back(group_by_expr); - return buildAggregation(agg_funcs, group_by_exprs); + return buildAggregation(agg_funcs, group_by_exprs, fine_grained_shuffle_stream_count); } -DAGRequestBuilder & DAGRequestBuilder::aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs) +DAGRequestBuilder & DAGRequestBuilder::aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs, uint64_t fine_grained_shuffle_stream_count) { auto agg_func_list = std::make_shared(); auto group_by_expr_list = std::make_shared(); @@ -307,13 +307,13 @@ DAGRequestBuilder & DAGRequestBuilder::aggregation(MockAstVec agg_funcs, MockAst agg_func_list->children.push_back(func); for (const auto & group_by : group_by_exprs) group_by_expr_list->children.push_back(group_by); - return buildAggregation(agg_func_list, group_by_expr_list); + return buildAggregation(agg_func_list, group_by_expr_list, fine_grained_shuffle_stream_count); } -DAGRequestBuilder & DAGRequestBuilder::buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs) +DAGRequestBuilder & DAGRequestBuilder::buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs, uint64_t fine_grained_shuffle_stream_count) { assert(root); - root = compileAggregation(root, getExecutorIndex(), agg_funcs, group_by_exprs); + root = compileAggregation(root, getExecutorIndex(), agg_funcs, group_by_exprs, fine_grained_shuffle_stream_count); return *this; } diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 14b314d9c20..8c9b2697ee3 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -122,16 +122,17 @@ class DAGRequestBuilder /// @param right_conds conditional expressions which only reference right table and the join type is right kind /// @param other_conds other conditional expressions /// @param other_eq_conds_from_in equality expressions within in subquery whose join type should be AntiSemiJoin, AntiLeftOuterSemiJoin or LeftOuterSemiJoin - DAGRequestBuilder & join(const DAGRequestBuilder & right, tipb::JoinType tp, MockAstVec join_col_exprs, MockAstVec left_conds, MockAstVec right_conds, MockAstVec other_conds, MockAstVec other_eq_conds_from_in); - DAGRequestBuilder & join(const DAGRequestBuilder & right, tipb::JoinType tp, MockAstVec join_col_exprs) + /// @param fine_grained_shuffle_stream_count decide the generated tipb executor's find_grained_shuffle_stream_count + DAGRequestBuilder & join(const DAGRequestBuilder & right, tipb::JoinType tp, MockAstVec join_col_exprs, MockAstVec left_conds, MockAstVec right_conds, MockAstVec other_conds, MockAstVec other_eq_conds_from_in, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & join(const DAGRequestBuilder & right, tipb::JoinType tp, MockAstVec join_col_exprs, uint64_t fine_grained_shuffle_stream_count = 0) { - return join(right, tp, join_col_exprs, {}, {}, {}, {}); + return join(right, tp, join_col_exprs, {}, {}, {}, {}, fine_grained_shuffle_stream_count); } // aggregation - DAGRequestBuilder & aggregation(ASTPtr agg_func, ASTPtr group_by_expr); - DAGRequestBuilder & aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs); + DAGRequestBuilder & aggregation(ASTPtr agg_func, ASTPtr group_by_expr, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs, uint64_t fine_grained_shuffle_stream_count = 0); // window DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); @@ -145,7 +146,7 @@ class DAGRequestBuilder private: void initDAGRequest(tipb::DAGRequest & dag_request); - DAGRequestBuilder & buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs); + DAGRequestBuilder & buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs, uint64_t fine_grained_shuffle_stream_count = 0); DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count = 0); mock::ExecutorBinderPtr root; From 816b8d5d5dff61b7fa452311c0b666947c34deb8 Mon Sep 17 00:00:00 2001 From: Meng Xin Date: Tue, 27 Dec 2022 22:46:16 +0800 Subject: [PATCH 41/42] merge output blocks if need in hash join (#6529) close pingcap/tiflash#6533 --- dbms/src/Core/Block.cpp | 39 ++++++++ dbms/src/Core/Block.h | 2 + .../HashJoinProbeBlockInputStream.cpp | 21 ++++- .../HashJoinProbeBlockInputStream.h | 4 +- .../SquashingHashJoinBlockTransform.cpp | 85 ++++++++++++++++++ .../SquashingHashJoinBlockTransform.h | 44 +++++++++ dbms/src/Flash/tests/gtest_join_executor.cpp | 21 ++++- .../gtest_squashing_hash_join_transform.cpp | 89 +++++++++++++++++++ .../tests/gtest_segment_test_basic.cpp | 4 +- dbms/src/TestUtils/ExecutorTestUtils.cpp | 6 +- dbms/src/TestUtils/ExecutorTestUtils.h | 2 - dbms/src/TestUtils/MPPTaskTestUtils.cpp | 2 +- 12 files changed, 307 insertions(+), 12 deletions(-) create mode 100644 dbms/src/DataStreams/SquashingHashJoinBlockTransform.cpp create mode 100644 dbms/src/DataStreams/SquashingHashJoinBlockTransform.h create mode 100644 dbms/src/Flash/tests/gtest_squashing_hash_join_transform.cpp diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 69fc45ec3c1..b8adade5a84 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -514,6 +514,45 @@ static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, cons return ReturnType(true); } +Block mergeBlocks(Blocks && blocks) +{ + if (blocks.empty()) + { + return {}; + } + + if (blocks.size() == 1) + { + return std::move(blocks[0]); + } + + auto & first_block = blocks[0]; + size_t result_rows = 0; + for (const auto & block : blocks) + { + result_rows += block.rows(); + } + + MutableColumns dst_columns(first_block.columns()); + + for (size_t i = 0; i < first_block.columns(); ++i) + { + dst_columns[i] = (*std::move(first_block.getByPosition(i).column)).mutate(); + dst_columns[i]->reserve(result_rows); + } + + for (size_t i = 1; i < blocks.size(); ++i) + { + if (likely(blocks[i].rows()) > 0) + { + for (size_t column = 0; column < blocks[i].columns(); ++column) + { + dst_columns[column]->insertRangeFrom(*blocks[i].getByPosition(column).column, 0, blocks[i].rows()); + } + } + } + return first_block.cloneWithColumns(std::move(dst_columns)); +} bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs) { diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 206d6d959cc..5f2cabe7859 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -149,6 +149,7 @@ class Block */ void updateHash(SipHash & hash) const; + private: void eraseImpl(size_t position); void initializeIndexByName(); @@ -157,6 +158,7 @@ class Block using Blocks = std::vector; using BlocksList = std::list; +Block mergeBlocks(Blocks && blocks); /// Compare number of columns, data types, column types, column names, and values of constant columns. bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs); diff --git a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp index b7ae64cfafc..2fd304b162f 100644 --- a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp +++ b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp @@ -13,7 +13,6 @@ // limitations under the License. #include -#include namespace DB { @@ -25,6 +24,7 @@ HashJoinProbeBlockInputStream::HashJoinProbeBlockInputStream( : log(Logger::get(req_id)) , join(join_) , probe_process_info(max_block_size) + , squashing_transform(max_block_size) { children.push_back(input); @@ -66,12 +66,30 @@ Block HashJoinProbeBlockInputStream::getHeader() const } Block HashJoinProbeBlockInputStream::readImpl() +{ + // if join finished, return {} directly. + if (squashing_transform.isJoinFinished()) + { + return Block{}; + } + + while (squashing_transform.needAppendBlock()) + { + Block result_block = getOutputBlock(); + squashing_transform.appendBlock(result_block); + } + return squashing_transform.getFinalOutputBlock(); +} + +Block HashJoinProbeBlockInputStream::getOutputBlock() { if (probe_process_info.all_rows_joined_finish) { Block block = children.back()->read(); if (!block) + { return block; + } join->checkTypes(block); probe_process_info.resetBlock(std::move(block)); } @@ -79,5 +97,4 @@ Block HashJoinProbeBlockInputStream::readImpl() return join->joinBlock(probe_process_info); } - } // namespace DB diff --git a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h index 3cc6fc4af6b..cf6e557d32c 100644 --- a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h +++ b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.h @@ -15,12 +15,12 @@ #pragma once #include +#include #include namespace DB { - /** Executes a certain expression over the block. * Basically the same as ExpressionBlockInputStream, * but requires that there must be a join probe action in the Expression. @@ -47,11 +47,13 @@ class HashJoinProbeBlockInputStream : public IProfilingBlockInputStream protected: Block readImpl() override; + Block getOutputBlock(); private: const LoggerPtr log; JoinPtr join; ProbeProcessInfo probe_process_info; + SquashingHashJoinBlockTransform squashing_transform; }; } // namespace DB diff --git a/dbms/src/DataStreams/SquashingHashJoinBlockTransform.cpp b/dbms/src/DataStreams/SquashingHashJoinBlockTransform.cpp new file mode 100644 index 00000000000..9c876d7883d --- /dev/null +++ b/dbms/src/DataStreams/SquashingHashJoinBlockTransform.cpp @@ -0,0 +1,85 @@ +// 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 + +namespace DB +{ + +SquashingHashJoinBlockTransform::SquashingHashJoinBlockTransform(UInt64 max_block_size_) + : output_rows(0) + , max_block_size(max_block_size_) + , join_finished(false) +{} + +void SquashingHashJoinBlockTransform::handleOverLimitBlock() +{ + // if over_limit_block is not null, we need to push it into blocks. + if (over_limit_block) + { + assert(!(output_rows && blocks.empty())); + output_rows += over_limit_block->rows(); + blocks.push_back(std::move(over_limit_block.value())); + over_limit_block.reset(); + } +} + +void SquashingHashJoinBlockTransform::appendBlock(Block & block) +{ + if (!block) + { + // if append block is {}, mark join finished. + join_finished = true; + return; + } + size_t current_rows = block.rows(); + + if (!output_rows || output_rows + current_rows <= max_block_size) + { + blocks.push_back(std::move(block)); + output_rows += current_rows; + } + else + { + // if output_rows + current_rows > max block size, put the current result block into over_limit_block and handle it in next read. + assert(!over_limit_block); + over_limit_block.emplace(std::move(block)); + } +} + +Block SquashingHashJoinBlockTransform::getFinalOutputBlock() +{ + Block final_block = mergeBlocks(std::move(blocks)); + reset(); + handleOverLimitBlock(); + return final_block; +} + +void SquashingHashJoinBlockTransform::reset() +{ + blocks.clear(); + output_rows = 0; +} + +bool SquashingHashJoinBlockTransform::isJoinFinished() const +{ + return join_finished; +} + +bool SquashingHashJoinBlockTransform::needAppendBlock() const +{ + return !over_limit_block && !join_finished; +} + +} // namespace DB diff --git a/dbms/src/DataStreams/SquashingHashJoinBlockTransform.h b/dbms/src/DataStreams/SquashingHashJoinBlockTransform.h new file mode 100644 index 00000000000..956dac0903f --- /dev/null +++ b/dbms/src/DataStreams/SquashingHashJoinBlockTransform.h @@ -0,0 +1,44 @@ +// 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. + +#pragma once + +#include + +namespace DB +{ + +class SquashingHashJoinBlockTransform +{ +public: + SquashingHashJoinBlockTransform(UInt64 max_block_size_); + + void appendBlock(Block & block); + Block getFinalOutputBlock(); + bool isJoinFinished() const; + bool needAppendBlock() const; + + +private: + void handleOverLimitBlock(); + void reset(); + + Blocks blocks; + std::optional over_limit_block; + size_t output_rows; + UInt64 max_block_size; + bool join_finished; +}; + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/tests/gtest_join_executor.cpp b/dbms/src/Flash/tests/gtest_join_executor.cpp index b7e3ff58683..e20a19b3174 100644 --- a/dbms/src/Flash/tests/gtest_join_executor.cpp +++ b/dbms/src/Flash/tests/gtest_join_executor.cpp @@ -700,7 +700,7 @@ CATCH TEST_F(JoinExecutorTestRunner, SplitJoinResult) try { - context.addMockTable("split_test", "t1", {{"a", TiDB::TP::TypeLong}}, {toVec("a", {1, 1, 1, 1, 1, 1, 1, 1, 1, 1})}); + context.addMockTable("split_test", "t1", {{"a", TiDB::TP::TypeLong}, {"b", TiDB::TP::TypeLong}}, {toVec("a", {1, 1, 1, 1, 1, 1, 1, 1, 1, 1}), toVec("b", {1, 1, 3, 3, 1, 1, 3, 3, 1, 3})}); context.addMockTable("split_test", "t2", {{"a", TiDB::TP::TypeLong}}, {toVec("a", {1, 1, 1, 1, 1})}); auto request = context @@ -720,6 +720,25 @@ try ASSERT_EQ(expect[i][j], blocks[j].rows()); } } + + // with other condition + const auto cond = gt(col("b"), lit(Field(static_cast(2)))); + request = context + .scan("split_test", "t1") + .join(context.scan("split_test", "t2"), tipb::JoinType::TypeInnerJoin, {col("a")}, {}, {}, {cond}, {}) + + .build(context); + expect = {{5, 5, 5, 5, 5}, {5, 5, 5, 5, 5}, {5, 5, 5, 5, 5}, {25}, {25}, {25}, {25}, {25}}; + for (size_t i = 0; i < block_sizes.size(); ++i) + { + context.context.setSetting("max_block_size", Field(static_cast(block_sizes[i]))); + auto blocks = getExecuteStreamsReturnBlocks(request); + ASSERT_EQ(expect[i].size(), blocks.size()); + for (size_t j = 0; j < blocks.size(); ++j) + { + ASSERT_EQ(expect[i][j], blocks[j].rows()); + } + } } CATCH diff --git a/dbms/src/Flash/tests/gtest_squashing_hash_join_transform.cpp b/dbms/src/Flash/tests/gtest_squashing_hash_join_transform.cpp new file mode 100644 index 00000000000..1f61878da48 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_squashing_hash_join_transform.cpp @@ -0,0 +1,89 @@ +// 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 + + +namespace DB +{ +namespace tests +{ +class SquashingHashJoinBlockTransformTest : public ::testing::Test +{ +public: + void SetUp() override {} + static ColumnWithTypeAndName toVec(const std::vector & v) + { + return createColumn(v); + } + + static void check(Blocks blocks, UInt64 max_block_size) + { + for (size_t i = 0; i < blocks.size(); ++i) + { + ASSERT(blocks[i].rows() <= max_block_size); + } + } +}; + +TEST_F(SquashingHashJoinBlockTransformTest, testALL) +try +{ + std::vector block_size{1, 5, 10, 99, 999, 9999, 39999, DEFAULT_BLOCK_SIZE}; + size_t merge_block_count = 10000; + + for (auto size : block_size) + { + Int64 expect_rows = 0; + Blocks test_blocks; + + for (size_t i = 0; i < merge_block_count; ++i) + { + size_t rand_block_size = std::rand() % size + 1; + expect_rows += rand_block_size; + std::vector values; + for (size_t j = 0; j < rand_block_size; ++j) + { + values.push_back(1); + } + Block block{toVec(values)}; + test_blocks.push_back(block); + } + test_blocks.push_back(Block{}); + + Blocks final_blocks; + size_t index = 0; + Int64 actual_rows = 0; + SquashingHashJoinBlockTransform squashing_transform(size); + while (!squashing_transform.isJoinFinished()) + { + while (squashing_transform.needAppendBlock()) + { + Block result_block = test_blocks[index++]; + squashing_transform.appendBlock(result_block); + } + final_blocks.push_back(squashing_transform.getFinalOutputBlock()); + actual_rows += final_blocks.back().rows(); + } + check(final_blocks, std::min(size, expect_rows)); + ASSERT(actual_rows == expect_rows); + } +} +CATCH + +} // namespace tests +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp index 617a1e7f13c..a890c868d0b 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -301,7 +301,7 @@ Block SegmentTestBasic::prepareWriteBlock(Int64 start_key, Int64 end_key, bool i is_deleted); } -Block mergeBlocks(std::vector && blocks) +Block sortMergeBlocks(std::vector && blocks) { auto accumulated_block = std::move(blocks[0]); @@ -391,7 +391,7 @@ Block SegmentTestBasic::prepareWriteBlockInSegmentRange(PageId segment_id, UInt6 remaining_rows); } - return mergeBlocks(std::move(blocks)); + return sortMergeBlocks(std::move(blocks)); } void SegmentTestBasic::writeSegment(PageId segment_id, UInt64 write_rows, std::optional start_at) diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index dc279f3ea7f..7719d021b37 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -166,7 +166,7 @@ void ExecutorTest::executeAndAssertRowsEqual(const std::shared_ptr streams) Blocks actual_blocks; for (const auto & stream : streams) readStream(actual_blocks, stream); - return mergeBlocks(actual_blocks).getColumnsWithTypeAndName(); + return mergeBlocksForTest(std::move(actual_blocks)).getColumnsWithTypeAndName(); } void ExecutorTest::enablePlanner(bool is_enable) @@ -238,7 +238,7 @@ ColumnsWithTypeAndName ExecutorTest::executeStreams(DAGContext * dag_context) // Currently, don't care about regions information in tests. Blocks blocks; queryExecute(context.context, /*internal=*/true)->execute([&blocks](const Block & block) { blocks.push_back(block); }).verify(); - return mergeBlocks(blocks).getColumnsWithTypeAndName(); + return mergeBlocksForTest(std::move(blocks)).getColumnsWithTypeAndName(); } Blocks ExecutorTest::getExecuteStreamsReturnBlocks(const std::shared_ptr & request, size_t concurrency) diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index ee014e4b069..79c279f2822 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -30,8 +30,6 @@ TiDB::TP dataTypeToTP(const DataTypePtr & type); ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream); ColumnsWithTypeAndName readBlocks(std::vector streams); -Block mergeBlocks(Blocks blocks); - #define WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN \ std::vector bools{false, true}; \ diff --git a/dbms/src/TestUtils/MPPTaskTestUtils.cpp b/dbms/src/TestUtils/MPPTaskTestUtils.cpp index d33ae8e5910..b187f3e6f5a 100644 --- a/dbms/src/TestUtils/MPPTaskTestUtils.cpp +++ b/dbms/src/TestUtils/MPPTaskTestUtils.cpp @@ -98,7 +98,7 @@ ColumnsWithTypeAndName extractColumns(Context & context, const std::shared_ptrchunks()) blocks.emplace_back(codec->decode(chunk.rows_data(), schema)); - return mergeBlocks(blocks).getColumnsWithTypeAndName(); + return mergeBlocks(std::move(blocks)).getColumnsWithTypeAndName(); } ColumnsWithTypeAndName MPPTaskTestUtils::executeCoprocessorTask(std::shared_ptr & dag_request) From d989239ad7739822dd458de3aa814da34c6accd0 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 28 Dec 2022 12:26:17 +0800 Subject: [PATCH 42/42] refine agg/topn and add more unit tests (#6549) ref pingcap/tiflash#5900 --- .../MergeSortingBlockInputStream.cpp | 114 ++--------------- .../MergeSortingBlockInputStream.h | 60 +-------- .../MergeSortingBlocksBlockInputStream.cpp | 118 ++++++++++++++++++ .../MergeSortingBlocksBlockInputStream.h | 81 ++++++++++++ .../MergingAndConvertingBlockInputStream.h | 0 .../tests/gtest_aggregation_executor.cpp | 72 ++++++++++- .../Flash/tests/gtest_projection_executor.cpp | 6 +- dbms/src/Flash/tests/gtest_topn_executor.cpp | 53 +++++++- dbms/src/Interpreters/Aggregator.cpp | 2 +- .../tests/gtest_segment_test_basic.cpp | 17 +-- dbms/src/TestUtils/ExecutorTestUtils.cpp | 32 +---- 11 files changed, 337 insertions(+), 218 deletions(-) create mode 100644 dbms/src/DataStreams/MergeSortingBlocksBlockInputStream.cpp create mode 100644 dbms/src/DataStreams/MergeSortingBlocksBlockInputStream.h rename dbms/src/{Interpreters => DataStreams}/MergingAndConvertingBlockInputStream.h (100%) diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index 9d7a87eabf1..6358f517408 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -13,17 +13,21 @@ // limitations under the License. #include +#include #include #include #include #include #include +#include namespace DB { +namespace +{ /** Remove constant columns from block. */ -static void removeConstantsFromBlock(Block & block) +void removeConstantsFromBlock(Block & block) { size_t columns = block.columns(); size_t i = 0; @@ -39,7 +43,7 @@ static void removeConstantsFromBlock(Block & block) } } -static void removeConstantsFromSortDescription(const Block & header, SortDescription & description) +void removeConstantsFromSortDescription(const Block & header, SortDescription & description) { description.erase( std::remove_if(description.begin(), description.end(), [&](const SortColumnDescription & elem) { @@ -54,7 +58,7 @@ static void removeConstantsFromSortDescription(const Block & header, SortDescrip /** Add into block, whose constant columns was removed by previous function, * constant columns from header (which must have structure as before removal of constants from block). */ -static void enrichBlockWithConstants(Block & block, const Block & header) +void enrichBlockWithConstants(Block & block, const Block & header) { size_t rows = block.rows(); size_t columns = header.columns(); @@ -66,7 +70,7 @@ static void enrichBlockWithConstants(Block & block, const Block & header) block.insert(i, {col_type_name.column->cloneResized(rows), col_type_name.type, col_type_name.name}); } } - +} // namespace MergeSortingBlockInputStream::MergeSortingBlockInputStream( const BlockInputStreamPtr & input, @@ -177,110 +181,8 @@ Block MergeSortingBlockInputStream::readImpl() return res; } - -MergeSortingBlocksBlockInputStream::MergeSortingBlocksBlockInputStream( - Blocks & blocks_, - SortDescription & description_, - const String & req_id, - size_t max_merged_block_size_, - size_t limit_) - : blocks(blocks_) - , header(blocks.at(0).cloneEmpty()) - , description(description_) - , max_merged_block_size(max_merged_block_size_) - , limit(limit_) - , log(Logger::get(req_id)) -{ - Blocks nonempty_blocks; - for (const auto & block : blocks) - { - if (block.rows() == 0) - continue; - - nonempty_blocks.push_back(block); - cursors.emplace_back(block, description); - has_collation |= cursors.back().has_collation; - } - - blocks.swap(nonempty_blocks); - - if (!has_collation) - { - for (auto & cursor : cursors) - queue.push(SortCursor(&cursor)); - } - else - { - for (auto & cursor : cursors) - queue_with_collation.push(SortCursorWithCollation(&cursor)); - } -} - - -Block MergeSortingBlocksBlockInputStream::readImpl() -{ - if (blocks.empty()) - return Block(); - - if (blocks.size() == 1) - { - Block res = blocks[0]; - blocks.clear(); - return res; - } - - return !has_collation - ? mergeImpl(queue) - : mergeImpl(queue_with_collation); -} - - -template -Block MergeSortingBlocksBlockInputStream::mergeImpl(std::priority_queue & queue) -{ - size_t num_columns = blocks[0].columns(); - - MutableColumns merged_columns = blocks[0].cloneEmptyColumns(); - /// TODO: reserve (in each column) - - /// Take rows from queue in right order and push to 'merged'. - size_t merged_rows = 0; - while (!queue.empty()) - { - TSortCursor current = queue.top(); - queue.pop(); - - for (size_t i = 0; i < num_columns; ++i) - merged_columns[i]->insertFrom(*current->all_columns[i], current->pos); - - if (!current->isLast()) - { - current->next(); - queue.push(current); - } - - ++total_merged_rows; - if (limit && total_merged_rows == limit) - { - auto res = blocks[0].cloneWithColumns(std::move(merged_columns)); - blocks.clear(); - return res; - } - - ++merged_rows; - if (merged_rows == max_merged_block_size) - return blocks[0].cloneWithColumns(std::move(merged_columns)); - } - - if (merged_rows == 0) - return {}; - - return blocks[0].cloneWithColumns(std::move(merged_columns)); -} - void MergeSortingBlockInputStream::appendInfo(FmtBuffer & buffer) const { buffer.fmtAppend(", limit = {}", limit); } - } // namespace DB diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index dd91576f8e1..3b75c80d472 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -21,69 +22,10 @@ #include #include #include -#include - -#include namespace DB { -/** Merges stream of sorted each-separately blocks to sorted as-a-whole stream of blocks. - * If data to sort is too much, could use external sorting, with temporary files. - */ - -/** Part of implementation. Merging array of ready (already read from somewhere) blocks. - * Returns result of merge as stream of blocks, not more than 'max_merged_block_size' rows in each. - */ -class MergeSortingBlocksBlockInputStream : public IProfilingBlockInputStream -{ - static constexpr auto NAME = "MergeSortingBlocks"; - -public: - /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. - MergeSortingBlocksBlockInputStream( - Blocks & blocks_, - SortDescription & description_, - const String & req_id, - size_t max_merged_block_size_, - size_t limit_ = 0); - - String getName() const override { return NAME; } - - bool isGroupedOutput() const override { return true; } - bool isSortedOutput() const override { return true; } - const SortDescription & getSortDescription() const override { return description; } - - Block getHeader() const override { return header; } - -protected: - Block readImpl() override; - -private: - Blocks & blocks; - Block header; - SortDescription description; - size_t max_merged_block_size; - size_t limit; - size_t total_merged_rows = 0; - - using CursorImpls = std::vector; - CursorImpls cursors; - - bool has_collation = false; - - std::priority_queue queue; - std::priority_queue queue_with_collation; - - /** Two different cursors are supported - with and without Collation. - * Templates are used (instead of virtual functions in SortCursor) for zero-overhead. - */ - template - Block mergeImpl(std::priority_queue & queue); - - LoggerPtr log; -}; - class MergeSortingBlockInputStream : public IProfilingBlockInputStream { static constexpr auto NAME = "MergeSorting"; diff --git a/dbms/src/DataStreams/MergeSortingBlocksBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlocksBlockInputStream.cpp new file mode 100644 index 00000000000..521b5e42eaa --- /dev/null +++ b/dbms/src/DataStreams/MergeSortingBlocksBlockInputStream.cpp @@ -0,0 +1,118 @@ +// 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 + +namespace DB +{ +MergeSortingBlocksBlockInputStream::MergeSortingBlocksBlockInputStream( + Blocks & blocks_, + SortDescription & description_, + const String & req_id, + size_t max_merged_block_size_, + size_t limit_) + : blocks(blocks_) + , header(blocks.at(0).cloneEmpty()) + , description(description_) + , max_merged_block_size(max_merged_block_size_) + , limit(limit_) + , log(Logger::get(req_id)) +{ + Blocks nonempty_blocks; + for (const auto & block : blocks) + { + if (block.rows() == 0) + continue; + + nonempty_blocks.push_back(block); + cursors.emplace_back(block, description); + has_collation |= cursors.back().has_collation; + } + + blocks.swap(nonempty_blocks); + + if (!has_collation) + { + for (auto & cursor : cursors) + queue.push(SortCursor(&cursor)); + } + else + { + for (auto & cursor : cursors) + queue_with_collation.push(SortCursorWithCollation(&cursor)); + } +} + + +Block MergeSortingBlocksBlockInputStream::readImpl() +{ + if (blocks.empty()) + return Block(); + + if (blocks.size() == 1) + { + Block res = blocks[0]; + blocks.clear(); + return res; + } + + return !has_collation + ? mergeImpl(queue) + : mergeImpl(queue_with_collation); +} + + +template +Block MergeSortingBlocksBlockInputStream::mergeImpl(std::priority_queue & queue) +{ + size_t num_columns = blocks[0].columns(); + + MutableColumns merged_columns = blocks[0].cloneEmptyColumns(); + /// TODO: reserve (in each column) + + /// Take rows from queue in right order and push to 'merged'. + size_t merged_rows = 0; + while (!queue.empty()) + { + TSortCursor current = queue.top(); + queue.pop(); + + for (size_t i = 0; i < num_columns; ++i) + merged_columns[i]->insertFrom(*current->all_columns[i], current->pos); + + if (!current->isLast()) + { + current->next(); + queue.push(current); + } + + ++total_merged_rows; + if (limit && total_merged_rows == limit) + { + auto res = blocks[0].cloneWithColumns(std::move(merged_columns)); + blocks.clear(); + return res; + } + + ++merged_rows; + if (merged_rows == max_merged_block_size) + return blocks[0].cloneWithColumns(std::move(merged_columns)); + } + + if (merged_rows == 0) + return {}; + + return blocks[0].cloneWithColumns(std::move(merged_columns)); +} +} // namespace DB diff --git a/dbms/src/DataStreams/MergeSortingBlocksBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlocksBlockInputStream.h new file mode 100644 index 00000000000..7656241436c --- /dev/null +++ b/dbms/src/DataStreams/MergeSortingBlocksBlockInputStream.h @@ -0,0 +1,81 @@ +// 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. + +#pragma once + +#include +#include +#include +#include + +#include + +namespace DB +{ +/** Merges stream of sorted each-separately blocks to sorted as-a-whole stream of blocks. + * If data to sort is too much, could use external sorting, with temporary files. + */ + +/** Part of implementation. Merging array of ready (already read from somewhere) blocks. + * Returns result of merge as stream of blocks, not more than 'max_merged_block_size' rows in each. + */ +class MergeSortingBlocksBlockInputStream : public IProfilingBlockInputStream +{ + static constexpr auto NAME = "MergeSortingBlocks"; + +public: + /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. + MergeSortingBlocksBlockInputStream( + Blocks & blocks_, + SortDescription & description_, + const String & req_id, + size_t max_merged_block_size_, + size_t limit_ = 0); + + String getName() const override { return NAME; } + + bool isGroupedOutput() const override { return true; } + bool isSortedOutput() const override { return true; } + const SortDescription & getSortDescription() const override { return description; } + + Block getHeader() const override { return header; } + +protected: + Block readImpl() override; + +private: + Blocks & blocks; + Block header; + SortDescription description; + size_t max_merged_block_size; + size_t limit; + size_t total_merged_rows = 0; + + using CursorImpls = std::vector; + CursorImpls cursors; + + bool has_collation = false; + + std::priority_queue queue; + std::priority_queue queue_with_collation; + + /** Two different cursors are supported - with and without Collation. + * Templates are used (instead of virtual functions in SortCursor) for zero-overhead. + */ + template + Block mergeImpl(std::priority_queue & queue); + + LoggerPtr log; +}; +} // namespace DB diff --git a/dbms/src/Interpreters/MergingAndConvertingBlockInputStream.h b/dbms/src/DataStreams/MergingAndConvertingBlockInputStream.h similarity index 100% rename from dbms/src/Interpreters/MergingAndConvertingBlockInputStream.h rename to dbms/src/DataStreams/MergingAndConvertingBlockInputStream.h diff --git a/dbms/src/Flash/tests/gtest_aggregation_executor.cpp b/dbms/src/Flash/tests/gtest_aggregation_executor.cpp index 4f033e6afe5..b95f847c42a 100644 --- a/dbms/src/Flash/tests/gtest_aggregation_executor.cpp +++ b/dbms/src/Flash/tests/gtest_aggregation_executor.cpp @@ -134,6 +134,50 @@ class ExecutorAggTestRunner : public ExecutorTest toVec("c4_str", {"1", "2 ", "2 "}), toVec("c5_date_time", {2000000, 12000000, 12000000}), }); + + /// agg table with 200 rows + { + // with 15 types of key. + std::vector::FieldType>> key(200); + std::vector> value(200); + for (size_t i = 0; i < 200; ++i) + { + key[i] = i % 15; + value[i] = {fmt::format("val_{}", i)}; + } + context.addMockTable( + {"test_db", "big_table_1"}, + {{"key", TiDB::TP::TypeLong}, {"value", TiDB::TP::TypeString}}, + {toNullableVec("key", key), toNullableVec("value", value)}); + } + { + // with 200 types of key. + std::vector::FieldType>> key(200); + std::vector> value(200); + for (size_t i = 0; i < 200; ++i) + { + key[i] = i; + value[i] = {fmt::format("val_{}", i)}; + } + context.addMockTable( + {"test_db", "big_table_2"}, + {{"key", TiDB::TP::TypeLong}, {"value", TiDB::TP::TypeString}}, + {toNullableVec("key", key), toNullableVec("value", value)}); + } + { + // with 1 types of key. + std::vector::FieldType>> key(200); + std::vector> value(200); + for (size_t i = 0; i < 200; ++i) + { + key[i] = 0; + value[i] = {fmt::format("val_{}", i)}; + } + context.addMockTable( + {"test_db", "big_table_3"}, + {{"key", TiDB::TP::TypeLong}, {"value", TiDB::TP::TypeString}}, + {toNullableVec("key", key), toNullableVec("value", value)}); + } } std::shared_ptr buildDAGRequest(std::pair src, MockAstVec agg_funcs, MockAstVec group_by_exprs, MockColumnNameVec proj) @@ -342,6 +386,9 @@ try } CATCH +// TODO support more type of min, max, count. +// support more aggregation functions: sum, forst_row, group_concat + TEST_F(ExecutorAggTestRunner, AggregationCountGroupByFastPathMultiKeys) try { @@ -510,8 +557,29 @@ try } CATCH -// TODO support more type of min, max, count. -// support more aggregation functions: sum, forst_row, group_concat +TEST_F(ExecutorAggTestRunner, AggMerge) +try +{ + std::vector tables{"big_table_1", "big_table_2", "big_table_3"}; + for (const auto & table : tables) + { + auto request = context + .scan("test_db", table) + .aggregation({Max(col("value"))}, {col("key")}) + .build(context); + auto expect = executeStreams(request, 1); + context.context.setSetting("group_by_two_level_threshold_bytes", Field(static_cast(0))); + // 0: use one level merge + // 1: use two level merge + std::vector two_level_thresholds{0, 1}; + for (auto two_level_threshold : two_level_thresholds) + { + context.context.setSetting("group_by_two_level_threshold", Field(static_cast(two_level_threshold))); + executeAndAssertColumnsEqual(request, expect); + } + } +} +CATCH } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_projection_executor.cpp b/dbms/src/Flash/tests/gtest_projection_executor.cpp index d7e90b6cb51..599044f5503 100644 --- a/dbms/src/Flash/tests/gtest_projection_executor.cpp +++ b/dbms/src/Flash/tests/gtest_projection_executor.cpp @@ -343,7 +343,7 @@ try .scan("test_db", "test_table3") .project({lit(Field(String("a")))}) .build(context); - executeAndAssertColumnsEqual(req, {createColumns({toVec({"a", "a", "a", "a", "a"})})}); + executeAndAssertColumnsEqual(req, {createColumns({createConstColumn(5, "a")})}); req = context .scan("test_db", "test_table3") @@ -351,7 +351,7 @@ try .project(MockAstVec{}) .project({lit(Field(String("a")))}) .build(context); - executeAndAssertColumnsEqual(req, {createColumns({toVec({"a", "a", "a", "a", "a"})})}); + executeAndAssertColumnsEqual(req, {createColumns({createConstColumn(5, "a")})}); req = context .scan("test_db", "test_table3") @@ -359,7 +359,7 @@ try .project(MockAstVec{}) .project({lit(Field(String("a")))}) .build(context); - executeAndAssertColumnsEqual(req, {createColumns({toVec({"a", "a", "a", "a", "a"})})}); + executeAndAssertColumnsEqual(req, {createColumns({createConstColumn(5, "a")})}); req = context .scan("test_db", "test_table3") diff --git a/dbms/src/Flash/tests/gtest_topn_executor.cpp b/dbms/src/Flash/tests/gtest_topn_executor.cpp index 461f0e8b4c9..407676c8a3e 100644 --- a/dbms/src/Flash/tests/gtest_topn_executor.cpp +++ b/dbms/src/Flash/tests/gtest_topn_executor.cpp @@ -19,7 +19,6 @@ namespace DB { namespace tests { - class ExecutorTopNTestRunner : public DB::tests::ExecutorTest { public: @@ -45,6 +44,38 @@ class ExecutorTopNTestRunner : public DB::tests::ExecutorTest toNullableVec(col_name[1], col_gender), toNullableVec(col_name[2], col_country), toNullableVec(col_name[3], col_salary)}); + + /// table with 200 rows + { + // with 15 types of key. + std::vector::FieldType>> key(200); + for (size_t i = 0; i < 200; ++i) + key[i] = i % 15; + context.addMockTable( + {"test_db", "big_table_1"}, + {{"key", TiDB::TP::TypeLong}}, + {toNullableVec("key", key)}); + } + { + // with 200 types of key. + std::vector::FieldType>> key(200); + for (size_t i = 0; i < 200; ++i) + key[i] = i; + context.addMockTable( + {"test_db", "big_table_2"}, + {{"key", TiDB::TP::TypeLong}}, + {toNullableVec("key", key)}); + } + { + // with 1 types of key. + std::vector::FieldType>> key(200); + for (size_t i = 0; i < 200; ++i) + key[i] = 0; + context.addMockTable( + {"test_db", "big_table_3"}, + {{"key", TiDB::TP::TypeLong}}, + {toNullableVec("key", key)}); + } } std::shared_ptr buildDAGRequest(const String & table_name, const String & col_name, bool is_desc, int limit_num) @@ -214,5 +245,25 @@ try } CATCH +TEST_F(ExecutorTopNTestRunner, BigTable) +try +{ + std::vector tables{"big_table_1", "big_table_2", "big_table_3"}; + for (const auto & table : tables) + { + std::vector limits{0, 1, 10, 20, 199, 200, 300}; + for (auto limit_num : limits) + { + auto request = context + .scan("test_db", table) + .topN("key", false, limit_num) + .build(context); + auto expect = executeStreams(request, 1); + executeAndAssertColumnsEqual(request, expect); + } + } +} +CATCH + } // namespace tests } // namespace DB diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 4a9ffc1c993..ba7d1b5e32a 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -32,7 +33,6 @@ #include #include #include -#include #include #include diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp index a890c868d0b..85c54c5e7c0 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -303,22 +303,7 @@ Block SegmentTestBasic::prepareWriteBlock(Int64 start_key, Int64 end_key, bool i Block sortMergeBlocks(std::vector && blocks) { - auto accumulated_block = std::move(blocks[0]); - - for (size_t block_idx = 1; block_idx < blocks.size(); ++block_idx) - { - auto block = std::move(blocks[block_idx]); - - size_t columns = block.columns(); - size_t rows = block.rows(); - - for (size_t i = 0; i < columns; ++i) - { - MutableColumnPtr mutable_column = (*std::move(accumulated_block.getByPosition(i).column)).mutate(); - mutable_column->insertRangeFrom(*block.getByPosition(i).column, 0, rows); - accumulated_block.getByPosition(i).column = std::move(mutable_column); - } - } + auto accumulated_block = mergeBlocks(std::move(blocks)); SortDescription sort; sort.emplace_back(EXTRA_HANDLE_COLUMN_NAME, 1, 0); diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 7719d021b37..7ca140ee650 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -166,34 +166,6 @@ void ExecutorTest::executeAndAssertRowsEqual(const std::shared_ptr actual_cols; - for (const auto & column : sample_block.getColumnsWithTypeAndName()) - { - actual_cols.push_back(column.type->createColumn()); - } - for (const auto & block : blocks) - { - for (size_t i = 0; i < block.columns(); ++i) - { - for (size_t j = 0; j < block.rows(); ++j) - { - actual_cols[i]->insert((*(block.getColumnsWithTypeAndName())[i].column)[j]); - } - } - } - - ColumnsWithTypeAndName actual_columns; - for (size_t i = 0; i < actual_cols.size(); ++i) - actual_columns.push_back({std::move(actual_cols[i]), sample_block.getColumnsWithTypeAndName()[i].type, sample_block.getColumnsWithTypeAndName()[i].name, sample_block.getColumnsWithTypeAndName()[i].column_id}); - return Block(actual_columns); -} - void readStream(Blocks & blocks, BlockInputStreamPtr stream) { stream->readPrefix(); @@ -214,7 +186,7 @@ DB::ColumnsWithTypeAndName readBlocks(std::vector streams) Blocks actual_blocks; for (const auto & stream : streams) readStream(actual_blocks, stream); - return mergeBlocksForTest(std::move(actual_blocks)).getColumnsWithTypeAndName(); + return mergeBlocks(std::move(actual_blocks)).getColumnsWithTypeAndName(); } void ExecutorTest::enablePlanner(bool is_enable) @@ -238,7 +210,7 @@ ColumnsWithTypeAndName ExecutorTest::executeStreams(DAGContext * dag_context) // Currently, don't care about regions information in tests. Blocks blocks; queryExecute(context.context, /*internal=*/true)->execute([&blocks](const Block & block) { blocks.push_back(block); }).verify(); - return mergeBlocksForTest(std::move(blocks)).getColumnsWithTypeAndName(); + return mergeBlocks(std::move(blocks)).getColumnsWithTypeAndName(); } Blocks ExecutorTest::getExecuteStreamsReturnBlocks(const std::shared_ptr & request, size_t concurrency)