From 6c31bd874af8d144f33af023def34286c0aeb6f6 Mon Sep 17 00:00:00 2001 From: stdpain Date: Tue, 6 Feb 2024 14:52:19 +0800 Subject: [PATCH] [Feature] support Group execution Signed-off-by: stdpain --- be/src/connector/connector.cpp | 3 +- be/src/exec/CMakeLists.txt | 3 + .../aggregate/aggregate_streaming_node.cpp | 5 +- be/src/exec/connector_scan_node.cpp | 3 + be/src/exec/cross_join_node.cpp | 6 +- be/src/exec/exchange_node.cpp | 3 +- be/src/exec/exec_node.cpp | 2 +- be/src/exec/hash_join_node.cpp | 77 +++++---- be/src/exec/hash_joiner.cpp | 1 + be/src/exec/olap_scan_node.cpp | 3 + be/src/exec/olap_scan_prepare.cpp | 4 +- be/src/exec/olap_scan_prepare.h | 1 + be/src/exec/pipeline/exchange/sink_buffer.cpp | 2 +- be/src/exec/pipeline/fragment_context.cpp | 100 ++++++++---- be/src/exec/pipeline/fragment_context.h | 63 +++---- be/src/exec/pipeline/fragment_executor.cpp | 87 +++------- .../group_execution/execution_group.cpp | 154 ++++++++++++++++++ .../group_execution/execution_group.h | 145 +++++++++++++++++ .../execution_group_builder.cpp | 38 +++++ .../group_execution/execution_group_builder.h | 41 +++++ .../group_execution/execution_group_fwd.h | 24 +++ .../group_execution/group_operator.cpp | 72 ++++++++ .../pipeline/group_execution/group_operator.h | 78 +++++++++ .../hashjoin/hash_join_build_operator.cpp | 69 +++++--- .../exec/pipeline/olap_table_sink_operator.h | 2 - be/src/exec/pipeline/operator.cpp | 33 +++- be/src/exec/pipeline/operator.h | 5 + be/src/exec/pipeline/pipeline.cpp | 14 +- be/src/exec/pipeline/pipeline.h | 8 +- be/src/exec/pipeline/pipeline_builder.cpp | 84 +++++++++- be/src/exec/pipeline/pipeline_builder.h | 55 +++++-- be/src/exec/pipeline/pipeline_driver.cpp | 5 +- be/src/exec/pipeline/pipeline_driver.h | 2 +- be/src/exec/pipeline/query_context.cpp | 4 +- be/src/exec/pipeline/runtime_filter_types.h | 84 +++++++--- .../exec/pipeline/scan/olap_scan_context.cpp | 3 +- be/src/exec/pipeline/scan/olap_scan_context.h | 2 +- .../scan/olap_scan_prepare_operator.cpp | 2 +- be/src/exec/pipeline/stream_epoch_manager.cpp | 4 +- be/src/exec/topn_node.cpp | 2 + be/src/exec/union_node.cpp | 8 +- be/src/exprs/runtime_filter.h | 7 + be/src/exprs/runtime_filter_bank.cpp | 21 +-- be/src/exprs/runtime_filter_bank.h | 50 ++++-- be/src/formats/orc/orc_chunk_reader.cpp | 2 +- be/src/formats/parquet/file_reader.cpp | 3 +- be/src/runtime/runtime_filter_worker.cpp | 9 + be/src/runtime/runtime_filter_worker.h | 1 + be/src/storage/olap_runtime_range_pruner.h | 7 +- be/src/storage/olap_runtime_range_pruner.hpp | 16 +- .../pipeline/pipeline_control_flow_test.cpp | 10 +- .../pipeline/pipeline_file_scan_node_test.cpp | 39 ++--- be/test/exec/pipeline/pipeline_test_base.cpp | 36 ++-- be/test/exec/pipeline/pipeline_test_base.h | 5 + be/test/exec/query_cache/query_cache_test.cpp | 4 +- be/test/exec/stream/stream_operators_test.cpp | 46 +++--- be/test/exec/stream/stream_pipeline_test.cpp | 52 +++--- be/test/exec/stream/stream_pipeline_test.h | 29 ++-- .../starrocks/planner/AggregationNode.java | 8 +- .../starrocks/planner/AnalyticEvalNode.java | 6 +- .../com/starrocks/planner/DecodeNode.java | 10 +- .../com/starrocks/planner/ExchangeNode.java | 14 +- .../java/com/starrocks/planner/ExecGroup.java | 98 +++++++++++ .../com/starrocks/planner/ExecGroupId.java | 38 +++++ .../com/starrocks/planner/ExecGroupSets.java | 51 ++++++ .../java/com/starrocks/planner/JoinNode.java | 25 +-- .../starrocks/planner/NestLoopJoinNode.java | 9 +- .../com/starrocks/planner/PlanFragment.java | 50 +++++- .../java/com/starrocks/planner/PlanNode.java | 26 +-- .../com/starrocks/planner/ProjectNode.java | 10 +- .../planner/RuntimeFilterBuildNode.java | 4 +- .../planner/RuntimeFilterDescription.java | 28 +++- .../planner/RuntimeFilterPushDownContext.java | 45 +++++ .../starrocks/planner/SetOperationNode.java | 8 +- .../java/com/starrocks/planner/SortNode.java | 7 +- .../com/starrocks/qe/SessionVariable.java | 29 ++++ .../LocalFragmentAssignmentStrategy.java | 53 ++++-- .../java/com/starrocks/sql/plan/ExecPlan.java | 9 + .../sql/plan/PlanFragmentBuilder.java | 110 +++++++++++-- .../starrocks/sql/plan/ColocateJoinTest.java | 4 +- .../sql/plan/GroupExecutionPlanTest.java | 95 +++++++++++ gensrc/thrift/Planner.thrift | 11 ++ gensrc/thrift/RuntimeFilter.thrift | 2 +- .../T/test_group_execution_agg | 14 ++ .../T/test_group_execution_combine | 0 .../T/test_group_execution_join | 66 ++++++++ 86 files changed, 1974 insertions(+), 494 deletions(-) create mode 100644 be/src/exec/pipeline/group_execution/execution_group.cpp create mode 100644 be/src/exec/pipeline/group_execution/execution_group.h create mode 100644 be/src/exec/pipeline/group_execution/execution_group_builder.cpp create mode 100644 be/src/exec/pipeline/group_execution/execution_group_builder.h create mode 100644 be/src/exec/pipeline/group_execution/execution_group_fwd.h create mode 100644 be/src/exec/pipeline/group_execution/group_operator.cpp create mode 100644 be/src/exec/pipeline/group_execution/group_operator.h create mode 100644 fe/fe-core/src/main/java/com/starrocks/planner/ExecGroup.java create mode 100644 fe/fe-core/src/main/java/com/starrocks/planner/ExecGroupId.java create mode 100644 fe/fe-core/src/main/java/com/starrocks/planner/ExecGroupSets.java create mode 100644 fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterPushDownContext.java create mode 100644 fe/fe-core/src/test/java/com/starrocks/sql/plan/GroupExecutionPlanTest.java create mode 100644 test/sql/test_group_execution/T/test_group_execution_agg create mode 100644 test/sql/test_group_execution/T/test_group_execution_combine create mode 100644 test/sql/test_group_execution/T/test_group_execution_join diff --git a/be/src/connector/connector.cpp b/be/src/connector/connector.cpp index 6df102d2ec60b1..bb4d842f085502 100644 --- a/be/src/connector/connector.cpp +++ b/be/src/connector/connector.cpp @@ -80,7 +80,8 @@ Status DataSource::parse_runtime_filters(RuntimeState* state) { if (_runtime_filters == nullptr || _runtime_filters->size() == 0) return Status::OK(); for (const auto& item : _runtime_filters->descriptors()) { RuntimeFilterProbeDescriptor* probe = item.second; - const JoinRuntimeFilter* filter = probe->runtime_filter(); + DCHECK(runtime_bloom_filter_eval_context.driver_sequence == -1); + const JoinRuntimeFilter* filter = probe->runtime_filter(runtime_bloom_filter_eval_context.driver_sequence); if (filter == nullptr) continue; SlotId slot_id; if (!probe->is_probe_slot_ref(&slot_id)) continue; diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt index 43115c253fc905..ed5a4f49d69fa6 100644 --- a/be/src/exec/CMakeLists.txt +++ b/be/src/exec/CMakeLists.txt @@ -269,6 +269,9 @@ set(EXEC_FILES pipeline/pipeline.cpp pipeline/spill_process_operator.cpp pipeline/spill_process_channel.cpp + pipeline/group_execution/execution_group.cpp + pipeline/group_execution/execution_group_builder.cpp + pipeline/group_execution/group_operator.cpp workgroup/work_group.cpp workgroup/scan_executor.cpp workgroup/scan_task_queue.cpp diff --git a/be/src/exec/aggregate/aggregate_streaming_node.cpp b/be/src/exec/aggregate/aggregate_streaming_node.cpp index ce0854ec904b15..3de35b514d220b 100644 --- a/be/src/exec/aggregate/aggregate_streaming_node.cpp +++ b/be/src/exec/aggregate/aggregate_streaming_node.cpp @@ -213,8 +213,9 @@ pipeline::OpFactories AggregateStreamingNode::decompose_to_pipeline(pipeline::Pi size_t degree_of_parallelism = context->source_operator(ops_with_sink)->degree_of_parallelism(); auto should_cache = context->should_interpolate_cache_operator(id(), ops_with_sink[0]); - if (!should_cache && _tnode.agg_node.__isset.interpolate_passthrough && _tnode.agg_node.interpolate_passthrough && - context->could_local_shuffle(ops_with_sink)) { + bool could_local_shuffle = !should_cache && !context->enable_group_execution(); + if (could_local_shuffle && _tnode.agg_node.__isset.interpolate_passthrough && + _tnode.agg_node.interpolate_passthrough && context->could_local_shuffle(ops_with_sink)) { ops_with_sink = context->maybe_interpolate_local_passthrough_exchange(runtime_state(), id(), ops_with_sink, degree_of_parallelism, true); } diff --git a/be/src/exec/connector_scan_node.cpp b/be/src/exec/connector_scan_node.cpp index 57e12878ef1ff0..7ff5a8208edfab 100644 --- a/be/src/exec/connector_scan_node.cpp +++ b/be/src/exec/connector_scan_node.cpp @@ -109,6 +109,9 @@ int ConnectorScanNode::_estimate_max_concurrent_chunks() const { } pipeline::OpFactories ConnectorScanNode::decompose_to_pipeline(pipeline::PipelineBuilderContext* context) { + auto exec_group = context->find_exec_group_by_plan_node_id(_id); + context->set_current_execution_group(exec_group); + size_t dop = context->dop_of_source_operator(id()); std::shared_ptr scan_op = nullptr; bool stream_data_source = _data_source_provider->stream_data_source(); diff --git a/be/src/exec/cross_join_node.cpp b/be/src/exec/cross_join_node.cpp index b3f5066ba69bea..b8f3a8f990a2c7 100644 --- a/be/src/exec/cross_join_node.cpp +++ b/be/src/exec/cross_join_node.cpp @@ -609,8 +609,10 @@ std::vector> CrossJoinNode::_decompos std::move(_conjunct_ctxs), std::move(cross_join_context), _join_op); // Initialize OperatorFactory's fields involving runtime filters. this->init_runtime_filter_for_operator(left_factory.get(), context, rc_rf_probe_collector); - left_ops = context->maybe_interpolate_local_adpative_passthrough_exchange(runtime_state(), id(), left_ops, - context->degree_of_parallelism()); + if (context->is_colocate_group()) { + left_ops = context->maybe_interpolate_local_adpative_passthrough_exchange(runtime_state(), id(), left_ops, + context->degree_of_parallelism()); + } left_ops.emplace_back(std::move(left_factory)); if (limit() != -1) { diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp index 53d6d1ddd85b57..f8999e2869c602 100644 --- a/be/src/exec/exchange_node.cpp +++ b/be/src/exec/exchange_node.cpp @@ -247,7 +247,8 @@ void ExchangeNode::debug_string(int indentation_level, std::stringstream* out) c pipeline::OpFactories ExchangeNode::decompose_to_pipeline(pipeline::PipelineBuilderContext* context) { using namespace pipeline; - + auto exec_group = context->find_exec_group_by_plan_node_id(_id); + context->set_current_execution_group(exec_group); OpFactories operators; if (!_is_merging) { auto* query_ctx = context->runtime_state()->query_ctx(); diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 8500dfd7d7e441..ba5f36050d7c88 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -220,7 +220,7 @@ Status ExecNode::prepare(RuntimeState* state) { _mem_tracker.reset(new MemTracker(_runtime_profile.get(), std::make_tuple(true, false, false), "", -1, _runtime_profile->name(), nullptr)); RETURN_IF_ERROR(Expr::prepare(_conjunct_ctxs, state)); - RETURN_IF_ERROR(_runtime_filter_collector.prepare(state, row_desc(), _runtime_profile.get())); + RETURN_IF_ERROR(_runtime_filter_collector.prepare(state, _runtime_profile.get())); // TODO(zc): // AddExprCtxsToFree(_conjunct_ctxs); diff --git a/be/src/exec/hash_join_node.cpp b/be/src/exec/hash_join_node.cpp index a0d4de895ac4ff..e1cb06a205f5d5 100644 --- a/be/src/exec/hash_join_node.cpp +++ b/be/src/exec/hash_join_node.cpp @@ -25,6 +25,8 @@ #include "exec/hash_joiner.h" #include "exec/pipeline/chunk_accumulate_operator.h" #include "exec/pipeline/exchange/exchange_source_operator.h" +#include "exec/pipeline/group_execution/execution_group_builder.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" #include "exec/pipeline/hashjoin/hash_join_build_operator.h" #include "exec/pipeline/hashjoin/hash_join_probe_operator.h" #include "exec/pipeline/hashjoin/hash_joiner_factory.h" @@ -38,6 +40,8 @@ #include "exprs/expr.h" #include "exprs/in_const_predicate.hpp" #include "exprs/runtime_filter_bank.h" +#include "gen_cpp/PlanNodes_types.h" +#include "gen_cpp/RuntimeFilter_types.h" #include "gutil/strings/substitute.h" #include "runtime/current_thread.h" #include "runtime/runtime_filter_worker.h" @@ -425,30 +429,27 @@ void HashJoinNode::close(RuntimeState* state) { template pipeline::OpFactories HashJoinNode::_decompose_to_pipeline(pipeline::PipelineBuilderContext* context) { using namespace pipeline; - auto rhs_operators = child(1)->decompose_to_pipeline(context); + // "col NOT IN (NULL, val1, val2)" always returns false, so hash join should + // return empty result in this case. Hash join cannot be divided into multiple + // partitions in this case. Otherwise, NULL value in right table will only occur + // in some partition hash table, and other partition hash table can output chunk. + // TODO: support nullaware left anti join with shuffle join + DCHECK(_join_type != TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || _distribution_mode == TJoinDistributionMode::BROADCAST); if (_distribution_mode == TJoinDistributionMode::BROADCAST) { // Broadcast join need only create one hash table, because all the HashJoinProbeOperators // use the same hash table with their own different probe states. rhs_operators = context->maybe_interpolate_local_passthrough_exchange(runtime_state(), id(), rhs_operators); } else { - // "col NOT IN (NULL, val1, val2)" always returns false, so hash join should - // return empty result in this case. Hash join cannot be divided into multiple - // partitions in this case. Otherwise, NULL value in right table will only occur - // in some partition hash table, and other partition hash table can output chunk. - if (_join_type == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { - rhs_operators = context->maybe_interpolate_local_passthrough_exchange(runtime_state(), id(), rhs_operators); - } else { - // Both HashJoin{Build, Probe}Operator are parallelized - // There are two ways of shuffle - // 1. If previous op is ExchangeSourceOperator and its partition type is HASH_PARTITIONED or BUCKET_SHUFFLE_HASH_PARTITIONED - // then pipeline level shuffle will be performed at sender side (ExchangeSinkOperator), so - // there is no need to perform local shuffle again at receiver side - // 2. Otherwise, add LocalExchangeOperator - // to shuffle multi-stream into #degree_of_parallelism# streams each of that pipes into HashJoin{Build, Probe}Operator. - rhs_operators = context->maybe_interpolate_local_shuffle_exchange(runtime_state(), id(), rhs_operators, - _build_equivalence_partition_expr_ctxs); - } + // Both HashJoin{Build, Probe}Operator are parallelized + // There are two ways of shuffle + // 1. If previous op is ExchangeSourceOperator and its partition type is HASH_PARTITIONED or BUCKET_SHUFFLE_HASH_PARTITIONED + // then pipeline level shuffle will be performed at sender side (ExchangeSinkOperator), so + // there is no need to perform local shuffle again at receiver side + // 2. Otherwise, add LocalExchangeOperator + // to shuffle multi-stream into #degree_of_parallelism# streams each of that pipes into HashJoin{Build, Probe}Operator. + rhs_operators = context->maybe_interpolate_local_shuffle_exchange(runtime_state(), id(), rhs_operators, + _build_equivalence_partition_expr_ctxs); } size_t num_right_partitions = context->source_operator(rhs_operators)->degree_of_parallelism(); @@ -468,12 +469,8 @@ pipeline::OpFactories HashJoinNode::_decompose_to_pipeline(pipeline::PipelineBui _build_runtime_filters, _output_slots, _output_slots, _distribution_mode, false); auto hash_joiner_factory = std::make_shared(param); - // add placeholder into RuntimeFilterHub, HashJoinBuildOperator will generate runtime filters and fill it, - // Operators consuming the runtime filters will inspect this placeholder. - context->fragment_context()->runtime_filter_hub()->add_holder(_id); - // Create a shared RefCountedRuntimeFilterCollector - auto&& rc_rf_probe_collector = std::make_shared(2, std::move(this->runtime_filter_collector())); + auto rc_rf_probe_collector = std::make_shared(2, std::move(this->runtime_filter_collector())); // In default query engine, we only build one hash table for join right child. // But for pipeline query engine, we will build `num_right_partitions` hash tables, so we need to enlarge the limit @@ -508,12 +505,20 @@ pipeline::OpFactories HashJoinNode::_decompose_to_pipeline(pipeline::PipelineBui DeferOp pop_dependent_pipeline([context]() { context->pop_dependent_pipeline(); }); auto lhs_operators = child(0)->decompose_to_pipeline(context); - if (_distribution_mode == TJoinDistributionMode::BROADCAST) { - lhs_operators = context->maybe_interpolate_local_passthrough_exchange(runtime_state(), id(), lhs_operators, - context->degree_of_parallelism()); + auto join_colocate_group = context->find_exec_group_by_plan_node_id(_id); + if (join_colocate_group->type() == ExecutionGroupType::COLOCATE) { + DCHECK(context->current_execution_group()->is_colocate_exec_group()); + DCHECK_EQ(context->current_execution_group(), join_colocate_group); + context->set_current_execution_group(join_colocate_group); } else { - if (_join_type == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { - lhs_operators = context->maybe_interpolate_local_passthrough_exchange(runtime_state(), id(), lhs_operators); + // left child is colocate group, but current join is not colocate group + if (context->current_execution_group()->is_colocate_exec_group()) { + lhs_operators = context->interpolate_grouped_exchange(_id, lhs_operators); + } + + if (_distribution_mode == TJoinDistributionMode::BROADCAST) { + lhs_operators = context->maybe_interpolate_local_passthrough_exchange(runtime_state(), id(), lhs_operators, + context->degree_of_parallelism()); } else { auto* rhs_source_op = context->source_operator(rhs_operators); auto* lhs_source_op = context->source_operator(lhs_operators); @@ -522,13 +527,27 @@ pipeline::OpFactories HashJoinNode::_decompose_to_pipeline(pipeline::PipelineBui _probe_equivalence_partition_expr_ctxs); } } + lhs_operators.emplace_back(std::move(probe_op)); + // add placeholder into RuntimeFilterHub, HashJoinBuildOperator will generate runtime filters and fill it, + // Operators consuming the runtime filters will inspect this placeholder. + if (context->is_colocate_group() && _distribution_mode == TJoinDistributionMode::COLOCATE) { + for (auto runtime_filter_build_desc : _build_runtime_filters) { + // local colocate won't generate global runtime filter + DCHECK(!runtime_filter_build_desc->has_remote_targets()); + runtime_filter_build_desc->set_num_colocate_partition(num_right_partitions); + } + context->fragment_context()->runtime_filter_hub()->add_holder(_id, num_right_partitions); + } else { + context->fragment_context()->runtime_filter_hub()->add_holder(_id); + } if (limit() != -1) { lhs_operators.emplace_back(std::make_shared(context->next_operator_id(), id(), limit())); } - if (_hash_join_node.__isset.interpolate_passthrough && _hash_join_node.interpolate_passthrough) { + if (_hash_join_node.__isset.interpolate_passthrough && _hash_join_node.interpolate_passthrough && + !context->is_colocate_group()) { lhs_operators = context->maybe_interpolate_local_passthrough_exchange(runtime_state(), id(), lhs_operators, context->degree_of_parallelism(), true); } diff --git a/be/src/exec/hash_joiner.cpp b/be/src/exec/hash_joiner.cpp index 650189a75ee7e8..ea15ad70eb8888 100644 --- a/be/src/exec/hash_joiner.cpp +++ b/be/src/exec/hash_joiner.cpp @@ -571,6 +571,7 @@ Status HashJoiner::_create_runtime_bloom_filters(RuntimeState* state, int64_t li bool eq_null = _is_null_safes[expr_order]; MutableJoinRuntimeFilterPtr filter = nullptr; auto multi_partitioned = rf_desc->layout().pipeline_level_multi_partitioned(); + multi_partitioned |= rf_desc->num_colocate_partition() > 0; if (multi_partitioned) { LogicalType build_type = rf_desc->build_expr_type(); filter = std::shared_ptr( diff --git a/be/src/exec/olap_scan_node.cpp b/be/src/exec/olap_scan_node.cpp index 5c79c0d0fda3ad..d34e1fe0250f50 100644 --- a/be/src/exec/olap_scan_node.cpp +++ b/be/src/exec/olap_scan_node.cpp @@ -872,6 +872,9 @@ pipeline::OpFactories OlapScanNode::decompose_to_pipeline(pipeline::PipelineBuil scan_prepare_op->set_degree_of_parallelism(shared_morsel_queue ? 1 : dop); this->init_runtime_filter_for_operator(scan_prepare_op.get(), context, rc_rf_probe_collector); + auto exec_group = context->find_exec_group_by_plan_node_id(_id); + context->set_current_execution_group(exec_group); + auto scan_prepare_pipeline = pipeline::OpFactories{ std::move(scan_prepare_op), std::make_shared(context->next_operator_id(), id()), diff --git a/be/src/exec/olap_scan_prepare.cpp b/be/src/exec/olap_scan_prepare.cpp index 541ff0b69c9147..b06a1fb432d2e4 100644 --- a/be/src/exec/olap_scan_prepare.cpp +++ b/be/src/exec/olap_scan_prepare.cpp @@ -387,7 +387,7 @@ void OlapScanConjunctsManager::normalize_join_runtime_filter(const SlotDescripto // bloom runtime filter for (const auto& it : runtime_filters->descriptors()) { const RuntimeFilterProbeDescriptor* desc = it.second; - const JoinRuntimeFilter* rf = desc->runtime_filter(); + const JoinRuntimeFilter* rf = desc->runtime_filter(driver_sequence); using RangeType = ColumnValueRange; using ValueType = typename RunTimeTypeTraits::CppType; SlotId slot_id; @@ -397,7 +397,7 @@ void OlapScanConjunctsManager::normalize_join_runtime_filter(const SlotDescripto // runtime filter existed and does not have null. if (rf == nullptr) { - rt_ranger_params.add_unarrived_rf(desc, &slot); + rt_ranger_params.add_unarrived_rf(desc, &slot, driver_sequence); continue; } diff --git a/be/src/exec/olap_scan_prepare.h b/be/src/exec/olap_scan_prepare.h index 477846d986635f..086e5544efc9ba 100644 --- a/be/src/exec/olap_scan_prepare.h +++ b/be/src/exec/olap_scan_prepare.h @@ -36,6 +36,7 @@ class OlapScanConjunctsManager { const std::vector* key_column_names; const RuntimeFilterProbeCollector* runtime_filters; RuntimeState* runtime_state; + int32_t driver_sequence = -1; private: // fields generated by parsing conjunct ctxs. diff --git a/be/src/exec/pipeline/exchange/sink_buffer.cpp b/be/src/exec/pipeline/exchange/sink_buffer.cpp index 1796ab3c369db0..291008801a872d 100644 --- a/be/src/exec/pipeline/exchange/sink_buffer.cpp +++ b/be/src/exec/pipeline/exchange/sink_buffer.cpp @@ -34,7 +34,7 @@ SinkBuffer::SinkBuffer(FragmentContext* fragment_ctx, const std::vectorruntime_state()->get_rpc_http_min_size()), _sent_audit_stats_frequency_upper_limit( - std::max((int64_t)64, BitUtil::RoundUpToPowerOfTwo(fragment_ctx->num_drivers() * 4))) { + std::max((int64_t)64, BitUtil::RoundUpToPowerOfTwo(fragment_ctx->total_dop() * 4))) { for (const auto& dest : destinations) { const auto& instance_id = dest.fragment_instance_id; // instance_id.lo == -1 indicates that the destination is pseudo for bucket shuffle join. diff --git a/be/src/exec/pipeline/fragment_context.cpp b/be/src/exec/pipeline/fragment_context.cpp index 7f5c3e92d66a4f..8caf7c3dca1c95 100644 --- a/be/src/exec/pipeline/fragment_context.cpp +++ b/be/src/exec/pipeline/fragment_context.cpp @@ -15,6 +15,7 @@ #include "exec/pipeline/fragment_context.h" #include "exec/data_sink.h" +#include "exec/pipeline/group_execution/execution_group.h" #include "exec/pipeline/pipeline_driver_executor.h" #include "exec/pipeline/stream_pipeline_driver.h" #include "exec/workgroup/work_group.h" @@ -31,38 +32,25 @@ FragmentContext::FragmentContext() : _data_sink(nullptr) {} FragmentContext::~FragmentContext() { _data_sink.reset(); _runtime_filter_hub.close_all_in_filters(_runtime_state.get()); - clear_all_drivers(); - close_all_pipelines(); + close_all_execution_groups(); if (_plan != nullptr) { _plan->close(_runtime_state.get()); } } -void FragmentContext::clear_all_drivers() { - for (auto& pipe : _pipelines) { - pipe->clear_drivers(); - } -} -void FragmentContext::close_all_pipelines() { - for (auto& pipe : _pipelines) { - pipe->close(_runtime_state.get()); - } -} - size_t FragmentContext::total_dop() const { size_t total = 0; - for (const auto& pipeline : _pipelines) { - total += pipeline->degree_of_parallelism(); + for (const auto& group : _execution_groups) { + total += group->total_logical_dop(); } return total; } -size_t FragmentContext::num_drivers() const { - size_t total = 0; - for (const auto& pipeline : _pipelines) { - total += pipeline->drivers().size(); +void FragmentContext::close_all_execution_groups() { + for (auto& group : _execution_groups) { + group->close(_runtime_state.get()); } - return total; + _execution_groups.clear(); } void FragmentContext::move_tplan(TPlan& tplan) { @@ -72,18 +60,18 @@ void FragmentContext::set_data_sink(std::unique_ptr data_sink) { _data_sink = std::move(data_sink); } -void FragmentContext::count_down_pipeline(size_t val) { +void FragmentContext::count_down_execution_group(size_t val) { // Note that _pipelines may be destructed after fetch_add // memory_order_seq_cst semantics ensure that previous code does not reorder after fetch_add - size_t total_pipelines = _pipelines.size(); - bool all_pipelines_finished = _num_finished_pipelines.fetch_add(val) + val == total_pipelines; - if (!all_pipelines_finished) { + size_t total_execution_groups = _execution_groups.size(); + bool all_groups_finished = _num_finished_execution_groups.fetch_add(val) + val == total_execution_groups; + if (!all_groups_finished) { return; } + // dump profile if necessary auto* state = runtime_state(); auto* query_ctx = state->query_ctx(); - state->runtime_profile()->reverse_childs(); if (config::pipeline_print_profile) { std::stringstream ss; @@ -136,12 +124,11 @@ void FragmentContext::report_exec_state_if_necessary() { normalized_report_ns = last_report_ns + interval_ns; } if (_last_report_exec_state_ns.compare_exchange_strong(last_report_ns, normalized_report_ns)) { - for (auto& pipeline : _pipelines) { - for (auto& driver : pipeline->drivers()) { + iterate_pipeline([](const Pipeline* pipeline) { + for (const auto& driver : pipeline->drivers()) { driver->runtime_report_action(); } - } - + }); state->exec_env()->wg_driver_executor()->report_exec_state(query_ctx, this, Status::OK(), false, true); } } @@ -169,6 +156,21 @@ void FragmentContext::set_final_status(const Status& status) { } } +void FragmentContext::set_exec_groups(ExecutionGroups&& exec_groups) { + for (auto& group : exec_groups) { + if (!group->is_empty()) { + _execution_groups.emplace_back(std::move(group)); + } + } +} + +Status FragmentContext::prepare_all_pipelines() { + for (auto& group : _execution_groups) { + RETURN_IF_ERROR(group->prepare_pipelines(_runtime_state.get())); + } + return Status::OK(); +} + void FragmentContext::set_stream_load_contexts(const std::vector& contexts) { _stream_load_contexts = std::move(contexts); _channel_stream_load = true; @@ -301,20 +303,50 @@ void FragmentContext::destroy_pass_through_chunk_buffer() { Status FragmentContext::reset_epoch() { _num_finished_epoch_pipelines = 0; - for (const auto& pipeline : _pipelines) { + const std::function caller = [this](Pipeline* pipeline) { RETURN_IF_ERROR(_runtime_state->reset_epoch()); RETURN_IF_ERROR(pipeline->reset_epoch(_runtime_state.get())); - } - return Status::OK(); + return Status::OK(); + }; + return iterate_pipeline(caller); } void FragmentContext::count_down_epoch_pipeline(RuntimeState* state, size_t val) { - bool all_pipelines_finished = _num_finished_epoch_pipelines.fetch_add(val) + val == _pipelines.size(); - if (!all_pipelines_finished) { + size_t total_execution_groups = _execution_groups.size(); + bool all_groups_finished = _num_finished_epoch_pipelines.fetch_add(val) + val == total_execution_groups; + if (!all_groups_finished) { return; } state->query_ctx()->stream_epoch_manager()->count_down_fragment_ctx(state, this); } +void FragmentContext::iterate_pipeline(const std::function& call) { + for (auto& group : _execution_groups) { + group->for_each_pipeline(call); + } +} + +Status FragmentContext::iterate_pipeline(const std::function& call) { + for (auto& group : _execution_groups) { + RETURN_IF_ERROR(group->for_each_pipeline(call)); + } + return Status::OK(); +} + +Status FragmentContext::prepare_active_drivers() { + for (auto& group : _execution_groups) { + RETURN_IF_ERROR(group->prepare_drivers(_runtime_state.get())); + } + return Status::OK(); +} + +Status FragmentContext::submit_active_drivers(DriverExecutor* executor) { + for (auto& group : _execution_groups) { + group->attach_driver_executor(executor); + group->submit_active_drivers(); + } + return Status::OK(); +} + } // namespace starrocks::pipeline diff --git a/be/src/exec/pipeline/fragment_context.h b/be/src/exec/pipeline/fragment_context.h index e88c18fab911bd..02ae1145905092 100644 --- a/be/src/exec/pipeline/fragment_context.h +++ b/be/src/exec/pipeline/fragment_context.h @@ -19,6 +19,7 @@ #include "exec/exec_node.h" #include "exec/pipeline/adaptive/adaptive_dop_param.h" #include "exec/pipeline/driver_limiter.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" #include "exec/pipeline/pipeline.h" #include "exec/pipeline/pipeline_driver.h" #include "exec/pipeline/pipeline_fwd.h" @@ -45,13 +46,6 @@ namespace pipeline { using RuntimeFilterPort = starrocks::RuntimeFilterPort; using PerDriverScanRangesMap = std::map>; -// clang-format off -template -concept DriverPtrCallable = std::invocable && - (std::same_as, void> || - std::same_as, Status>); -// clang-format on - class FragmentContext { friend FragmentContextManager; @@ -77,19 +71,16 @@ class FragmentContext { void set_data_sink(std::unique_ptr data_sink); size_t total_dop() const; - Pipelines& pipelines() { return _pipelines; } - void set_pipelines(Pipelines&& pipelines) { _pipelines = std::move(pipelines); } - size_t num_drivers() const; - bool all_pipelines_finished() const { return _num_finished_pipelines == _pipelines.size(); } - void count_down_pipeline(size_t val = 1); + bool all_execution_groups_finished() const { return _num_finished_execution_groups == _execution_groups.size(); } + void count_down_execution_group(size_t val = 1); bool need_report_exec_state(); void report_exec_state_if_necessary(); void set_final_status(const Status& status); - [[nodiscard]] Status final_status() const { + Status final_status() const { auto* status = _final_status.load(); return status == nullptr ? Status::OK() : *status; } @@ -102,33 +93,21 @@ class FragmentContext { MorselQueueFactoryMap& morsel_queue_factories() { return _morsel_queue_factories; } - [[nodiscard]] Status prepare_all_pipelines() { - for (auto& pipe : _pipelines) { - RETURN_IF_ERROR(pipe->prepare(_runtime_state.get())); - } - return Status::OK(); - } + void set_exec_groups(ExecutionGroups&& exec_groups); + + Status prepare_all_pipelines(); - template - [[nodiscard]] auto iterate_drivers(Func call) { - using ReturnType = std::invoke_result_t; - - for (const auto& pipeline : _pipelines) { - for (const auto& driver : pipeline->drivers()) { - if constexpr (std::is_same_v) { - RETURN_IF_ERROR(call(driver)); - } else { - call(driver); - } + template + void iterate_drivers(Func&& call) { + iterate_pipeline([&](const Pipeline* pipeline) { + for (auto& driver : pipeline->drivers()) { + call(driver); } - } - if constexpr (std::is_same_v) { - return Status::OK(); - } + }); } void clear_all_drivers(); - void close_all_pipelines(); + void close_all_execution_groups(); RuntimeFilterHub* runtime_filter_hub() { return &_runtime_filter_hub; } @@ -173,7 +152,17 @@ class FragmentContext { void set_expired_log_count(size_t val) { _expired_log_count = val; } + void iterate_pipeline(const std::function& call); + Status iterate_pipeline(const std::function& call); + + Status prepare_active_drivers(); + Status submit_active_drivers(DriverExecutor* executor); + + bool enable_group_execution() const { return _enable_group_execution; } + void set_enable_group_execution(bool enable_group_execution) { _enable_group_execution = enable_group_execution; } + private: + bool _enable_group_execution = false; // Id of this query TUniqueId _query_id; // Id of this instance @@ -193,8 +182,8 @@ class FragmentContext { std::shared_ptr _runtime_state = nullptr; ExecNode* _plan = nullptr; // lives in _runtime_state->obj_pool() size_t _next_driver_id = 0; - Pipelines _pipelines; - std::atomic _num_finished_pipelines = 0; + ExecutionGroups _execution_groups; + std::atomic _num_finished_execution_groups = 0; RuntimeFilterHub _runtime_filter_hub; diff --git a/be/src/exec/pipeline/fragment_executor.cpp b/be/src/exec/pipeline/fragment_executor.cpp index 415ebe24e21467..8dd3c18edf56dc 100644 --- a/be/src/exec/pipeline/fragment_executor.cpp +++ b/be/src/exec/pipeline/fragment_executor.cpp @@ -19,52 +19,29 @@ #include "common/config.h" #include "exec/cross_join_node.h" #include "exec/exchange_node.h" +#include "exec/exec_node.h" #include "exec/olap_scan_node.h" #include "exec/pipeline/adaptive/event.h" -#include "exec/pipeline/chunk_accumulate_operator.h" -#include "exec/pipeline/exchange/exchange_sink_operator.h" -#include "exec/pipeline/exchange/multi_cast_local_exchange.h" -#include "exec/pipeline/exchange/sink_buffer.h" #include "exec/pipeline/fragment_context.h" -#include "exec/pipeline/noop_sink_operator.h" -#include "exec/pipeline/olap_table_sink_operator.h" #include "exec/pipeline/pipeline_builder.h" #include "exec/pipeline/pipeline_driver_executor.h" #include "exec/pipeline/result_sink_operator.h" #include "exec/pipeline/scan/connector_scan_operator.h" #include "exec/pipeline/scan/morsel.h" #include "exec/pipeline/scan/scan_operator.h" -#include "exec/pipeline/sink/blackhole_table_sink_operator.h" -#include "exec/pipeline/sink/dictionary_cache_sink_operator.h" -#include "exec/pipeline/sink/export_sink_operator.h" -#include "exec/pipeline/sink/file_sink_operator.h" -#include "exec/pipeline/sink/hive_table_sink_operator.h" -#include "exec/pipeline/sink/iceberg_table_sink_operator.h" -#include "exec/pipeline/sink/memory_scratch_sink_operator.h" -#include "exec/pipeline/sink/mysql_table_sink_operator.h" -#include "exec/pipeline/sink/table_function_table_sink_operator.h" #include "exec/pipeline/stream_pipeline_driver.h" #include "exec/scan_node.h" #include "exec/tablet_sink.h" #include "exec/workgroup/work_group.h" #include "gutil/casts.h" #include "gutil/map_util.h" -#include "runtime/blackhole_table_sink.h" #include "runtime/data_stream_mgr.h" #include "runtime/data_stream_sender.h" #include "runtime/descriptors.h" -#include "runtime/dictionary_cache_sink.h" #include "runtime/exec_env.h" -#include "runtime/export_sink.h" -#include "runtime/hive_table_sink.h" -#include "runtime/iceberg_table_sink.h" -#include "runtime/memory_scratch_sink.h" -#include "runtime/multi_cast_data_stream_sink.h" -#include "runtime/mysql_table_sink.h" #include "runtime/result_sink.h" #include "runtime/stream_load/stream_load_context.h" #include "runtime/stream_load/transaction_mgr.h" -#include "runtime/table_function_table_sink.h" #include "util/debug/query_trace.h" #include "util/runtime_profile.h" #include "util/time.h" @@ -552,20 +529,28 @@ Status FragmentExecutor::_prepare_pipeline_driver(ExecEnv* exec_env, const Unifi const auto degree_of_parallelism = _calc_dop(exec_env, request); const auto& fragment = request.common().fragment; const auto& params = request.common().params; + + // check group execution params + std::unordered_map colocate_exec_groups; + if (request.common().fragment.__isset.group_execution_param && + request.common().fragment.group_execution_param.enable_group_execution) { + _fragment_ctx->set_enable_group_execution(true); + colocate_exec_groups = ExecutionGroupBuilder::create_colocate_exec_groups( + request.common().fragment.group_execution_param, degree_of_parallelism); + } + auto is_stream_pipeline = request.is_stream_pipeline(); ExecNode* plan = _fragment_ctx->plan(); Drivers drivers; MorselQueueFactoryMap& morsel_queue_factories = _fragment_ctx->morsel_queue_factories(); auto* runtime_state = _fragment_ctx->runtime_state(); - const auto& pipelines = _fragment_ctx->pipelines(); size_t sink_dop = _calc_sink_dop(ExecEnv::GetInstance(), request); - // Build pipelines PipelineBuilderContext context(_fragment_ctx.get(), degree_of_parallelism, sink_dop, is_stream_pipeline); + context.init_colocate_groups(std::move(colocate_exec_groups)); PipelineBuilder builder(context); auto exec_ops = builder.decompose_exec_node_to_pipeline(*_fragment_ctx, plan); - // Set up sink if required std::unique_ptr datasink; if (request.isset_output_sink()) { @@ -582,33 +567,31 @@ Status FragmentExecutor::_prepare_pipeline_driver(ExecEnv* exec_env, const Unifi tsink, fragment.output_exprs)); } _fragment_ctx->set_data_sink(std::move(datasink)); - - _fragment_ctx->set_pipelines(builder.build()); + _fragment_ctx->set_exec_groups(builder.build()); RETURN_IF_ERROR(_fragment_ctx->prepare_all_pipelines()); // Set morsel_queue_factory to pipeline. - for (const auto& pipeline : pipelines) { + _fragment_ctx->iterate_pipeline([&morsel_queue_factories](Pipeline* pipeline) { if (pipeline->source_operator_factory()->with_morsels()) { - auto source_id = pipeline->get_op_factories()[0]->plan_node_id(); + auto source_id = pipeline->source_operator_factory()->plan_node_id(); DCHECK(morsel_queue_factories.count(source_id)); auto& morsel_queue_factory = morsel_queue_factories[source_id]; - pipeline->source_operator_factory()->set_morsel_queue_factory(morsel_queue_factory.get()); } - } + }); + // collect unready pipeline groups and instantiate ready drivers PipelineGroupMap unready_pipeline_groups; - for (const auto& pipeline : pipelines) { + _fragment_ctx->iterate_pipeline([&unready_pipeline_groups, runtime_state](Pipeline* pipeline) { auto* source_op = pipeline->source_operator_factory(); if (!source_op->is_adaptive_group_initial_active()) { auto* group_leader_source_op = source_op->group_leader(); - unready_pipeline_groups[group_leader_source_op].emplace_back(pipeline.get()); - continue; + unready_pipeline_groups[group_leader_source_op].emplace_back(pipeline); + return; } - pipeline->instantiate_drivers(runtime_state); - } + }); if (!unready_pipeline_groups.empty()) { create_adaptive_group_initialize_events(runtime_state, std::move(unready_pipeline_groups)); @@ -753,40 +736,16 @@ Status FragmentExecutor::execute(ExecEnv* exec_env) { auto* prepare_driver_timer = ADD_CHILD_TIMER_THESHOLD(profile, "prepare-pipeline-driver", "FragmentInstancePrepareTime", 10_ms); - auto iterate_active_drivers = [fragment_ctx = _fragment_ctx.get()](auto call) { - using Func = decltype(call); - static_assert(DriverPtrCallable, "Function must be callable with DriverPtr and return void or Status."); - using ReturnType = std::invoke_result_t; - - for (const auto& pipeline : fragment_ctx->pipelines()) { - auto* source_op = pipeline->source_operator_factory(); - if (!source_op->is_adaptive_group_initial_active()) { - continue; - } - for (const auto& driver : pipeline->drivers()) { - if constexpr (std::is_same_v) { - RETURN_IF_ERROR(call(driver)); - } else { - call(driver); - } - } - } - if constexpr (std::is_same_v) { - return Status::OK(); - } - }; - { SCOPED_TIMER(prepare_instance_timer); SCOPED_TIMER(prepare_driver_timer); - RETURN_IF_ERROR(iterate_active_drivers( - [state = _fragment_ctx->runtime_state()](const DriverPtr& driver) { return driver->prepare(state); })); + RETURN_IF_ERROR(_fragment_ctx->prepare_active_drivers()); } prepare_success = true; DCHECK(_fragment_ctx->enable_resource_group()); auto* executor = exec_env->wg_driver_executor(); - iterate_active_drivers([executor](const DriverPtr& driver) { executor->submit(driver.get()); }); + RETURN_IF_ERROR(_fragment_ctx->submit_active_drivers(executor)); return Status::OK(); } diff --git a/be/src/exec/pipeline/group_execution/execution_group.cpp b/be/src/exec/pipeline/group_execution/execution_group.cpp new file mode 100644 index 00000000000000..22f59f78a863cf --- /dev/null +++ b/be/src/exec/pipeline/group_execution/execution_group.cpp @@ -0,0 +1,154 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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 "exec/pipeline/group_execution/execution_group.h" + +#include "common/logging.h" +#include "exec/pipeline/pipeline_driver_executor.h" +#include "exec/pipeline/pipeline_fwd.h" + +namespace starrocks::pipeline { +// clang-format off +template +concept DriverPtrCallable = std::invocable && + (std::same_as, void> || + std::same_as, Status>); +// clang-format on + +void ExecutionGroup::clear_all_drivers(Pipelines& pipelines) { + for (auto& pipeline : pipelines) { + pipeline->clear_drivers(); + } +} + +Status NormalExecutionGroup::prepare_pipelines(RuntimeState* state) { + for (auto& pipeline : _pipelines) { + RETURN_IF_ERROR(pipeline->prepare(state)); + _total_logical_dop += pipeline->degree_of_parallelism(); + } + return Status::OK(); +} + +template +auto for_each_active_driver(Pipelines& pipelines, Callable call) { + using ReturnType = std::invoke_result_t; + for (auto& pipeline : pipelines) { + for (auto& driver : pipeline->drivers()) { + auto* source_op = pipeline->source_operator_factory(); + if (!source_op->is_adaptive_group_initial_active()) { + continue; + } + if constexpr (std::same_as) { + RETURN_IF_ERROR(call(driver)); + } else { + call(driver); + } + } + } + if constexpr (std::is_same_v) { + return Status::OK(); + } +} + +Status NormalExecutionGroup::prepare_drivers(RuntimeState* state) { + return for_each_active_driver(_pipelines, [state](const DriverPtr& driver) { return driver->prepare(state); }); +} + +void NormalExecutionGroup::submit_active_drivers() { + VLOG_QUERY << "submit_active_drivers:" << to_string(); + return for_each_active_driver(_pipelines, [this](const DriverPtr& driver) { _executor->submit(driver.get()); }); +} + +void NormalExecutionGroup::add_pipeline(PipelinePtr pipeline) { + _pipelines.emplace_back(std::move(pipeline)); + _num_pipelines = _pipelines.size(); +} + +void NormalExecutionGroup::close(RuntimeState* state) { + for (auto& pipeline : _pipelines) { + pipeline->close(state); + } +} + +std::string NormalExecutionGroup::to_string() const { + std::stringstream ss; + ss << "NormalExecutionGroup: "; + for (const auto& pipeline : _pipelines) { + ss << pipeline->to_readable_string() << ","; + } + return ss.str(); +} + +Status ColocateExecutionGroup::prepare_pipelines(RuntimeState* state) { + for (auto& pipeline : _pipelines) { + RETURN_IF_ERROR(pipeline->prepare(state)); + _total_logical_dop = pipeline->degree_of_parallelism(); + } + _submit_drivers = std::make_unique[]>(_pipelines.size()); + return Status::OK(); +} + +Status ColocateExecutionGroup::prepare_drivers(RuntimeState* state) { + return for_each_active_driver(_pipelines, [state](const DriverPtr& driver) { return driver->prepare(state); }); +} + +void ColocateExecutionGroup::submit_active_drivers() { + VLOG_QUERY << "submit_active_drivers:" << to_string(); + for (size_t i = 0; i < _pipelines.size(); ++i) { + const auto& pipeline = _pipelines[i]; + DCHECK_EQ(pipeline->drivers().size(), pipeline->degree_of_parallelism()); + const auto& drivers = pipeline->drivers(); + size_t init_submit_drivers = std::min(_physical_dop, drivers.size()); + _submit_drivers[i] = init_submit_drivers; + for (size_t i = 0; i < init_submit_drivers; ++i) { + VLOG_QUERY << "submit_active_driver:" << i << ":" << drivers[i]->to_readable_string(); + _executor->submit(drivers[i].get()); + } + } +} + +void ColocateExecutionGroup::add_pipeline(PipelinePtr pipeline) { + _pipelines.emplace_back(std::move(pipeline)); + _num_pipelines = _pipelines.size(); +} + +void ColocateExecutionGroup::close(RuntimeState* state) { + for (auto& pipeline : _pipelines) { + pipeline->close(state); + } +} + +std::string ColocateExecutionGroup::to_string() const { + std::stringstream ss; + ss << "ColocateExecutionGroup: "; + for (const auto& pipeline : _pipelines) { + ss << pipeline->to_readable_string() << ","; + } + return ss.str(); +} + +void ColocateExecutionGroup::submit_next_driver() { + for (size_t i = 0; i < _pipelines.size(); ++i) { + auto next_driver_idx = _submit_drivers[i].fetch_add(1); + if (next_driver_idx >= _pipelines[i]->degree_of_parallelism()) { + continue; + } + const auto& drivers = _pipelines[i]->drivers(); + VLOG_QUERY << "submit_next_drivers:" << next_driver_idx << ":" + << drivers[next_driver_idx]->to_readable_string(); + _executor->submit(drivers[next_driver_idx].get()); + } +} + +} // namespace starrocks::pipeline \ No newline at end of file diff --git a/be/src/exec/pipeline/group_execution/execution_group.h b/be/src/exec/pipeline/group_execution/execution_group.h new file mode 100644 index 00000000000000..411a1e8f4f3665 --- /dev/null +++ b/be/src/exec/pipeline/group_execution/execution_group.h @@ -0,0 +1,145 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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 "exec/pipeline/fragment_context.h" +#include "exec/pipeline/group_execution/execution_group_builder.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" +#include "exec/pipeline/pipeline.h" +#include "exec/pipeline/pipeline_driver_executor.h" +#include "exec/pipeline/pipeline_fwd.h" +#include "runtime/runtime_state.h" + +namespace starrocks::pipeline { +// execution group is a collection of pipelines +// clang-format off +template +concept PipelineCallable = std::invocable && + (std::same_as, void> || + std::same_as, Status>); +// clang-format on +class ExecutionGroup { +public: + ExecutionGroup(ExecutionGroupType type) : _type(type) {} + virtual ~ExecutionGroup() = default; + virtual Status prepare_pipelines(RuntimeState* state) = 0; + virtual Status prepare_drivers(RuntimeState* state) = 0; + virtual void submit_active_drivers() = 0; + template + auto for_each_pipeline(Callable&& call) { + using ReturnType = std::invoke_result_t; + if constexpr (std::same_as) { + for (auto& pipeline : _pipelines) { + call(pipeline.get()); + } + } else { + for (auto& pipeline : _pipelines) { + RETURN_IF_ERROR(call(pipeline.get())); + } + return Status::OK(); + } + } + + virtual void add_pipeline(PipelinePtr pipeline) = 0; + virtual void close(RuntimeState* state) = 0; + virtual void submit_next_driver() = 0; + virtual bool is_empty() const = 0; + virtual std::string to_string() const = 0; + void attach_driver_executor(DriverExecutor* executor) { _executor = executor; } + + void count_down_pipeline(RuntimeState* state) { + if (++_num_finished_pipelines == _num_pipelines) { + state->fragment_ctx()->count_down_execution_group(); + } + } + + void count_down_epoch_pipeline(RuntimeState* state) { + if (++_num_epoch_finished_pipelines == _num_pipelines) { + state->fragment_ctx()->count_down_epoch_pipeline(state); + } + } + + size_t total_logical_dop() const { return _total_logical_dop; } + + ExecutionGroupType type() const { return _type; } + bool is_colocate_exec_group() const { return type() == ExecutionGroupType::COLOCATE; } + + bool contains(int32_t plan_node_id) { return _plan_node_ids.contains(plan_node_id); } + +protected: + // only used in colocate groups + std::unordered_set _plan_node_ids; + ExecutionGroupType _type; + // total logical degree of parallelism + // will be inited in prepare_pipelines + size_t _total_logical_dop{}; + std::atomic _num_finished_pipelines{}; + std::atomic _num_epoch_finished_pipelines{}; + size_t _num_pipelines{}; + DriverExecutor* _executor; + Pipelines _pipelines; + + void clear_all_drivers(Pipelines& pipelines); +}; + +class NormalExecutionGroup final : public ExecutionGroup { +public: + NormalExecutionGroup() : ExecutionGroup(ExecutionGroupType::NORMAL) {} + ~NormalExecutionGroup() override = default; + + Status prepare_pipelines(RuntimeState* state) override; + Status prepare_drivers(RuntimeState* state) override; + void submit_active_drivers() override; + void add_pipeline(PipelinePtr pipeline) override; + + void close(RuntimeState* state) override; + // nothing to do + void submit_next_driver() override {} + bool is_empty() const override { return _pipelines.empty(); } + std::string to_string() const override; +}; + +// execution group for colocate pipelines +// all pipelines in this group should have the same dop +// There should be no dependencies between the operators of multiple dops +class ColocateExecutionGroup final : public ExecutionGroup { +public: + ColocateExecutionGroup(size_t physical_dop) + : ExecutionGroup(ExecutionGroupType::COLOCATE), _physical_dop(physical_dop) {} + ~ColocateExecutionGroup() override = default; + + Status prepare_pipelines(RuntimeState* state) override; + Status prepare_drivers(RuntimeState* state) override; + void submit_active_drivers() override; + void add_pipeline(PipelinePtr pipeline) override; + + void close(RuntimeState* state) override; + void submit_next_driver() override; + bool is_empty() const override { return _pipelines.empty(); } + std::string to_string() const override; + void add_plan_node_id(int32_t plan_node_id) { _plan_node_ids.insert(plan_node_id); } + +private: + size_t _physical_dop; + // TODO: add Pad to fix false sharing problems + std::unique_ptr[]> _submit_drivers; +}; + +} // namespace starrocks::pipeline \ No newline at end of file diff --git a/be/src/exec/pipeline/group_execution/execution_group_builder.cpp b/be/src/exec/pipeline/group_execution/execution_group_builder.cpp new file mode 100644 index 00000000000000..91232b1b025ab8 --- /dev/null +++ b/be/src/exec/pipeline/group_execution/execution_group_builder.cpp @@ -0,0 +1,38 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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 "exec/pipeline/group_execution/execution_group_builder.h" + +#include "common/logging.h" +#include "exec/pipeline/group_execution/execution_group.h" + +namespace starrocks::pipeline { +ExecutionGroupPtr ExecutionGroupBuilder::create_normal_exec_group() { + return std::make_shared(); +} + +std::unordered_map ExecutionGroupBuilder::create_colocate_exec_groups( + const TGroupExecutionParam& group_execution_param, size_t physical_dop) { + std::unordered_map colocate_exec_groups; + for (const auto& colocate_group : group_execution_param.exec_groups) { + auto exec_group = std::make_shared(physical_dop); + for (const auto& plan_node_id : colocate_group.plan_node_ids) { + exec_group->add_plan_node_id(plan_node_id); + } + colocate_exec_groups.emplace(colocate_group.group_id, exec_group); + } + return colocate_exec_groups; +} + +} // namespace starrocks::pipeline \ No newline at end of file diff --git a/be/src/exec/pipeline/group_execution/execution_group_builder.h b/be/src/exec/pipeline/group_execution/execution_group_builder.h new file mode 100644 index 00000000000000..07aff198dc0cf4 --- /dev/null +++ b/be/src/exec/pipeline/group_execution/execution_group_builder.h @@ -0,0 +1,41 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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 "exec/pipeline/group_execution/execution_group_fwd.h" + +namespace starrocks { +class TGroupExecutionParam; + +namespace pipeline { +enum class ExecutionGroupType { + NORMAL, + COLOCATE, +}; + +class ExecutionGroupBuilder { +public: + ExecutionGroupBuilder() = default; + ~ExecutionGroupBuilder() = default; + + static ExecutionGroupPtr create_normal_exec_group(); + static std::unordered_map create_colocate_exec_groups(const TGroupExecutionParam& param, + size_t physical_dop); +}; +} // namespace pipeline +} // namespace starrocks diff --git a/be/src/exec/pipeline/group_execution/execution_group_fwd.h b/be/src/exec/pipeline/group_execution/execution_group_fwd.h new file mode 100644 index 00000000000000..058597ebe11667 --- /dev/null +++ b/be/src/exec/pipeline/group_execution/execution_group_fwd.h @@ -0,0 +1,24 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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 starrocks::pipeline { +class ExecutionGroup; +using ExecutionGroupRawPtr = ExecutionGroup*; +using ExecutionGroupPtr = std::shared_ptr; +using ExecutionGroups = std::vector; +} // namespace starrocks::pipeline \ No newline at end of file diff --git a/be/src/exec/pipeline/group_execution/group_operator.cpp b/be/src/exec/pipeline/group_execution/group_operator.cpp new file mode 100644 index 00000000000000..5a00e3507b5fdf --- /dev/null +++ b/be/src/exec/pipeline/group_execution/group_operator.cpp @@ -0,0 +1,72 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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 "exec/pipeline/group_execution/group_operator.h" + +#include "exec/pipeline/group_execution/execution_group.h" +#include "exec/pipeline/operator.h" +#include "gutil/casts.h" + +namespace starrocks::pipeline { +Status GroupedExecutionSinkOperator::prepare(RuntimeState* state) { + RETURN_IF_ERROR(Operator::prepare(state)); + _exchanger->incr_sinker(); + _unique_metrics->add_info_string("ShuffleNum", std::to_string(_exchanger->source_dop())); + _peak_memory_usage_counter = _unique_metrics->AddHighWaterMarkCounter( + "GroupLocalExchangePeakMemoryUsage", TUnit::BYTES, + RuntimeProfile::Counter::create_strategy(TUnit::BYTES, TCounterMergeType::SKIP_FIRST_MERGE)); + return Status::OK(); +} + +bool GroupedExecutionSinkOperator::need_input() const { + return !_is_finished && _exchanger->need_input(); +} + +void GroupedExecutionSinkOperator::close(RuntimeState* state) { + Operator::close(state); +} + +Status GroupedExecutionSinkOperator::set_finishing(RuntimeState* state) { + _is_finished = true; + _exchanger->finish(state); + down_cast(_factory)->submit(); + return Status::OK(); +} + +Status GroupedExecutionSinkOperator::push_chunk(RuntimeState* state, const ChunkPtr& chunk) { + auto res = _exchanger->accept(chunk, _driver_sequence); + _peak_memory_usage_counter->set(_exchanger->get_memory_usage()); + return Status::OK(); +} + +Status GroupedExecutionSinkFactory::prepare(RuntimeState* state) { + RETURN_IF_ERROR(OperatorFactory::prepare(state)); + RETURN_IF_ERROR(_exchanger->prepare(state)); + return Status::OK(); +} + +OperatorPtr GroupedExecutionSinkFactory::create(int32_t degree_of_parallelism, int32_t driver_sequence) { + return std::make_shared(this, _id, _plan_node_id, driver_sequence, _exchanger); +} + +void GroupedExecutionSinkFactory::close(RuntimeState* state) { + _exchanger->close(state); + OperatorFactory::close(state); +} + +void GroupedExecutionSinkFactory::submit() { + _exec_group->submit_next_driver(); +} + +} // namespace starrocks::pipeline \ No newline at end of file diff --git a/be/src/exec/pipeline/group_execution/group_operator.h b/be/src/exec/pipeline/group_execution/group_operator.h new file mode 100644 index 00000000000000..8c62b0c9c4b841 --- /dev/null +++ b/be/src/exec/pipeline/group_execution/group_operator.h @@ -0,0 +1,78 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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 "common/statusor.h" +#include "exec/pipeline/exchange/local_exchange.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" +#include "exec/pipeline/operator.h" +#include "exec/pipeline/pipeline_fwd.h" +#include "exec/pipeline/source_operator.h" +#include "runtime/runtime_state.h" + +namespace starrocks::pipeline { +// P1: SourceOp->ProjectOp->JoinProbeOp->ProjectOp->ExchangeSink +// => +// P1: SourceOp->ProjectOp->JoinProbeOp->ProjectOp->GroupedSinkOp +// P2: GroupedSourceOp->ExchangeSink +class GroupedExecutionSinkOperator : public Operator { +public: + GroupedExecutionSinkOperator(OperatorFactory* factory, int32_t id, int32_t plan_node_id, int32_t driver_sequence, + const std::shared_ptr& exchanger) + : Operator(factory, id, "group_exchange_sink", plan_node_id, true, driver_sequence), _exchanger(exchanger) { + _unique_metrics->add_info_string("Type", exchanger->name()); + } + ~GroupedExecutionSinkOperator() override = default; + + bool has_output() const override { return false; } + bool need_input() const override; + bool is_finished() const override { return _is_finished; } + Status set_finishing(RuntimeState* state) override; + + Status prepare(RuntimeState* state) override; + void close(RuntimeState* state) override; + + Status push_chunk(RuntimeState* state, const ChunkPtr& chunk) override; + StatusOr pull_chunk(RuntimeState* state) override { return Status::InternalError("Not implemented"); } + +private: + std::atomic_bool _is_finished = false; + RuntimeProfile::HighWaterMarkCounter* _peak_memory_usage_counter = nullptr; + const std::shared_ptr& _exchanger; +}; + +class GroupedExecutionSinkFactory final : public OperatorFactory { +public: + GroupedExecutionSinkFactory(int32_t id, int32_t plan_node_id, std::shared_ptr exchanger, + ExecutionGroupRawPtr exec_group) + : OperatorFactory(id, "grouped_execution_sink", plan_node_id), + _exec_group(exec_group), + _exchanger(std::move(exchanger)) {} + + ~GroupedExecutionSinkFactory() override = default; + + Status prepare(RuntimeState* state) override; + OperatorPtr create(int32_t degree_of_parallelism, int32_t driver_sequence) override; + void close(RuntimeState* state) override; + void submit(); + +private: + ExecutionGroupRawPtr _exec_group; + std::shared_ptr _exchanger; +}; + +} // namespace starrocks::pipeline \ No newline at end of file diff --git a/be/src/exec/pipeline/hashjoin/hash_join_build_operator.cpp b/be/src/exec/pipeline/hashjoin/hash_join_build_operator.cpp index 3dfe5210e5385f..918ecfc987ae75 100644 --- a/be/src/exec/pipeline/hashjoin/hash_join_build_operator.cpp +++ b/be/src/exec/pipeline/hashjoin/hash_join_build_operator.cpp @@ -85,8 +85,10 @@ Status HashJoinBuildOperator::set_finishing(RuntimeState* state) { size_t merger_index = _driver_sequence; // Broadcast Join only has one build operator. DCHECK(_distribution_mode != TJoinDistributionMode::BROADCAST || _driver_sequence == 0); - - RETURN_IF_ERROR(_join_builder->create_runtime_filters(state)); + { + SCOPED_TIMER(_join_builder->build_metrics().build_runtime_filter_timer); + RETURN_IF_ERROR(_join_builder->create_runtime_filters(state)); + } auto ht_row_count = _join_builder->get_ht_row_count(); auto& partial_in_filters = _join_builder->get_runtime_in_filters(); @@ -101,26 +103,49 @@ Status HashJoinBuildOperator::set_finishing(RuntimeState* state) { ((HashJoinBuildOperatorFactory*)_factory) ->retain_string_key_columns(_driver_sequence, _join_builder->string_key_columns()); - // add partial filters generated by this HashJoinBuildOperator to PartialRuntimeFilterMerger to merge into a - // total one. - StatusOr status; - { - SCOPED_TIMER(_join_builder->build_metrics().build_runtime_filter_timer); - status = _partial_rf_merger->add_partial_filters(merger_index, ht_row_count, std::move(partial_in_filters), - std::move(partial_bloom_filter_build_params), - std::move(partial_bloom_filters)); - } - if (!status.ok()) { - return status.status(); - } else if (status.value()) { - auto&& in_filters = _partial_rf_merger->get_total_in_filters(); - auto&& bloom_filters = _partial_rf_merger->get_total_bloom_filters(); - - // publish runtime bloom-filters - state->runtime_filter_port()->publish_runtime_filters(bloom_filters); - // move runtime filters into RuntimeFilterHub. - runtime_filter_hub()->set_collector(_plan_node_id, std::make_unique( - std::move(in_filters), std::move(bloom_filters))); + // push colocate partial runtime filter + bool is_colocate_runtime_filter = runtime_filter_hub()->is_colocate_runtime_filters(_plan_node_id); + if (is_colocate_runtime_filter) { + // init local colocate in/bloom filters + RuntimeInFilterList in_filter_lists(partial_in_filters.begin(), partial_in_filters.end()); + if (partial_bloom_filters.size() != partial_bloom_filter_build_params.size()) { + // if in short-circuit mode, phase is EOS. partial_bloom_filter_build_params is empty. + DCHECK(_join_builder->is_done()); + } else { + for (size_t i = 0; i < partial_bloom_filters.size(); ++i) { + if (partial_bloom_filter_build_params[i].has_value()) { + partial_bloom_filters[i]->set_or_concat(partial_bloom_filter_build_params[i]->runtime_filter.get(), + _driver_sequence); + } + } + } + RuntimeBloomFilterList bloom_filters(partial_bloom_filters.begin(), partial_bloom_filters.end()); + runtime_filter_hub()->set_collector(_plan_node_id, _driver_sequence, + std::make_unique(in_filter_lists)); + state->runtime_filter_port()->publish_local_colocate_filters(bloom_filters); + + } else { + // add partial filters generated by this HashJoinBuildOperator to PartialRuntimeFilterMerger to merge into a + // total one. + bool all_build_merged = false; + { + SCOPED_TIMER(_join_builder->build_metrics().build_runtime_filter_timer); + auto status = _partial_rf_merger->add_partial_filters( + merger_index, ht_row_count, std::move(partial_in_filters), + std::move(partial_bloom_filter_build_params), std::move(partial_bloom_filters)); + ASSIGN_OR_RETURN(all_build_merged, status); + } + + if (all_build_merged) { + auto&& in_filters = _partial_rf_merger->get_total_in_filters(); + auto&& bloom_filters = _partial_rf_merger->get_total_bloom_filters(); + + // publish runtime bloom-filters + state->runtime_filter_port()->publish_runtime_filters(bloom_filters); + // move runtime filters into RuntimeFilterHub. + runtime_filter_hub()->set_collector(_plan_node_id, + std::make_unique(std::move(in_filters))); + } } _join_builder->enter_probe_phase(); diff --git a/be/src/exec/pipeline/olap_table_sink_operator.h b/be/src/exec/pipeline/olap_table_sink_operator.h index 1413907fa42075..6a683c15974e78 100644 --- a/be/src/exec/pipeline/olap_table_sink_operator.h +++ b/be/src/exec/pipeline/olap_table_sink_operator.h @@ -96,8 +96,6 @@ class OlapTableSinkOperatorFactory final : public OperatorFactory { private: void _increment_num_sinkers_no_barrier() { _num_sinkers.fetch_add(1, std::memory_order_relaxed); } - - std::unique_ptr _data_sink; starrocks::stream_load::OlapTableSink* _sink0; FragmentContext* const _fragment_ctx; std::atomic _num_sinkers = 0; diff --git a/be/src/exec/pipeline/operator.cpp b/be/src/exec/pipeline/operator.cpp index d8607433364378..e82898ae2b443b 100644 --- a/be/src/exec/pipeline/operator.cpp +++ b/be/src/exec/pipeline/operator.cpp @@ -17,6 +17,7 @@ #include #include +#include "common/logging.h" #include "exec/exec_node.h" #include "exec/pipeline/query_context.h" #include "gutil/strings/substitute.h" @@ -90,7 +91,12 @@ void Operator::set_prepare_time(int64_t cost_ns) { } void Operator::set_precondition_ready(RuntimeState* state) { + _runtime_in_filters = _factory->get_colocate_runtime_in_filters(_driver_sequence); _factory->prepare_runtime_in_filters(state); + const auto& instance_runtime_filters = _factory->get_runtime_in_filters(); + _runtime_in_filters.insert(_runtime_in_filters.end(), instance_runtime_filters.begin(), + instance_runtime_filters.end()); + VLOG_QUERY << "local in runtime filter num:" << _runtime_in_filters.size(); } const LocalRFWaitingSet& Operator::rf_waiting_set() const { @@ -136,7 +142,7 @@ void Operator::close(RuntimeState* state) { } std::vector& Operator::runtime_in_filters() { - return _factory->get_runtime_in_filters(); + return _runtime_in_filters; } RuntimeFilterProbeCollector* Operator::runtime_bloom_filters() { @@ -243,6 +249,7 @@ void Operator::_init_rf_counters(bool init_bloom) { ADD_COUNTER(_common_metrics, "JoinRuntimeFilterOutputRows", TUnit::UNIT); _bloom_filter_eval_context.join_runtime_filter_eval_counter = ADD_COUNTER(_common_metrics, "JoinRuntimeFilterEvaluate", TUnit::UNIT); + _bloom_filter_eval_context.driver_sequence = _driver_sequence; } } @@ -268,10 +275,10 @@ Status OperatorFactory::prepare(RuntimeState* state) { _state = state; if (_runtime_filter_collector) { // TODO(hcf) no proper profile for rf_filter_collector attached to - RETURN_IF_ERROR(_runtime_filter_collector->prepare(state, _row_desc, _runtime_profile.get())); + RETURN_IF_ERROR(_runtime_filter_collector->prepare(state, _runtime_profile.get())); auto& descriptors = _runtime_filter_collector->get_rf_probe_collector()->descriptors(); for (auto& [filter_id, desc] : descriptors) { - if (desc->is_local() || desc->runtime_filter() != nullptr) { + if (desc->is_local() || desc->runtime_filter(-1) != nullptr) { continue; } auto grf = state->exec_env()->runtime_filter_cache()->get(state->query_id(), filter_id); @@ -297,7 +304,12 @@ void OperatorFactory::close(RuntimeState* state) { } void OperatorFactory::_prepare_runtime_in_filters(RuntimeState* state) { - auto holders = _runtime_filter_hub->gather_holders(_rf_waiting_set); + auto holders = _runtime_filter_hub->gather_holders(_rf_waiting_set, -1, true); + _prepare_runtime_holders(holders, &_runtime_in_filters); +} + +void OperatorFactory::_prepare_runtime_holders(const std::vector& holders, + std::vector* runtime_in_filters) { for (auto& holder : holders) { DCHECK(holder->is_ready()); auto* collector = holder->get_collector(); @@ -306,13 +318,20 @@ void OperatorFactory::_prepare_runtime_in_filters(RuntimeState* state) { auto&& in_filters = collector->get_in_filters_bounded_by_tuple_ids(_tuple_ids); for (auto* filter : in_filters) { - WARN_IF_ERROR(filter->prepare(state), "prepare filter expression failed"); - WARN_IF_ERROR(filter->open(state), "open filter expression failed"); - _runtime_in_filters.push_back(filter); + WARN_IF_ERROR(filter->prepare(runtime_state()), "prepare filter expression failed"); + WARN_IF_ERROR(filter->open(runtime_state()), "open filter expression failed"); + runtime_in_filters->push_back(filter); } } } +std::vector OperatorFactory::get_colocate_runtime_in_filters(size_t driver_sequence) { + std::vector runtime_in_filter; + auto holders = _runtime_filter_hub->gather_holders(_rf_waiting_set, driver_sequence, true); + _prepare_runtime_holders(holders, &runtime_in_filter); + return runtime_in_filter; +} + bool OperatorFactory::has_runtime_filters() const { // Check runtime in-filters. if (!_rf_waiting_set.empty()) { diff --git a/be/src/exec/pipeline/operator.h b/be/src/exec/pipeline/operator.h index 39f1e762580815..ba60e2e5dc65c8 100644 --- a/be/src/exec/pipeline/operator.h +++ b/be/src/exec/pipeline/operator.h @@ -326,6 +326,7 @@ class Operator { // The MemTracker is owned by QueryContext, so that all the operators with the same plan_node_id can share // the same MemTracker. MemTracker* _mem_tracker = nullptr; + std::vector _runtime_in_filters; }; class OperatorFactory { @@ -372,6 +373,8 @@ class OperatorFactory { RuntimeFilterHub* runtime_filter_hub() { return _runtime_filter_hub; } std::vector& get_runtime_in_filters() { return _runtime_in_filters; } + // acquire local colocate runtime filter + std::vector get_colocate_runtime_in_filters(size_t driver_sequence); RuntimeFilterProbeCollector* get_runtime_bloom_filters() { if (_runtime_filter_collector == nullptr) { return nullptr; @@ -402,6 +405,8 @@ class OperatorFactory { protected: void _prepare_runtime_in_filters(RuntimeState* state); + void _prepare_runtime_holders(const std::vector& holders, + std::vector* runtime_in_filters); const int32_t _id; const std::string _name; diff --git a/be/src/exec/pipeline/pipeline.cpp b/be/src/exec/pipeline/pipeline.cpp index 4e8f96b5f94da9..0fab38cd5d7fb6 100644 --- a/be/src/exec/pipeline/pipeline.cpp +++ b/be/src/exec/pipeline/pipeline.cpp @@ -15,6 +15,7 @@ #include "exec/pipeline/pipeline.h" #include "exec/pipeline/adaptive/event.h" +#include "exec/pipeline/group_execution/execution_group.h" #include "exec/pipeline/operator.h" #include "exec/pipeline/pipeline_driver.h" #include "exec/pipeline/scan/connector_scan_operator.h" @@ -23,8 +24,11 @@ namespace starrocks::pipeline { -Pipeline::Pipeline(uint32_t id, OpFactories op_factories) - : _id(id), _op_factories(std::move(op_factories)), _pipeline_event(Event::create_event()) { +Pipeline::Pipeline(uint32_t id, OpFactories op_factories, ExecutionGroupRawPtr execution_group) + : _id(id), + _op_factories(std::move(op_factories)), + _pipeline_event(Event::create_event()), + _execution_group(execution_group) { _runtime_profile = std::make_shared(strings::Substitute("Pipeline (id=$0)", _id)); } @@ -38,7 +42,7 @@ void Pipeline::count_down_driver(RuntimeState* state) { bool all_drivers_finished = ++_num_finished_drivers >= num_drivers; if (all_drivers_finished) { _pipeline_event->finish(state); - state->fragment_ctx()->count_down_pipeline(); + _execution_group->count_down_pipeline(state); } } @@ -112,6 +116,8 @@ void Pipeline::setup_pipeline_profile(RuntimeState* runtime_state) { } void Pipeline::setup_drivers_profile(const DriverPtr& driver) { + runtime_profile()->add_info_string("isGroupExecution", + _execution_group->is_colocate_exec_group() ? "true" : "false"); runtime_profile()->add_child(driver->runtime_profile(), true, nullptr); auto* dop_counter = ADD_COUNTER_SKIP_MERGE(runtime_profile(), "DegreeOfParallelism", TUnit::UNIT, TCounterMergeType::SKIP_ALL); @@ -133,7 +139,7 @@ void Pipeline::setup_drivers_profile(const DriverPtr& driver) { void Pipeline::count_down_epoch_finished_driver(RuntimeState* state) { bool all_drivers_finished = ++_num_epoch_finished_drivers == _drivers.size(); if (all_drivers_finished) { - state->fragment_ctx()->count_down_epoch_pipeline(state); + _execution_group->count_down_epoch_pipeline(state); } } diff --git a/be/src/exec/pipeline/pipeline.h b/be/src/exec/pipeline/pipeline.h index 534eb235eb1463..7348501f574c49 100644 --- a/be/src/exec/pipeline/pipeline.h +++ b/be/src/exec/pipeline/pipeline.h @@ -18,6 +18,7 @@ #include #include "exec/pipeline/adaptive/adaptive_fwd.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" #include "exec/pipeline/operator.h" #include "exec/pipeline/pipeline_fwd.h" #include "exec/pipeline/source_operator.h" @@ -32,13 +33,10 @@ namespace pipeline { class Pipeline { public: Pipeline() = delete; - Pipeline(uint32_t id, OpFactories op_factories); + Pipeline(uint32_t id, OpFactories op_factories, ExecutionGroupRawPtr execution_group); uint32_t get_id() const { return _id; } - OpFactories& get_op_factories() { return _op_factories; } - void add_op_factory(const OpFactoryPtr& op) { _op_factories.emplace_back(op); } - Operators create_operators(int32_t degree_of_parallelism, int32_t i) { Operators operators; for (const auto& factory : _op_factories) { @@ -111,7 +109,7 @@ class Pipeline { std::atomic _num_finished_drivers = 0; EventPtr _pipeline_event; - + ExecutionGroupRawPtr _execution_group = nullptr; // STREAM MV std::atomic _num_epoch_finished_drivers = 0; }; diff --git a/be/src/exec/pipeline/pipeline_builder.cpp b/be/src/exec/pipeline/pipeline_builder.cpp index e6bd59833df261..83455eecc4dd4b 100644 --- a/be/src/exec/pipeline/pipeline_builder.cpp +++ b/be/src/exec/pipeline/pipeline_builder.cpp @@ -14,6 +14,8 @@ #include "exec/pipeline/pipeline_builder.h" +#include + #include "adaptive/event.h" #include "common/config.h" #include "exec/exec_node.h" @@ -21,7 +23,12 @@ #include "exec/pipeline/adaptive/collect_stats_sink_operator.h" #include "exec/pipeline/adaptive/collect_stats_source_operator.h" #include "exec/pipeline/exchange/exchange_source_operator.h" +#include "exec/pipeline/exchange/local_exchange_source_operator.h" +#include "exec/pipeline/group_execution/execution_group.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" +#include "exec/pipeline/group_execution/group_operator.h" #include "exec/pipeline/noop_sink_operator.h" +#include "exec/pipeline/pipeline_fwd.h" #include "exec/pipeline/spill_process_operator.h" #include "exec/query_cache/cache_manager.h" #include "exec/query_cache/cache_operator.h" @@ -31,6 +38,22 @@ namespace starrocks::pipeline { +void PipelineBuilderContext::init_colocate_groups(std::unordered_map&& colocate_groups) { + _group_id_to_colocate_groups = std::move(colocate_groups); + for (auto& [group_id, group] : _group_id_to_colocate_groups) { + _execution_groups.emplace_back(group); + } +} + +ExecutionGroupRawPtr PipelineBuilderContext::find_exec_group_by_plan_node_id(int32_t plan_node_id) { + for (auto& [group_id, group] : _group_id_to_colocate_groups) { + if (group->contains(plan_node_id)) { + return group.get(); + } + } + return _normal_exec_group; +} + /// PipelineBuilderContext. OpFactories PipelineBuilderContext::maybe_interpolate_local_broadcast_exchange(RuntimeState* state, int32_t plan_node_id, @@ -98,6 +121,8 @@ OpFactories PipelineBuilderContext::_maybe_interpolate_local_passthrough_exchang return pred_operators; } + pred_operators = maybe_interpolate_grouped_exchange(plan_node_id, pred_operators); + int max_input_dop = std::max(num_receivers, static_cast(source_op->degree_of_parallelism())); auto mem_mgr = std::make_shared(max_input_dop, config::local_exchange_buffer_mem_limit_per_driver); @@ -176,6 +201,10 @@ OpFactories PipelineBuilderContext::_do_maybe_interpolate_local_shuffle_exchange const std::vector& partition_expr_ctxs, const TPartitionType::type part_type) { DCHECK(!pred_operators.empty() && pred_operators[0]->is_source()); + // interpolate grouped exchange if needed + // TODO: If the local exchange supports spills, we don't need to prevent group execution + pred_operators = maybe_interpolate_grouped_exchange(plan_node_id, pred_operators); + // If DOP is one, we needn't partition input chunks. size_t shuffle_partitions_num = degree_of_parallelism(); if (shuffle_partitions_num <= 1) { @@ -208,6 +237,11 @@ OpFactories PipelineBuilderContext::maybe_interpolate_local_ordered_partition_ex RuntimeState* state, int32_t plan_node_id, OpFactories& pred_operators, const std::vector& partition_expr_ctxs) { DCHECK(!pred_operators.empty() && pred_operators[0]->is_source()); + + // interpolate grouped exchange if needed + // TODO: If the local exchange supports spills, we don't need to prevent group execution + pred_operators = maybe_interpolate_grouped_exchange(plan_node_id, pred_operators); + // If DOP is one, we needn't partition input chunks. size_t shuffle_partitions_num = degree_of_parallelism(); if (shuffle_partitions_num <= 1) { @@ -249,6 +283,40 @@ void PipelineBuilderContext::interpolate_spill_process(size_t plan_node_id, add_pipeline(std::move(spill_process_operators)); } +OpFactories PipelineBuilderContext::interpolate_grouped_exchange(int32_t plan_node_id, OpFactories& pred_operators) { + size_t physical_dop = degree_of_parallelism(); + auto* source_op = source_operator(pred_operators); + int logical_dop = source_op->degree_of_parallelism(); + + auto mem_mgr = + std::make_shared(logical_dop, config::local_exchange_buffer_mem_limit_per_driver); + auto local_shuffle_source = + std::make_shared(next_operator_id(), plan_node_id, mem_mgr); + auto local_exchanger = std::make_shared(mem_mgr, local_shuffle_source.get()); + auto group_exchange_sink = std::make_shared(next_operator_id(), plan_node_id, + local_exchanger, _current_execution_group); + pred_operators.emplace_back(std::move(group_exchange_sink)); + + auto prev_source_operator = source_operator(pred_operators); + inherit_upstream_source_properties(local_shuffle_source.get(), prev_source_operator); + local_shuffle_source->set_could_local_shuffle(true); + local_shuffle_source->set_degree_of_parallelism(physical_dop); + add_pipeline(pred_operators); + // switch to new execution group + _execution_groups.emplace_back(std::make_shared()); + _current_execution_group = _execution_groups.back().get(); + + return {local_shuffle_source}; +} + +OpFactories PipelineBuilderContext::maybe_interpolate_grouped_exchange(int32_t plan_node_id, + OpFactories& pred_operators) { + if (dynamic_cast(_current_execution_group) != nullptr) { + return interpolate_grouped_exchange(plan_node_id, pred_operators); + } + return pred_operators; +} + OpFactories PipelineBuilderContext::maybe_gather_pipelines_to_one(RuntimeState* state, int32_t plan_node_id, std::vector& pred_operators_list) { // If there is only one pred pipeline, we needn't local passthrough anymore. @@ -434,19 +502,23 @@ void PipelineBuilderContext::pop_dependent_pipeline() { _dependent_pipelines.pop_back(); } -void PipelineBuilderContext::subscribe_pipeline_event(Pipeline* pipeline, Event* event) { - pipeline->pipeline_event()->set_need_wait_dependencies_finished(true); - pipeline->pipeline_event()->add_dependency(event); +void PipelineBuilderContext::_subscribe_pipeline_event(Pipeline* pipeline) { + bool enable_wait_event = _fragment_context->runtime_state()->enable_wait_dependent_event(); + enable_wait_event &= !_current_execution_group->is_colocate_exec_group(); + if (enable_wait_event && !_dependent_pipelines.empty()) { + pipeline->pipeline_event()->set_need_wait_dependencies_finished(true); + pipeline->pipeline_event()->add_dependency(_dependent_pipelines.back()->pipeline_event()); + } } OpFactories PipelineBuilder::decompose_exec_node_to_pipeline(const FragmentContext& fragment, ExecNode* exec_node) { pipeline::OpFactories operators = exec_node->decompose_to_pipeline(&_context); + operators = _context.maybe_interpolate_grouped_exchange(exec_node->id(), operators); return operators; } -/// PipelineBuilder. -Pipelines PipelineBuilder::build() { - return _context.get_pipelines(); +ExecutionGroups PipelineBuilder::build() { + return _context.execution_groups(); } } // namespace starrocks::pipeline diff --git a/be/src/exec/pipeline/pipeline_builder.h b/be/src/exec/pipeline/pipeline_builder.h index 6204c53546b88d..03e3d56f151a0e 100644 --- a/be/src/exec/pipeline/pipeline_builder.h +++ b/be/src/exec/pipeline/pipeline_builder.h @@ -14,12 +14,15 @@ #pragma once -#include +#include #include "exec/pipeline/exchange/local_exchange.h" #include "exec/pipeline/exchange/local_exchange_sink_operator.h" #include "exec/pipeline/exchange/local_exchange_source_operator.h" #include "exec/pipeline/fragment_context.h" +#include "exec/pipeline/group_execution/execution_group.h" +#include "exec/pipeline/group_execution/execution_group_builder.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" #include "exec/pipeline/pipeline.h" #include "exec/pipeline/spill_process_channel.h" @@ -34,16 +37,32 @@ class PipelineBuilderContext { : _fragment_context(fragment_context), _degree_of_parallelism(degree_of_parallelism), _data_sink_dop(sink_dop), - _is_stream_pipeline(is_stream_pipeline) {} - - void add_pipeline(const OpFactories& operators) { - _pipelines.emplace_back(std::make_shared(next_pipe_id(), operators)); - bool enable_wait_event = _fragment_context->runtime_state()->enable_wait_dependent_event(); - if (enable_wait_event && !_dependent_pipelines.empty()) { - subscribe_pipeline_event(_pipelines.back().get(), _dependent_pipelines.back()->pipeline_event()); - } + _is_stream_pipeline(is_stream_pipeline), + _enable_group_execution(fragment_context->enable_group_execution()) { + // init the default execution group + _execution_groups.emplace_back(ExecutionGroupBuilder::create_normal_exec_group()); + _normal_exec_group = _execution_groups.back().get(); + _current_execution_group = _execution_groups.back().get(); } + void init_colocate_groups(std::unordered_map&& colocate_groups); + ExecutionGroupRawPtr find_exec_group_by_plan_node_id(int32_t plan_node_id); + void set_current_execution_group(ExecutionGroupRawPtr exec_group) { _current_execution_group = exec_group; } + ExecutionGroupRawPtr current_execution_group() { return _current_execution_group; } + + void add_pipeline(const OpFactories& operators, ExecutionGroupRawPtr execution_group) { + // TODO: refactor Pipelines to PipelineRawPtrs + _pipelines.emplace_back(std::make_shared(next_pipe_id(), operators, execution_group)); + execution_group->add_pipeline(_pipelines.back()); + _subscribe_pipeline_event(_pipelines.back().get()); + } + + void add_pipeline(const OpFactories& operators) { add_pipeline(operators, _current_execution_group); } + + void add_independent_pipeline(const OpFactories& operators) { add_pipeline(operators, _normal_exec_group); } + + bool is_colocate_group() const { return _current_execution_group->type() == ExecutionGroupType::COLOCATE; } + OpFactories maybe_interpolate_local_broadcast_exchange(RuntimeState* state, int32_t plan_node_id, OpFactories& pred_operators, int num_receivers); @@ -89,6 +108,9 @@ class PipelineBuilderContext { void interpolate_spill_process(size_t plan_node_id, const SpillProcessChannelFactoryPtr& channel_factory, size_t dop); + OpFactories interpolate_grouped_exchange(int32_t plan_node_id, OpFactories& pred_operators); + OpFactories maybe_interpolate_grouped_exchange(int32_t plan_node_id, OpFactories& pred_operators); + // Uses local exchange to gather the output chunks of multiple predecessor pipelines // into a new pipeline, which the successor operator belongs to. // Append a LocalExchangeSinkOperator to the tail of each pipeline. @@ -108,7 +130,6 @@ class PipelineBuilderContext { bool is_stream_pipeline() const { return _is_stream_pipeline; } - const Pipelines& get_pipelines() const { return _pipelines; } const Pipeline* last_pipeline() const { DCHECK(!_pipelines.empty()); return _pipelines[_pipelines.size() - 1].get(); @@ -116,6 +137,7 @@ class PipelineBuilderContext { RuntimeState* runtime_state() { return _fragment_context->runtime_state(); } FragmentContext* fragment_context() { return _fragment_context; } + bool enable_group_execution() const { return _enable_group_execution; } size_t dop_of_source_operator(int source_node_id); MorselQueueFactory* morsel_queue_factory_of_source_operator(int source_node_id); @@ -141,9 +163,11 @@ class PipelineBuilderContext { void push_dependent_pipeline(const Pipeline* pipeline); void pop_dependent_pipeline(); - void subscribe_pipeline_event(Pipeline* pipeline, Event* event); + ExecutionGroups execution_groups() { return std::move(_execution_groups); } private: + void _subscribe_pipeline_event(Pipeline* pipeline); + OpFactories _maybe_interpolate_local_passthrough_exchange(RuntimeState* state, int32_t plan_node_id, OpFactories& pred_operators, int num_receivers, bool force, @@ -158,6 +182,10 @@ class PipelineBuilderContext { FragmentContext* _fragment_context; Pipelines _pipelines; + ExecutionGroups _execution_groups; + std::unordered_map _group_id_to_colocate_groups; + ExecutionGroupRawPtr _normal_exec_group = nullptr; + ExecutionGroupRawPtr _current_execution_group = nullptr; std::list _dependent_pipelines; @@ -168,16 +196,17 @@ class PipelineBuilderContext { const size_t _data_sink_dop; const bool _is_stream_pipeline; + const bool _enable_group_execution; }; class PipelineBuilder { public: explicit PipelineBuilder(PipelineBuilderContext& context) : _context(context) {} + // Build pipeline from exec node tree OpFactories decompose_exec_node_to_pipeline(const FragmentContext& fragment, ExecNode* exec_node); - // Build pipeline from exec node tree - Pipelines build(); + ExecutionGroups build(); private: PipelineBuilderContext& _context; diff --git a/be/src/exec/pipeline/pipeline_driver.cpp b/be/src/exec/pipeline/pipeline_driver.cpp index c403054cbbecc1..59887e9dfead07 100644 --- a/be/src/exec/pipeline/pipeline_driver.cpp +++ b/be/src/exec/pipeline/pipeline_driver.cpp @@ -156,8 +156,9 @@ Status PipelineDriver::prepare(RuntimeState* runtime_state) { if (!all_local_rf_set.empty()) { _runtime_profile->add_info_string("LocalRfWaitingSet", strings::Substitute("$0", all_local_rf_set.size())); } - _local_rf_holders = fragment_ctx()->runtime_filter_hub()->gather_holders(all_local_rf_set); - + size_t subscribe_filter_sequence = source_op->get_driver_sequence(); + _local_rf_holders = + fragment_ctx()->runtime_filter_hub()->gather_holders(all_local_rf_set, subscribe_filter_sequence); if (use_cache) { ssize_t cache_op_idx = -1; query_cache::CacheOperatorPtr cache_op = nullptr; diff --git a/be/src/exec/pipeline/pipeline_driver.h b/be/src/exec/pipeline/pipeline_driver.h index bfe4218fc1e0d8..b7cbeb98f6bbca 100644 --- a/be/src/exec/pipeline/pipeline_driver.h +++ b/be/src/exec/pipeline/pipeline_driver.h @@ -338,7 +338,7 @@ class PipelineDriver { _all_global_rf_ready_or_timeout = _precondition_block_timer_sw->elapsed_time() >= _global_rf_wait_timeout_ns || // Timeout, std::all_of(_global_rf_descriptors.begin(), _global_rf_descriptors.end(), [](auto* rf_desc) { - return rf_desc->is_local() || rf_desc->runtime_filter() != nullptr; + return rf_desc->is_local() || rf_desc->runtime_filter(-1) != nullptr; }); // or all the remote RFs are ready. return !_all_global_rf_ready_or_timeout; diff --git a/be/src/exec/pipeline/query_context.cpp b/be/src/exec/pipeline/query_context.cpp index b339280f609172..53f201b5715442 100644 --- a/be/src/exec/pipeline/query_context.cpp +++ b/be/src/exec/pipeline/query_context.cpp @@ -475,7 +475,7 @@ void QueryContextManager::report_fragments_with_same_host( if (reported[i] == false) { FragmentContext* fragment_ctx = need_report_fragment_context[i].get(); - if (fragment_ctx->all_pipelines_finished()) { + if (fragment_ctx->all_execution_groups_finished()) { reported[i] = true; continue; } @@ -578,7 +578,7 @@ void QueryContextManager::report_fragments( FragmentContext* fragment_ctx = need_report_fragment_context[i].get(); - if (fragment_ctx->all_pipelines_finished()) { + if (fragment_ctx->all_execution_groups_finished()) { continue; } diff --git a/be/src/exec/pipeline/runtime_filter_types.h b/be/src/exec/pipeline/runtime_filter_types.h index 7c809fedfb903f..598adb55288d24 100644 --- a/be/src/exec/pipeline/runtime_filter_types.h +++ b/be/src/exec/pipeline/runtime_filter_types.h @@ -22,6 +22,7 @@ #include "exprs/expr_context.h" #include "exprs/predicate.h" #include "exprs/runtime_filter_bank.h" +#include "gen_cpp/Types_types.h" namespace starrocks::pipeline { class RuntimeFilterHolder; @@ -63,8 +64,8 @@ struct RuntimeBloomFilterBuildParam { struct RuntimeFilterCollector { RuntimeFilterCollector(RuntimeInFilterList&& in_filters, RuntimeBloomFilterList&& bloom_filters) : _in_filters(std::move(in_filters)), _bloom_filters(std::move(bloom_filters)) {} + RuntimeFilterCollector(RuntimeInFilterList in_filters) : _in_filters(std::move(in_filters)) {} - RuntimeBloomFilterList& get_bloom_filters() { return _bloom_filters; } RuntimeInFilterList& get_in_filters() { return _in_filters; } // In-filters are constructed by a node and may be pushed down to its descendant node. @@ -104,6 +105,7 @@ struct RuntimeFilterCollector { private: // local runtime in-filter RuntimeInFilterList _in_filters; + // TODO: unused, FIXME later // global/local runtime bloom-filter(including max-min filter) RuntimeBloomFilterList _bloom_filters; }; @@ -111,6 +113,7 @@ struct RuntimeFilterCollector { class RuntimeFilterHolder { public: void set_collector(RuntimeFilterCollectorPtr&& collector) { + DCHECK(_collector.load(std::memory_order_acquire) == nullptr); _collector_ownership = std::move(collector); _collector.store(_collector_ownership.get(), std::memory_order_release); } @@ -122,44 +125,81 @@ class RuntimeFilterHolder { std::atomic _collector; }; -// RuntimeFilterHub is a mediator that used to gather all runtime filters generated by HashJoinBuildOperator instances. -// It has a RuntimeFilterHolder for each HashJoinBuilder instance, when total runtime filter is generated, then it is -// added into RuntimeFilterHub; the operators consuming runtime filters inspect RuntimeFilterHub and find out its bounded -// runtime filters. RuntimeFilterHub is reserved beforehand, and there is no need to use mutex to guard concurrent access. +// RuntimeFilterHub is a mediator that used to gather all runtime filters generated by RuntimeFilterBuild instances. +// The life cycle of RuntimeFilterHub is the same as FragmentContext. +// RuntimeFilterHub maintains the mapping from RuntimeFilter Subscriber to RuntimeFilter Producer (RuntimeFilter Holder). +// Subscriber can be operator level (node_id, sequence) -> holder. or operator factory level (node_id, -1) -> holder. +// Typically, the operator level is used in colocate group execution mode, and the operator factory level is used in other modes. class RuntimeFilterHub { public: - void add_holder(TPlanNodeId id) { _holders.emplace(std::make_pair(id, std::make_unique())); } + void add_holder(TPlanNodeId id, int32_t total_dop = -1) { + if (total_dop > 0) { + for (size_t i = 0; i < total_dop; ++i) { + _holders[id].emplace(i, std::make_unique()); + } + } else { + _holders[id].emplace(-1, std::make_unique()); + } + } + void set_collector(TPlanNodeId id, RuntimeFilterCollectorPtr&& collector) { - get_holder(id)->set_collector(std::move(collector)); + get_holder(id, -1)->set_collector(std::move(collector)); + } + + void set_collector(TPlanNodeId id, int32_t sequence_id, RuntimeFilterCollectorPtr&& collector) { + get_holder(id, sequence_id)->set_collector(std::move(collector)); } void close_all_in_filters(RuntimeState* state) { - for (auto& [_, holder] : _holders) { - if (auto* collector = holder->get_collector()) { - for (auto& in_filter : collector->get_in_filters()) { - in_filter->close(state); + for (auto& [plan_node_id, seq_to_holder] : _holders) { + for (const auto& [seq, holder] : seq_to_holder) { + if (auto* collector = holder->get_collector()) { + for (auto& in_filter : collector->get_in_filters()) { + in_filter->close(state); + } } } } } - std::vector gather_holders(const std::set& ids) { + bool is_colocate_runtime_filters(TPlanNodeId plan_node_id) const { + auto it = _holders.find(plan_node_id); + DCHECK(it != _holders.end()); + return it->second.find(-1) == it->second.end(); + } + + // if strict is false, return instance level holder if not found pipeline level holder + std::vector gather_holders(const std::set& ids, size_t driver_sequence, + bool strict = false) { std::vector holders; holders.reserve(ids.size()); for (auto id : ids) { - holders.push_back(get_holder(id).get()); + if (auto holder = get_holder(id, driver_sequence, strict)) { + holders.push_back(holder); + } } return holders; } private: - RuntimeFilterHolderPtr& get_holder(TPlanNodeId id) { + RuntimeFilterHolder* get_holder(TPlanNodeId id, int32_t sequence_id, bool strict = false) { auto it = _holders.find(id); DCHECK(it != _holders.end()); - return it->second; + auto it_holder = it->second.find(sequence_id); + if (it_holder == it->second.end()) { + if (strict) return nullptr; + it_holder = it->second.find(-1); + DCHECK(it_holder != it->second.end()); + return it_holder->second.get(); + } + return it_holder->second.get(); } + + using SequenceToHolder = std::unordered_map; // Each HashJoinBuildOperatorFactory has a corresponding Holder indexed by its TPlanNodeId. - std::unordered_map _holders; + // For instance level runtime filters, the sequence_id is -1 + // For pipeline level runtime filters, the sequence_id is corresponding operator driver sequence + std::unordered_map _holders; }; // A ExecNode in non-pipeline engine can be decomposed into more than one OperatorFactories in pipeline engine. @@ -171,15 +211,17 @@ class RefCountedRuntimeFilterProbeCollector; using RefCountedRuntimeFilterProbeCollectorPtr = std::shared_ptr; class RefCountedRuntimeFilterProbeCollector { public: - RefCountedRuntimeFilterProbeCollector(size_t num_operators_generated, + RefCountedRuntimeFilterProbeCollector(size_t num_factories_generated, RuntimeFilterProbeCollector&& rf_probe_collector) - : _count((num_operators_generated << 32) | num_operators_generated), - _num_operators_generated(num_operators_generated), + : _count((num_factories_generated << 32) | num_factories_generated), + _num_operators_generated(num_factories_generated), _rf_probe_collector(std::move(rf_probe_collector)) {} - [[nodiscard]] Status prepare(RuntimeState* state, const RowDescriptor& row_desc, RuntimeProfile* p) { + template + Status prepare(RuntimeState* state, Args&&... args) { + // TODO: stdpain assign operator nums here if ((_count.fetch_sub(1) & PREPARE_COUNTER_MASK) == _num_operators_generated) { - RETURN_IF_ERROR(_rf_probe_collector.prepare(state, row_desc, p)); + RETURN_IF_ERROR(_rf_probe_collector.prepare(state, std::forward(args)...)); RETURN_IF_ERROR(_rf_probe_collector.open(state)); } return Status::OK(); diff --git a/be/src/exec/pipeline/scan/olap_scan_context.cpp b/be/src/exec/pipeline/scan/olap_scan_context.cpp index 99106119982b20..5361be6ba91162 100644 --- a/be/src/exec/pipeline/scan/olap_scan_context.cpp +++ b/be/src/exec/pipeline/scan/olap_scan_context.cpp @@ -103,7 +103,7 @@ Status OlapScanContext::capture_tablet_rowsets(const std::vector& runtime_in_filters, - RuntimeFilterProbeCollector* runtime_bloom_filters) { + RuntimeFilterProbeCollector* runtime_bloom_filters, int32_t driver_sequence) { const TOlapScanNode& thrift_olap_scan_node = _scan_node->thrift_olap_scan_node(); const TupleDescriptor* tuple_desc = state->desc_tbl().get_tuple_descriptor(thrift_olap_scan_node.tuple_id); @@ -126,6 +126,7 @@ Status OlapScanContext::parse_conjuncts(RuntimeState* state, const std::vectorquery_options(); int32_t max_scan_key_num; diff --git a/be/src/exec/pipeline/scan/olap_scan_context.h b/be/src/exec/pipeline/scan/olap_scan_context.h index 94872dd6b77cb8..39a3624273ee44 100644 --- a/be/src/exec/pipeline/scan/olap_scan_context.h +++ b/be/src/exec/pipeline/scan/olap_scan_context.h @@ -97,7 +97,7 @@ class OlapScanContext final : public ContextWithDependency { bool is_prepare_finished() const { return _is_prepare_finished.load(std::memory_order_acquire); } Status parse_conjuncts(RuntimeState* state, const std::vector& runtime_in_filters, - RuntimeFilterProbeCollector* runtime_bloom_filters); + RuntimeFilterProbeCollector* runtime_bloom_filters, int32_t driver_sequence); OlapScanNode* scan_node() const { return _scan_node; } OlapScanConjunctsManager& conjuncts_manager() { return _conjuncts_manager; } diff --git a/be/src/exec/pipeline/scan/olap_scan_prepare_operator.cpp b/be/src/exec/pipeline/scan/olap_scan_prepare_operator.cpp index f3682e2acef41d..baceb97685aad0 100644 --- a/be/src/exec/pipeline/scan/olap_scan_prepare_operator.cpp +++ b/be/src/exec/pipeline/scan/olap_scan_prepare_operator.cpp @@ -62,7 +62,7 @@ bool OlapScanPrepareOperator::is_finished() const { } StatusOr OlapScanPrepareOperator::pull_chunk(RuntimeState* state) { - Status status = _ctx->parse_conjuncts(state, runtime_in_filters(), runtime_bloom_filters()); + Status status = _ctx->parse_conjuncts(state, runtime_in_filters(), runtime_bloom_filters(), _driver_sequence); _morsel_queue->set_key_ranges(_ctx->key_ranges()); _morsel_queue->set_tablets(_ctx->tablets()); diff --git a/be/src/exec/pipeline/stream_epoch_manager.cpp b/be/src/exec/pipeline/stream_epoch_manager.cpp index 118e412eeee9b2..c4bcf65ceee162 100644 --- a/be/src/exec/pipeline/stream_epoch_manager.cpp +++ b/be/src/exec/pipeline/stream_epoch_manager.cpp @@ -99,11 +99,11 @@ Status StreamEpochManager::prepare(const MVMaintenanceTaskInfo& maintenance_task _maintenance_task_info = maintenance_task_info; // TODO(lism): - // - Prepare enable_resource_gorup in FE. + // - Prepare enable_resource_group in FE. // - Ensure all fragment ctx's enable_resource_group are the same. for (auto* fragment_ctx : fragment_ctxs) { _enable_resource_group &= fragment_ctx->enable_resource_group(); - _num_drivers += fragment_ctx->num_drivers(); + _num_drivers += fragment_ctx->total_dop(); } return Status::OK(); } diff --git a/be/src/exec/topn_node.cpp b/be/src/exec/topn_node.cpp index 6561bd31477b6c..0a9a0bcbd2b469 100644 --- a/be/src/exec/topn_node.cpp +++ b/be/src/exec/topn_node.cpp @@ -265,6 +265,8 @@ std::vector> TopNNode::_decompose_to_ using namespace pipeline; OpFactories ops_sink_with_sort = _children[0]->decompose_to_pipeline(context); + ops_sink_with_sort = context->maybe_interpolate_grouped_exchange(_id, ops_sink_with_sort); + int64_t partition_limit = _limit; if (is_partition_topn) { diff --git a/be/src/exec/union_node.cpp b/be/src/exec/union_node.cpp index 76f02e446fbe59..1d0d16b0fd1433 100644 --- a/be/src/exec/union_node.cpp +++ b/be/src/exec/union_node.cpp @@ -346,7 +346,9 @@ pipeline::OpFactories UnionNode::decompose_to_pipeline(pipeline::PipelineBuilder size_t i = 0; // UnionPassthroughOperator is used for the passthrough sub-node. for (; i < _first_materialized_child_idx; i++) { - operators_list.emplace_back(child(i)->decompose_to_pipeline(context)); + auto child_ops = child(i)->decompose_to_pipeline(context); + child_ops = context->maybe_interpolate_grouped_exchange(_id, child_ops); + operators_list.emplace_back(child_ops); UnionPassthroughOperator::SlotMap* dst2src_slot_map = nullptr; if (!_pass_through_slot_maps.empty()) { @@ -370,7 +372,9 @@ pipeline::OpFactories UnionNode::decompose_to_pipeline(pipeline::PipelineBuilder // ProjectOperatorFactory is used for the materialized sub-node. for (; i < _children.size(); i++) { - operators_list.emplace_back(child(i)->decompose_to_pipeline(context)); + auto child_ops = child(i)->decompose_to_pipeline(context); + child_ops = context->maybe_interpolate_grouped_exchange(_id, child_ops); + operators_list.emplace_back(child_ops); const auto& dst_tuple_desc = context->fragment_context()->runtime_state()->desc_tbl().get_tuple_descriptor(_tuple_id); diff --git a/be/src/exprs/runtime_filter.h b/be/src/exprs/runtime_filter.h index 4680fc9df8ed0e..67c8dec1b8b4be 100644 --- a/be/src/exprs/runtime_filter.h +++ b/be/src/exprs/runtime_filter.h @@ -364,6 +364,11 @@ class JoinRuntimeFilter { virtual JoinRuntimeFilter* create_empty(ObjectPool* pool) = 0; void set_global() { this->_global = true; } + // only used in local colocate filter + bool is_colocate_filter() const { return !_colocate_filters.empty(); } + std::vector& colocate_filter() { return _colocate_filters; } + const std::vector& colocate_filter() const { return _colocate_filters; } + protected: void _update_version() { _rf_version++; } @@ -375,6 +380,8 @@ class JoinRuntimeFilter { std::vector _hash_partition_bf; bool _always_true = false; size_t _rf_version = 0; + // local colocate filters is local filter we don't have to serialize them + std::vector _colocate_filters; }; template diff --git a/be/src/exprs/runtime_filter_bank.cpp b/be/src/exprs/runtime_filter_bank.cpp index 289ac4e572312e..ce121b8dd1bd76 100644 --- a/be/src/exprs/runtime_filter_bank.cpp +++ b/be/src/exprs/runtime_filter_bank.cpp @@ -14,6 +14,7 @@ #include "exprs/runtime_filter_bank.h" +#include #include #include "column/column.h" @@ -273,7 +274,7 @@ Status RuntimeFilterProbeDescriptor::init(int32_t filter_id, ExprContext* probe_ return Status::OK(); } -Status RuntimeFilterProbeDescriptor::prepare(RuntimeState* state, const RowDescriptor& row_desc, RuntimeProfile* p) { +Status RuntimeFilterProbeDescriptor::prepare(RuntimeState* state, RuntimeProfile* p) { if (_probe_expr_ctx != nullptr) { RETURN_IF_ERROR(_probe_expr_ctx->prepare(state)); } @@ -346,13 +347,12 @@ RuntimeFilterProbeCollector::RuntimeFilterProbeCollector(RuntimeFilterProbeColle _eval_context(that._eval_context), _plan_node_id(that._plan_node_id) {} -Status RuntimeFilterProbeCollector::prepare(RuntimeState* state, const RowDescriptor& row_desc, - RuntimeProfile* profile) { +Status RuntimeFilterProbeCollector::prepare(RuntimeState* state, RuntimeProfile* profile) { _runtime_profile = profile; _runtime_state = state; for (auto& it : _descriptors) { RuntimeFilterProbeDescriptor* rf_desc = it.second; - RETURN_IF_ERROR(rf_desc->prepare(state, row_desc, profile)); + RETURN_IF_ERROR(rf_desc->prepare(state, profile)); } if (state != nullptr) { const TQueryOptions& options = state->query_options(); @@ -396,7 +396,7 @@ void RuntimeFilterProbeCollector::do_evaluate(Chunk* chunk, RuntimeBloomFilterEv for (auto& kv : seletivity_map) { RuntimeFilterProbeDescriptor* rf_desc = kv.second; - const JoinRuntimeFilter* filter = rf_desc->runtime_filter(); + const JoinRuntimeFilter* filter = rf_desc->runtime_filter(eval_context.driver_sequence); if (filter == nullptr || filter->always_true()) { continue; } @@ -429,7 +429,7 @@ void RuntimeFilterProbeCollector::do_evaluate_partial_chunk(Chunk* partial_chunk // without computing each rf's selectivity for (auto kv : _descriptors) { RuntimeFilterProbeDescriptor* rf_desc = kv.second; - const JoinRuntimeFilter* filter = rf_desc->runtime_filter(); + const JoinRuntimeFilter* filter = rf_desc->runtime_filter(eval_context.driver_sequence); if (filter == nullptr || filter->always_true()) { continue; } @@ -531,7 +531,7 @@ void RuntimeFilterProbeCollector::compute_hash_values(Chunk* chunk, Column* colu RuntimeBloomFilterEvalContext& eval_context) { // TODO: Hash values will be computed multi times for runtime filters with the same partition_by_exprs. SCOPED_TIMER(eval_context.join_runtime_filter_hash_timer); - const JoinRuntimeFilter* filter = rf_desc->runtime_filter(); + const JoinRuntimeFilter* filter = rf_desc->runtime_filter(eval_context.driver_sequence); DCHECK(filter); if (filter->num_hash_partitions() == 0) { return; @@ -564,7 +564,7 @@ void RuntimeFilterProbeCollector::update_selectivity(Chunk* chunk, RuntimeBloomF seletivity_map.clear(); for (auto& kv : _descriptors) { RuntimeFilterProbeDescriptor* rf_desc = kv.second; - const JoinRuntimeFilter* filter = rf_desc->runtime_filter(); + const JoinRuntimeFilter* filter = rf_desc->runtime_filter(eval_context.driver_sequence); if (filter == nullptr || filter->always_true()) { continue; } @@ -655,6 +655,7 @@ void RuntimeFilterProbeCollector::add_descriptor(RuntimeFilterProbeDescriptor* d _descriptors[desc->filter_id()] = desc; } +// only used in non-pipeline mode void RuntimeFilterProbeCollector::wait(bool on_scan_node) { if (_descriptors.empty()) return; @@ -676,7 +677,7 @@ void RuntimeFilterProbeCollector::wait(bool on_scan_node) { while (wait_time >= 0 && !wait_list.empty()) { auto it = wait_list.begin(); while (it != wait_list.end()) { - auto* rf = (*it)->runtime_filter(); + auto* rf = (*it)->runtime_filter(-1); // find runtime filter in cache. if (rf == nullptr) { JoinRuntimeFilterPtr t = _runtime_state->exec_env()->runtime_filter_cache()->get( @@ -704,7 +705,7 @@ void RuntimeFilterProbeCollector::wait(bool on_scan_node) { for (const auto& it : _descriptors) { auto* rf = it.second; int filter_id = rf->filter_id(); - bool ready = (rf->runtime_filter() != nullptr); + bool ready = (rf->runtime_filter(-1) != nullptr); VLOG_FILE << "RuntimeFilterCollector::wait start. filter_id = " << filter_id << ", plan_node_id = " << _plan_node_id << ", finst_id = " << _runtime_state->fragment_instance_id() diff --git a/be/src/exprs/runtime_filter_bank.h b/be/src/exprs/runtime_filter_bank.h index fbc1415f8e32d2..f54c3b9a2d8145 100644 --- a/be/src/exprs/runtime_filter_bank.h +++ b/be/src/exprs/runtime_filter_bank.h @@ -70,7 +70,8 @@ class RuntimeFilterHelper { }; // how to generate & publish this runtime filter -// it only happens in hash join node. +// used in runtime filter build node. (TOPN/NLJoin/HashJoin) +// in pipeline engine, all operators generated by the same factory use the same build descriptor. class RuntimeFilterBuildDescriptor : public WithLayoutMixin { public: RuntimeFilterBuildDescriptor() = default; @@ -88,6 +89,7 @@ class RuntimeFilterBuildDescriptor : public WithLayoutMixin { bool has_consumer() const { return _has_consumer; } const std::vector& merge_nodes() const { return _merge_nodes; } void set_runtime_filter(JoinRuntimeFilter* rf) { _runtime_filter = rf; } + // used in TopN filter to intersect with other runtime filters. void set_or_intersect_filter(JoinRuntimeFilter* rf) { std::lock_guard guard(_mutex); if (_runtime_filter) { @@ -97,10 +99,24 @@ class RuntimeFilterBuildDescriptor : public WithLayoutMixin { } } + // used in local colocate runtime filter + void set_or_concat(JoinRuntimeFilter* rf, int32_t driver_sequence) { + std::lock_guard guard(_mutex); + if (_runtime_filter == nullptr) { + _runtime_filter = rf; + _runtime_filter->colocate_filter().resize(_num_colocate_partition); + } + _runtime_filter->colocate_filter()[driver_sequence] = rf; + } + JoinRuntimeFilter* runtime_filter() { return _runtime_filter; } void set_is_pipeline(bool flag) { _is_pipeline = flag; } bool is_pipeline() const { return _is_pipeline; } + // TRuntimeFilterBuildJoinMode int8_t join_mode() const { return _join_mode; }; + // only use when layout type == local colocate + size_t num_colocate_partition() const { return _num_colocate_partition; } + void set_num_colocate_partition(size_t num) { _num_colocate_partition = num; } private: friend class HashJoinNode; @@ -118,6 +134,7 @@ class RuntimeFilterBuildDescriptor : public WithLayoutMixin { std::vector _merge_nodes; JoinRuntimeFilter* _runtime_filter = nullptr; bool _is_pipeline = false; + size_t _num_colocate_partition = 0; std::mutex _mutex; }; @@ -128,16 +145,13 @@ class RuntimeFilterProbeDescriptor : public WithLayoutMixin { Status init(ObjectPool* pool, const TRuntimeFilterDescription& desc, TPlanNodeId node_id, RuntimeState* state); // for testing. Status init(int32_t filter_id, ExprContext* probe_expr_ctx); - Status prepare(RuntimeState* state, const RowDescriptor& row_desc, RuntimeProfile* p); + Status prepare(RuntimeState* state, RuntimeProfile* p); Status open(RuntimeState* state); void close(RuntimeState* state); int32_t filter_id() const { return _filter_id; } bool skip_wait() const { return _skip_wait; } bool is_topn_filter() const { return _is_topn_filter; } ExprContext* probe_expr_ctx() { return _probe_expr_ctx; } - const JoinRuntimeFilter* runtime_filter() const { return _runtime_filter.load(); } - void set_runtime_filter(const JoinRuntimeFilter* rf); - void set_shared_runtime_filter(const std::shared_ptr& rf); bool is_bound(const std::vector& tuple_ids) const { return _probe_expr_ctx->root()->is_bound(tuple_ids); } // Disable pushing down runtime filters when: // - partition_by_exprs have multi columns; @@ -155,7 +169,6 @@ class RuntimeFilterProbeDescriptor : public WithLayoutMixin { LogicalType probe_expr_type() const { return _probe_expr_ctx->root()->type().type; } void replace_probe_expr_ctx(RuntimeState* state, const RowDescriptor& row_desc, ExprContext* new_probe_expr_ctx); std::string debug_string() const; - JoinRuntimeFilter::RunningContext* runtime_filter_ctx() { return &_runtime_filter_ctx; } bool is_local() const { return _is_local; } TPlanNodeId build_plan_node_id() const { return _build_plan_node_id; } TPlanNodeId probe_plan_node_id() const { return _probe_plan_node_id; } @@ -163,6 +176,18 @@ class RuntimeFilterProbeDescriptor : public WithLayoutMixin { int8_t join_mode() const { return _join_mode; }; const std::vector* partition_by_expr_contexts() const { return &_partition_by_exprs_contexts; } + const JoinRuntimeFilter* runtime_filter(int32_t driver_sequence) const { + auto runtime_filter = _runtime_filter.load(); + if (runtime_filter != nullptr && runtime_filter->is_colocate_filter()) { + DCHECK_GE(driver_sequence, 0); + DCHECK_LT(driver_sequence, runtime_filter->colocate_filter().size()); + return runtime_filter->colocate_filter()[driver_sequence]; + } + return runtime_filter; + } + void set_runtime_filter(const JoinRuntimeFilter* rf); + void set_shared_runtime_filter(const std::shared_ptr& rf); + private: friend class HashJoinNode; friend class hashJoiner; @@ -172,9 +197,6 @@ class RuntimeFilterProbeDescriptor : public WithLayoutMixin { bool _is_local; TPlanNodeId _build_plan_node_id; TPlanNodeId _probe_plan_node_id; - std::atomic _runtime_filter = nullptr; - std::shared_ptr _shared_runtime_filter = nullptr; - JoinRuntimeFilter::RunningContext _runtime_filter_ctx; // we want to measure when this runtime filter is applied since it's opened. RuntimeProfile::Counter* _latency_timer = nullptr; int64_t _open_timestamp = 0; @@ -183,6 +205,9 @@ class RuntimeFilterProbeDescriptor : public WithLayoutMixin { bool _is_topn_filter = false; bool _skip_wait = false; std::vector _partition_by_exprs_contexts; + + std::atomic _runtime_filter = nullptr; + std::shared_ptr _shared_runtime_filter = nullptr; }; // RuntimeFilterProbeCollector::do_evaluate function apply runtime bloom filter to Operators to filter chunk. @@ -194,6 +219,9 @@ struct RuntimeBloomFilterEvalContext { std::map selectivity; size_t input_chunk_nums = 0; int run_filter_nums = 0; + // driver sequence, used in colocate local runtime filter + // It represents the ith driver to call this runtime filter. + int32_t driver_sequence = -1; JoinRuntimeFilter::RunningContext running_context; RuntimeProfile::Counter* join_runtime_filter_timer = nullptr; RuntimeProfile::Counter* join_runtime_filter_hash_timer = nullptr; @@ -208,13 +236,15 @@ class RuntimeFilterProbeCollector { RuntimeFilterProbeCollector(); RuntimeFilterProbeCollector(RuntimeFilterProbeCollector&& that) noexcept; size_t size() const { return _descriptors.size(); } - Status prepare(RuntimeState* state, const RowDescriptor& row_desc, RuntimeProfile* p); + Status prepare(RuntimeState* state, RuntimeProfile* p); Status open(RuntimeState* state); void close(RuntimeState* state); void compute_hash_values(Chunk* chunk, Column* column, RuntimeFilterProbeDescriptor* rf_desc, RuntimeBloomFilterEvalContext& eval_context); + // only used in no-pipeline mode (deprecated) void evaluate(Chunk* chunk); + void evaluate(Chunk* chunk, RuntimeBloomFilterEvalContext& eval_context); // evaluate partial chunk that may not contain slots referenced by runtime filter void evaluate_partial_chunk(Chunk* partial_chunk, RuntimeBloomFilterEvalContext& eval_context); diff --git a/be/src/formats/orc/orc_chunk_reader.cpp b/be/src/formats/orc/orc_chunk_reader.cpp index cdab21b1d1dcc7..7264cecda73fe3 100644 --- a/be/src/formats/orc/orc_chunk_reader.cpp +++ b/be/src/formats/orc/orc_chunk_reader.cpp @@ -1191,7 +1191,7 @@ Status OrcChunkReader::set_conjuncts_and_runtime_filters(const std::vectordescriptors()) { RuntimeFilterProbeDescriptor* rf_desc = it.second; - const JoinRuntimeFilter* filter = rf_desc->runtime_filter(); + const JoinRuntimeFilter* filter = rf_desc->runtime_filter(-1); SlotId probe_slot_id; if (filter == nullptr || filter->has_null() || !rf_desc->is_probe_slot_ref(&probe_slot_id)) continue; auto it2 = _slot_id_to_desc.find(probe_slot_id); diff --git a/be/src/formats/parquet/file_reader.cpp b/be/src/formats/parquet/file_reader.cpp index 1d60a1b9b9b14e..14df6edb97a898 100644 --- a/be/src/formats/parquet/file_reader.cpp +++ b/be/src/formats/parquet/file_reader.cpp @@ -266,7 +266,8 @@ StatusOr FileReader::_filter_group(const tparquet::RowGroup& row_group) { for (auto& it : _scanner_ctx->runtime_filter_collector->descriptors()) { RuntimeFilterProbeDescriptor* rf_desc = it.second; - const JoinRuntimeFilter* filter = rf_desc->runtime_filter(); + // external node won't have colocate runtime filter + const JoinRuntimeFilter* filter = rf_desc->runtime_filter(-1); SlotId probe_slot_id; if (filter == nullptr || filter->has_null() || !rf_desc->is_probe_slot_ref(&probe_slot_id)) continue; // !!linear search slot by slot_id. diff --git a/be/src/runtime/runtime_filter_worker.cpp b/be/src/runtime/runtime_filter_worker.cpp index 0af3f108585ce3..31d358cbece3cf 100644 --- a/be/src/runtime/runtime_filter_worker.cpp +++ b/be/src/runtime/runtime_filter_worker.cpp @@ -176,6 +176,15 @@ void RuntimeFilterPort::publish_runtime_filters(std::list& rf_descs) { + RuntimeState* state = _state; + for (auto* rf_desc : rf_descs) { + auto* filter = rf_desc->runtime_filter(); + if (filter == nullptr) continue; + state->runtime_filter_port()->receive_runtime_filter(rf_desc->filter_id(), filter); + } +} + void RuntimeFilterPort::receive_runtime_filter(int32_t filter_id, const JoinRuntimeFilter* rf) { _state->exec_env()->add_rf_event({ _state->query_id(), diff --git a/be/src/runtime/runtime_filter_worker.h b/be/src/runtime/runtime_filter_worker.h index ff49d396a9b06d..676d6819b3e7ad 100644 --- a/be/src/runtime/runtime_filter_worker.h +++ b/be/src/runtime/runtime_filter_worker.h @@ -48,6 +48,7 @@ class RuntimeFilterPort { RuntimeFilterPort(RuntimeState* state) : _state(state) {} void add_listener(RuntimeFilterProbeDescriptor* rf_desc); void publish_runtime_filters(std::list& rf_descs); + void publish_local_colocate_filters(std::list& rf_descs); // receiver runtime filter allocated in this fragment instance(broadcast join generate it) // or allocated in this query(shuffle join generate global runtime filter) void receive_runtime_filter(int32_t filter_id, const JoinRuntimeFilter* rf); diff --git a/be/src/storage/olap_runtime_range_pruner.h b/be/src/storage/olap_runtime_range_pruner.h index e8c0ede12e456d..5fd4cbb0532f5c 100644 --- a/be/src/storage/olap_runtime_range_pruner.h +++ b/be/src/storage/olap_runtime_range_pruner.h @@ -28,13 +28,17 @@ class SlotDescriptor; class RuntimeFilterProbeDescriptor; class PredicateParser; class ColumnPredicate; +class RuntimeBloomFilterEvalContext; struct UnarrivedRuntimeFilterList { std::vector unarrived_runtime_filters; std::vector slot_descs; - void add_unarrived_rf(const RuntimeFilterProbeDescriptor* desc, const SlotDescriptor* slot_desc) { + int32_t driver_sequence = -1; + void add_unarrived_rf(const RuntimeFilterProbeDescriptor* desc, const SlotDescriptor* slot_desc, + int32_t driver_sequence_) { unarrived_runtime_filters.push_back(desc); slot_descs.push_back(slot_desc); + driver_sequence = driver_sequence_; } }; @@ -62,6 +66,7 @@ class OlapRuntimeScanRangePruner { private: std::vector _unarrived_runtime_filters; std::vector _slot_descs; + int32_t _driver_sequence = -1; std::vector _arrived_runtime_filters_masks; std::vector _rf_versions; PredicateParser* _parser = nullptr; diff --git a/be/src/storage/olap_runtime_range_pruner.hpp b/be/src/storage/olap_runtime_range_pruner.hpp index 2de2d6f41bd7e4..60b1ca6f89b435 100644 --- a/be/src/storage/olap_runtime_range_pruner.hpp +++ b/be/src/storage/olap_runtime_range_pruner.hpp @@ -33,7 +33,8 @@ struct RuntimeColumnPredicateBuilder { StatusOr>> operator()(const ColumnIdToGlobalDictMap* global_dictmaps, PredicateParser* parser, const RuntimeFilterProbeDescriptor* desc, - const SlotDescriptor* slot) { + const SlotDescriptor* slot, + int32_t driver_sequence) { // keep consistent with ColumnRangeBuilder if constexpr (ltype == TYPE_TIME || ltype == TYPE_NULL || ltype == TYPE_JSON || lt_is_float || lt_is_binary) { @@ -61,7 +62,7 @@ struct RuntimeColumnPredicateBuilder { RangeType& range = full_range; range.set_index_filter_only(true); - const JoinRuntimeFilter* rf = desc->runtime_filter(); + const JoinRuntimeFilter* rf = desc->runtime_filter(driver_sequence); // applied global-dict optimized column if constexpr (ltype == TYPE_VARCHAR) { @@ -176,7 +177,7 @@ inline Status OlapRuntimeScanRangePruner::_update(const ColumnIdToGlobalDictMap* // 1. runtime filter arrived // 2. runtime filter updated and read rows greater than rf_update_threhold // we will filter by index - if (auto rf = _unarrived_runtime_filters[i]->runtime_filter()) { + if (auto rf = _unarrived_runtime_filters[i]->runtime_filter(_driver_sequence)) { size_t rf_version = rf->rf_version(); if (_arrived_runtime_filters_masks[i] == 0 || (rf_version > _rf_versions[i] && raw_read_rows - _raw_read_rows > rf_update_threhold)) { @@ -197,13 +198,13 @@ inline Status OlapRuntimeScanRangePruner::_update(const ColumnIdToGlobalDictMap* inline auto OlapRuntimeScanRangePruner::_get_predicates(const ColumnIdToGlobalDictMap* global_dictmaps, size_t idx) -> StatusOr { - auto rf = _unarrived_runtime_filters[idx]->runtime_filter(); + auto rf = _unarrived_runtime_filters[idx]->runtime_filter(_driver_sequence); if (rf->has_null()) return PredicatesPtrs{}; // convert to olap filter auto slot_desc = _slot_descs[idx]; - return type_dispatch_predicate>(slot_desc->type().type, false, - detail::RuntimeColumnPredicateBuilder(), global_dictmaps, - _parser, _unarrived_runtime_filters[idx], slot_desc); + return type_dispatch_predicate>( + slot_desc->type().type, false, detail::RuntimeColumnPredicateBuilder(), global_dictmaps, _parser, + _unarrived_runtime_filters[idx], slot_desc, _driver_sequence); } inline auto OlapRuntimeScanRangePruner::_as_raw_predicates( @@ -222,6 +223,7 @@ inline void OlapRuntimeScanRangePruner::_init(const UnarrivedRuntimeFilterList& _slot_descs.emplace_back(params.slot_descs[i]); _arrived_runtime_filters_masks.emplace_back(); _rf_versions.emplace_back(); + _driver_sequence = params.driver_sequence; } } } diff --git a/be/test/exec/pipeline/pipeline_control_flow_test.cpp b/be/test/exec/pipeline/pipeline_control_flow_test.cpp index 3503b26bb7098a..5cf590f0dd71e5 100644 --- a/be/test/exec/pipeline/pipeline_control_flow_test.cpp +++ b/be/test/exec/pipeline/pipeline_control_flow_test.cpp @@ -382,7 +382,7 @@ TEST_F(TestPipelineControlFlow, test_two_operatories) { op_factories.push_back( std::make_shared(next_operator_id(), next_plan_node_id(), sinkCounter)); - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group.get())); }; start_test(); @@ -411,7 +411,7 @@ TEST_F(TestPipelineControlFlow, test_three_operatories) { op_factories.push_back( std::make_shared(next_operator_id(), next_plan_node_id(), sinkCounter)); - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group.get())); }; start_test(); @@ -449,7 +449,7 @@ TEST_F(TestPipelineControlFlow, test_multi_operators) { op_factories.push_back( std::make_shared(next_operator_id(), next_plan_node_id(), sinkCounter)); - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group.get())); }; start_test(); @@ -481,7 +481,7 @@ TEST_F(TestPipelineControlFlow, test_full_chunk_size) { op_factories.push_back( std::make_shared(next_operator_id(), next_plan_node_id(), sinkCounter)); - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group.get())); }; start_test(); @@ -511,7 +511,7 @@ TEST_F(TestPipelineControlFlow, test_multi_chunks) { op_factories.push_back( std::make_shared(next_operator_id(), next_plan_node_id(), sinkCounter)); - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group.get())); }; start_test(); diff --git a/be/test/exec/pipeline/pipeline_file_scan_node_test.cpp b/be/test/exec/pipeline/pipeline_file_scan_node_test.cpp index 7c79321c744dc9..506e542602f153 100644 --- a/be/test/exec/pipeline/pipeline_file_scan_node_test.cpp +++ b/be/test/exec/pipeline/pipeline_file_scan_node_test.cpp @@ -27,6 +27,7 @@ #include "exec/pipeline/exchange/local_exchange.h" #include "exec/pipeline/exchange/local_exchange_sink_operator.h" #include "exec/pipeline/exchange/local_exchange_source_operator.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" #include "exec/pipeline/pipeline.h" #include "exec/pipeline/pipeline_builder.h" #include "exec/pipeline/pipeline_driver_executor.h" @@ -40,6 +41,7 @@ #include "runtime/exec_env.h" #include "runtime/runtime_state.h" #include "storage/storage_engine.h" +#include "testutil/assert.h" #include "util/defer_op.h" #include "util/disk_info.h" #include "util/mem_info.h" @@ -130,6 +132,7 @@ class PipeLineFileScanNodeTest : public ::testing::Test { std::string _file = "./be/test/exec/test_data/csv_scanner/csv_file1"; Pipelines _pipelines; + ExecutionGroupPtr exec_group; }; ChunkPtr PipeLineFileScanNodeTest::_create_chunk(const std::vector& types) { @@ -225,39 +228,32 @@ DescriptorTbl* PipeLineFileScanNodeTest::_create_table_desc(const std::vectorset_pipelines(std::move(_pipelines)); ASSERT_TRUE(_fragment_ctx->prepare_all_pipelines().ok()); MorselQueueFactoryMap& morsel_queues = _fragment_ctx->morsel_queue_factories(); - const auto& pipelines = _fragment_ctx->pipelines(); - for (const auto& pipeline : pipelines) { + _fragment_ctx->iterate_pipeline([&morsel_queues](auto pipeline) { if (pipeline->source_operator_factory()->with_morsels()) { - auto source_id = pipeline->get_op_factories()[0]->plan_node_id(); + auto source_id = pipeline->source_operator_factory()->plan_node_id(); DCHECK(morsel_queues.count(source_id)); auto& morsel_queue_factory = morsel_queues[source_id]; pipeline->source_operator_factory()->set_morsel_queue_factory(morsel_queue_factory.get()); } - } + }); - for (const auto& pipeline : pipelines) { - pipeline->instantiate_drivers(_fragment_ctx->runtime_state()); - } + _fragment_ctx->iterate_pipeline( + [this](auto pipeline) { pipeline->instantiate_drivers(_fragment_ctx->runtime_state()); }); } void PipeLineFileScanNodeTest::execute_pipeline() { - Status prepare_status = _fragment_ctx->iterate_drivers( + _fragment_ctx->iterate_drivers( [state = _fragment_ctx->runtime_state()](const DriverPtr& driver) { return driver->prepare(state); }); - ASSERT_TRUE(prepare_status.ok()); - - ASSERT_TRUE(_fragment_ctx - ->iterate_drivers([exec_env = _exec_env](const DriverPtr& driver) { - exec_env->wg_driver_executor()->submit(driver.get()); - return Status::OK(); - }) - .ok()); + + _fragment_ctx->iterate_drivers([exec_env = _exec_env](const DriverPtr& driver) { + LOG(WARNING) << driver->to_readable_string(); + exec_env->wg_driver_executor()->submit(driver.get()); + }); } void PipeLineFileScanNodeTest::generate_morse_queue(const std::vector& scan_nodes, @@ -410,12 +406,17 @@ TEST_F(PipeLineFileScanNodeTest, CSVBasic) { starrocks::pipeline::CounterPtr sinkCounter = std::make_shared(); + exec_group = ExecutionGroupBuilder::create_normal_exec_group(); + OpFactories op_factories = file_scan_node->decompose_to_pipeline(_context); op_factories.push_back(std::make_shared( _context->next_operator_id(), 0, sinkCounter)); - _pipelines.push_back(std::make_shared(_context->next_pipe_id(), op_factories)); + _pipelines.push_back( + std::make_shared(_context->next_pipe_id(), op_factories, exec_group.get())); + exec_group->add_pipeline(_pipelines.back()); + _fragment_ctx->set_exec_groups({exec_group}); prepare_pipeline(); diff --git a/be/test/exec/pipeline/pipeline_test_base.cpp b/be/test/exec/pipeline/pipeline_test_base.cpp index c06629bc733753..d11928cbab55fd 100644 --- a/be/test/exec/pipeline/pipeline_test_base.cpp +++ b/be/test/exec/pipeline/pipeline_test_base.cpp @@ -19,10 +19,12 @@ #include "column/nullable_column.h" #include "common/config.h" #include "exec/pipeline/fragment_context.h" +#include "exec/pipeline/group_execution/execution_group_builder.h" #include "exec/pipeline/pipeline_driver_executor.h" #include "exec/workgroup/work_group.h" #include "exprs/function_context.h" #include "storage/chunk_helper.h" +#include "testutil/assert.h" #include "types/date_value.h" #include "types/timestamp_value.h" #include "util/thrift_util.h" @@ -53,7 +55,8 @@ OpFactories PipelineTestBase::maybe_interpolate_local_passthrough_exchange(OpFac // Add LocalExchangeSinkOperator to predecessor pipeline. pred_operators.emplace_back(std::move(local_exchange_sink)); // predecessor pipeline comes to end. - _pipelines.emplace_back(std::make_unique(next_pipeline_id(), pred_operators)); + _pipelines.emplace_back(std::make_unique(next_pipeline_id(), pred_operators, + _fragment_ctx->_execution_groups[0].get())); OpFactories operators_source_with_local_exchange; // Multiple LocalChangeSinkOperators pipe into one LocalChangeSourceOperator. @@ -105,30 +108,23 @@ void PipelineTestBase::_prepare() { _obj_pool = _runtime_state->obj_pool(); ASSERT_TRUE(_pipeline_builder != nullptr); - _pipelines.clear(); + exec_group = ExecutionGroupBuilder::create_normal_exec_group(); _pipeline_builder(_fragment_ctx->runtime_state()); - _fragment_ctx->set_pipelines(std::move(_pipelines)); - ASSERT_TRUE(_fragment_ctx->prepare_all_pipelines().ok()); - - const auto& pipelines = _fragment_ctx->pipelines(); - const size_t num_pipelines = pipelines.size(); - for (auto n = 0; n < num_pipelines; ++n) { - const auto& pipeline = pipelines[n]; - pipeline->instantiate_drivers(_fragment_ctx->runtime_state()); + for (auto pipeline : _pipelines) { + exec_group->add_pipeline(std::move(pipeline)); } + _pipelines.clear(); + _fragment_ctx->set_exec_groups({exec_group}); + ASSERT_TRUE(_fragment_ctx->prepare_all_pipelines().ok()); + _fragment_ctx->iterate_pipeline([this](Pipeline* pipeline) { pipeline->instantiate_drivers(_runtime_state); }); } void PipelineTestBase::_execute() { - Status prepare_status = _fragment_ctx->iterate_drivers( - [state = _fragment_ctx->runtime_state()](const DriverPtr& driver) { return driver->prepare(state); }); - ASSERT_TRUE(prepare_status.ok()); - - ASSERT_TRUE(_fragment_ctx - ->iterate_drivers([exec_env = _exec_env](const DriverPtr& driver) { - exec_env->wg_driver_executor()->submit(driver.get()); - return Status::OK(); - }) - .ok()); + _fragment_ctx->iterate_drivers( + [state = _fragment_ctx->runtime_state()](const DriverPtr& driver) { CHECK_OK(driver->prepare(state)); }); + + _fragment_ctx->iterate_drivers( + [exec_env = _exec_env](const DriverPtr& driver) { exec_env->wg_driver_executor()->submit(driver.get()); }); } ChunkPtr PipelineTestBase::_create_and_fill_chunk(const std::vector& slots, size_t row_num) { diff --git a/be/test/exec/pipeline/pipeline_test_base.h b/be/test/exec/pipeline/pipeline_test_base.h index bd657f286311c4..db5cc482e7082a 100644 --- a/be/test/exec/pipeline/pipeline_test_base.h +++ b/be/test/exec/pipeline/pipeline_test_base.h @@ -17,6 +17,9 @@ #include "exec/pipeline/exchange/local_exchange.h" #include "exec/pipeline/exchange/local_exchange_sink_operator.h" #include "exec/pipeline/exchange/local_exchange_source_operator.h" +#include "exec/pipeline/group_execution/execution_group.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" +#include "exec/pipeline/pipeline_fwd.h" #include "gen_cpp/InternalService_types.h" #include "gtest/gtest.h" #include "runtime/descriptors.h" @@ -60,7 +63,9 @@ class PipelineTestBase : public ::testing::Test { // lambda used to init _pipelines std::function _pipeline_builder; + Pipelines _pipelines; + ExecutionGroupPtr exec_group; private: // Prepare execution context of pipeline diff --git a/be/test/exec/query_cache/query_cache_test.cpp b/be/test/exec/query_cache/query_cache_test.cpp index 72b1eec26961ad..cb9b71607c320f 100644 --- a/be/test/exec/query_cache/query_cache_test.cpp +++ b/be/test/exec/query_cache/query_cache_test.cpp @@ -22,6 +22,8 @@ #include "column/fixed_length_column.h" #include "column/vectorized_fwd.h" +#include "exec/pipeline/group_execution/execution_group_builder.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" #include "exec/pipeline/pipeline.h" #include "exec/pipeline/pipeline_driver.h" #include "exec/query_cache/cache_manager.h" @@ -235,7 +237,7 @@ Tasks create_test_pipelines(const query_cache::CacheParam& cache_param, size_t d Tasks tasks; tasks.resize(dop); for (auto i = 0; i < dop; ++i) { - pipeline::Pipeline pipeline(0, opFactories); + pipeline::Pipeline pipeline(0, opFactories, nullptr); auto upstream_operators = pipeline.create_operators(dop, i); auto downstream_operator = reduce_source->create(dop, i); tasks[i].upstream = std::move(upstream_operators); diff --git a/be/test/exec/stream/stream_operators_test.cpp b/be/test/exec/stream/stream_operators_test.cpp index 40bd8d20be1b6a..ef09c1c6df65af 100644 --- a/be/test/exec/stream/stream_operators_test.cpp +++ b/be/test/exec/stream/stream_operators_test.cpp @@ -188,16 +188,17 @@ void StreamOperatorsTest::_generate_morse_queue(ConnectorScanNode* scan_node, } TEST_F(StreamOperatorsTest, Dop_1) { - ASSERT_IF_ERROR(start_mv([&]() { + ASSERT_IF_ERROR(start_mv([&](auto* stream_ctx) { + auto exec_group = stream_ctx->exec_group.get(); _degree_of_parallelism = 1; - _pipeline_builder = [&](RuntimeState* state) { + _pipeline_builder = [=](RuntimeState* state) { OpFactories op_factories{ std::make_shared( next_operator_id(), next_plan_node_id(), GeneratorStreamSourceParam{.num_column = 2, .start = 0, .step = 1, .chunk_size = 4}), std::make_shared(next_operator_id(), next_plan_node_id()), }; - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group)); }; return Status::OK(); })); @@ -211,9 +212,10 @@ TEST_F(StreamOperatorsTest, Dop_1) { } TEST_F(StreamOperatorsTest, MultiDop_4) { - ASSERT_IF_ERROR(start_mv([&]() { + ASSERT_IF_ERROR(start_mv([&](auto* stream_ctx) { + auto exec_group = stream_ctx->exec_group.get(); _degree_of_parallelism = 4; - _pipeline_builder = [&](RuntimeState* state) { + _pipeline_builder = [=](RuntimeState* state) { OpFactories op_factories; auto source_factory = std::make_shared( next_operator_id(), next_plan_node_id(), @@ -222,10 +224,10 @@ TEST_F(StreamOperatorsTest, MultiDop_4) { source_factory->set_degree_of_parallelism(_degree_of_parallelism); op_factories.emplace_back(std::move(source_factory)); // add exchange node to gather multi source operator to one sink operator - op_factories = maybe_interpolate_local_passthrough_exchange(op_factories); + op_factories = maybe_interpolate_local_passthrough_exchange(op_factories, exec_group); op_factories.emplace_back( std::make_shared(next_operator_id(), next_plan_node_id())); - auto pipeline = std::make_shared(next_pipeline_id(), op_factories); + auto pipeline = std::make_shared(next_pipeline_id(), op_factories, exec_group); _pipelines.push_back(std::move(pipeline)); }; return Status::OK(); @@ -243,9 +245,10 @@ TEST_F(StreamOperatorsTest, MultiDop_4) { } TEST_F(StreamOperatorsTest, Test_StreamAggregator_Dop1) { - ASSERT_IF_ERROR(start_mv([&]() { + ASSERT_IF_ERROR(start_mv([&](auto* stream_ctx) { + auto exec_group = stream_ctx->exec_group.get(); _degree_of_parallelism = 1; - _pipeline_builder = [&](RuntimeState* state) { + _pipeline_builder = [=](RuntimeState* state) { _slot_infos = std::vector>{ // input slots { @@ -279,7 +282,7 @@ TEST_F(StreamOperatorsTest, Test_StreamAggregator_Dop1) { _stream_aggregator), std::make_shared(next_operator_id(), next_plan_node_id()), }; - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group)); }; return Status::OK(); })); @@ -296,9 +299,10 @@ TEST_F(StreamOperatorsTest, Test_StreamAggregator_Dop1) { } TEST_F(StreamOperatorsTest, Test_StreamAggregator_MultiDop) { - ASSERT_IF_ERROR(start_mv([&]() { + ASSERT_IF_ERROR(start_mv([&](auto* stream_ctx) { + auto exec_group = stream_ctx->exec_group.get(); _degree_of_parallelism = 4; - _pipeline_builder = [&](RuntimeState* state) { + _pipeline_builder = [=](RuntimeState* state) { _slot_infos = std::vector>{ // input slots { @@ -331,12 +335,12 @@ TEST_F(StreamOperatorsTest, Test_StreamAggregator_MultiDop) { source_factory->set_degree_of_parallelism(_degree_of_parallelism); op_factories.emplace_back(std::move(source_factory)); // add exchange node to gather multi source operator to one sink operator - op_factories = maybe_interpolate_local_passthrough_exchange(op_factories); + op_factories = maybe_interpolate_local_passthrough_exchange(op_factories, exec_group); op_factories.emplace_back(std::make_shared( next_operator_id(), next_plan_node_id(), _stream_aggregator)); op_factories.emplace_back( std::make_shared(next_operator_id(), next_plan_node_id())); - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group)); }; return Status::OK(); })); @@ -354,9 +358,10 @@ TEST_F(StreamOperatorsTest, Test_StreamAggregator_MultiDop) { } TEST_F(StreamOperatorsTest, binlog_dop_1) { - ASSERT_IF_ERROR(start_mv([&]() { + ASSERT_IF_ERROR(start_mv([&](auto* stream_ctx) { + auto exec_group = stream_ctx->exec_group.get(); _degree_of_parallelism = 1; - _pipeline_builder = [&](RuntimeState* state) { + _pipeline_builder = [=](RuntimeState* state) { auto* descs = _create_table_desc(2, 4); auto tnode = _create_tplan_node(next_plan_node_id(), 0); auto binlog_scan_node = std::make_shared(_obj_pool, *tnode, *descs); @@ -371,7 +376,7 @@ TEST_F(StreamOperatorsTest, binlog_dop_1) { op_factories.push_back( std::make_shared(next_operator_id(), next_plan_node_id())); - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group)); }; return Status::OK(); })); @@ -385,9 +390,10 @@ TEST_F(StreamOperatorsTest, binlog_dop_1) { } TEST_F(StreamOperatorsTest, binlog_dop_1_multi_epoch) { - ASSERT_IF_ERROR(start_mv([&]() { + ASSERT_IF_ERROR(start_mv([&](auto* stream_ctx) { + auto exec_group = stream_ctx->exec_group.get(); _degree_of_parallelism = 1; - _pipeline_builder = [&](RuntimeState* state) { + _pipeline_builder = [=](RuntimeState* state) { auto* descs = _create_table_desc(2, 4); auto tnode = _create_tplan_node(next_plan_node_id(), 0); auto binlog_scan_node = std::make_shared(_obj_pool, *tnode, *descs); @@ -402,7 +408,7 @@ TEST_F(StreamOperatorsTest, binlog_dop_1_multi_epoch) { op_factories.push_back( std::make_shared(next_operator_id(), next_plan_node_id())); - _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories)); + _pipelines.push_back(std::make_shared(next_pipeline_id(), op_factories, exec_group)); }; return Status::OK(); })); diff --git a/be/test/exec/stream/stream_pipeline_test.cpp b/be/test/exec/stream/stream_pipeline_test.cpp index b410da65cf0342..8e123108eed9fc 100644 --- a/be/test/exec/stream/stream_pipeline_test.cpp +++ b/be/test/exec/stream/stream_pipeline_test.cpp @@ -25,6 +25,7 @@ #include "exec/stream/stream_operators_test.h" #include "gtest/gtest.h" #include "runtime/exec_env.h" +#include "testutil/assert.h" #include "testutil/desc_tbl_helper.h" namespace starrocks::stream { @@ -75,28 +76,27 @@ Status StreamPipelineTest::prepare() { DCHECK(_pipeline_builder != nullptr); _pipelines.clear(); _pipeline_builder(_fragment_ctx->runtime_state()); - _fragment_ctx->set_pipelines(std::move(_pipelines)); + for (auto pipeline : _pipelines) { + exec_group->add_pipeline(std::move(pipeline)); + } + _fragment_ctx->set_exec_groups({exec_group}); + exec_group.reset(); + _pipelines.clear(); RETURN_IF_ERROR(_fragment_ctx->prepare_all_pipelines()); - const auto& pipelines = _fragment_ctx->pipelines(); - const size_t num_pipelines = pipelines.size(); - // morsel queue starrocks::pipeline::MorselQueueFactoryMap& morsel_queues = _fragment_ctx->morsel_queue_factories(); - for (const auto& pipeline : pipelines) { + _fragment_ctx->iterate_pipeline([&morsel_queues](auto pipeline) { if (pipeline->source_operator_factory()->with_morsels()) { - auto source_id = pipeline->get_op_factories()[0]->plan_node_id(); + auto source_id = pipeline->source_operator_factory()->plan_node_id(); DCHECK(morsel_queues.count(source_id)); auto& morsel_queue_factory = morsel_queues[source_id]; - pipeline->source_operator_factory()->set_morsel_queue_factory(morsel_queue_factory.get()); } - } + }); - for (auto n = 0; n < num_pipelines; ++n) { - const auto& pipeline = pipelines[n]; - pipeline->instantiate_drivers(_fragment_ctx->runtime_state()); - } + _fragment_ctx->iterate_pipeline( + [this](auto pipeline) { pipeline->instantiate_drivers(_fragment_ctx->runtime_state()); }); // prepare epoch manager auto stream_epoch_manager = _query_ctx->stream_epoch_manager(); @@ -108,20 +108,19 @@ Status StreamPipelineTest::prepare() { Status StreamPipelineTest::execute() { VLOG_ROW << "ExecutePipeline"; - Status prepare_status = _fragment_ctx->iterate_drivers( - [state = _fragment_ctx->runtime_state()](const DriverPtr& driver) { return driver->prepare(state); }); - DCHECK(prepare_status.ok()); - bool enable_resource_group = _fragment_ctx->enable_resource_group(); - CHECK(_fragment_ctx - ->iterate_drivers([exec_env = _exec_env, enable_resource_group](const DriverPtr& driver) { - exec_env->wg_driver_executor()->submit(driver.get()); - return Status::OK(); - }) - .ok()); + _fragment_ctx->iterate_drivers( + [state = _fragment_ctx->runtime_state()](const DriverPtr& driver) { CHECK_OK(driver->prepare(state)); }); + + // CHECK_OK(_fragment_ctx->submit_active_drivers(_exec_env->wg_driver_executor())); + _fragment_ctx->iterate_drivers([exec_env = _exec_env](const DriverPtr& driver) { + LOG(WARNING) << driver->to_readable_string(); + exec_env->wg_driver_executor()->submit(driver.get()); + }); return Status::OK(); } -OpFactories StreamPipelineTest::maybe_interpolate_local_passthrough_exchange(OpFactories& pred_operators) { +OpFactories StreamPipelineTest::maybe_interpolate_local_passthrough_exchange( + OpFactories& pred_operators, pipeline::ExecutionGroupRawPtr exec_group) { DCHECK(!pred_operators.empty() && pred_operators[0]->is_source()); auto* source_operator = down_cast(pred_operators[0].get()); if (source_operator->degree_of_parallelism() > 1) { @@ -140,7 +139,7 @@ OpFactories StreamPipelineTest::maybe_interpolate_local_passthrough_exchange(OpF // Add LocalExchangeSinkOperator to predecessor pipeline. pred_operators.emplace_back(std::move(local_exchange_sink)); // predecessor pipeline comes to end. - _pipelines.emplace_back(std::make_unique(next_pipeline_id(), pred_operators)); + _pipelines.emplace_back(std::make_unique(next_pipeline_id(), pred_operators, exec_group)); OpFactories operators_source_with_local_exchange; // Multiple LocalChangeSinkOperators pipe into one LocalChangeSourceOperator. @@ -154,7 +153,8 @@ OpFactories StreamPipelineTest::maybe_interpolate_local_passthrough_exchange(OpF } Status StreamPipelineTest::start_mv(InitiliazeFunc&& init_func) { - RETURN_IF_ERROR(init_func()); + exec_group = pipeline::ExecutionGroupBuilder::create_normal_exec_group(); + RETURN_IF_ERROR(init_func(this)); RETURN_IF_ERROR(prepare()); RETURN_IF_ERROR(execute()); return Status::OK(); @@ -203,7 +203,7 @@ Status StreamPipelineTest::wait_until_epoch_finished(const EpochInfo& epoch_info [query_id](const pipeline::PipelineDriver* driver) { return driver->query_ctx()->query_id() == query_id; }); - return num_parked_drivers == _fragment_ctx->num_drivers(); + return num_parked_drivers == _fragment_ctx->total_dop(); }; while (!are_all_drivers_parked_func()) { diff --git a/be/test/exec/stream/stream_pipeline_test.h b/be/test/exec/stream/stream_pipeline_test.h index ce8b2a0a9580a9..c190a283d9237f 100644 --- a/be/test/exec/stream/stream_pipeline_test.h +++ b/be/test/exec/stream/stream_pipeline_test.h @@ -20,6 +20,7 @@ #include "column/vectorized_fwd.h" #include "exec/pipeline/exchange/local_exchange.h" #include "exec/pipeline/fragment_context.h" +#include "exec/pipeline/group_execution/execution_group_fwd.h" #include "exec/pipeline/pipeline_builder.h" #include "exec/pipeline/stream_epoch_manager.h" #include "gtest/gtest.h" @@ -31,8 +32,8 @@ class ConnectorScanNode; } namespace starrocks::stream { - -using InitiliazeFunc = std::function; +class StreamPipelineTest; +using InitiliazeFunc = std::function; class StreamPipelineTest { public: @@ -54,7 +55,8 @@ class StreamPipelineTest { uint32_t next_pipeline_id() { return _pipeline_context->next_pipe_id(); } protected: - OpFactories maybe_interpolate_local_passthrough_exchange(OpFactories& pred_operators); + OpFactories maybe_interpolate_local_passthrough_exchange(OpFactories& pred_operators, + pipeline::ExecutionGroupRawPtr exec_group); ExecEnv* _exec_env = nullptr; pipeline::QueryContext* _query_ctx = nullptr; @@ -66,6 +68,7 @@ class StreamPipelineTest { // lambda used to init _pipelines std::function _pipeline_builder; pipeline::Pipelines _pipelines; + pipeline::ExecutionGroupPtr exec_group; std::vector _tablet_ids; std::shared_ptr _connector_node; size_t _degree_of_parallelism; @@ -79,20 +82,16 @@ template std::vector StreamPipelineTest::fetch_results(const EpochInfo& epoch_info) { VLOG_ROW << "FetchResults: " << epoch_info.debug_string(); std::vector result_chunks; - const auto& pipelines = _fragment_ctx->pipelines(); - for (auto& pipeline : pipelines) { - for (auto& driver : pipeline->drivers()) { - auto* sink_op = driver->sink_operator(); - if (auto* stream_sink_op = dynamic_cast(sink_op); stream_sink_op != nullptr) { - result_chunks = stream_sink_op->output_chunks(); - for (auto& chunk : result_chunks) { - VLOG_ROW << "FetchResults, result: " << chunk->debug_columns(); - } - CHECK(stream_sink_op->reset_epoch(nullptr).ok()); - break; + _fragment_ctx->iterate_drivers([this, &result_chunks](auto driver) { + auto* sink_op = driver->sink_operator(); + if (auto* stream_sink_op = dynamic_cast(sink_op); stream_sink_op != nullptr) { + result_chunks = stream_sink_op->output_chunks(); + for (auto& chunk : result_chunks) { + VLOG_ROW << "FetchResults, result: " << chunk->debug_columns(); } + CHECK(stream_sink_op->reset_epoch(nullptr).ok()); } - } + }); return result_chunks; } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/AggregationNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/AggregationNode.java index a5641e4bcb8c25..7375358fca67f2 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/AggregationNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/AggregationNode.java @@ -90,7 +90,7 @@ public class AggregationNode extends PlanNode { private boolean useSortAgg = false; private boolean usePerBucketOptimize = false; - + private boolean withLocalShuffle = false; // identicallyDistributed meanings the PlanNode above OlapScanNode are cases as follows: @@ -346,8 +346,10 @@ public Optional> candidatesOfSlotExpr(Expr expr, Function partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); + DescriptorTable descTbl = context.getDescTbl(); if (!canPushDownRuntimeFilter()) { return false; } @@ -357,7 +359,7 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc } Function couldBoundChecker = couldBound(description, descTbl); - return pushdownRuntimeFilterForChildOrAccept(descTbl, description, probeExpr, + return pushdownRuntimeFilterForChildOrAccept(context, probeExpr, candidatesOfSlotExpr(probeExpr, couldBoundChecker), partitionByExprs, candidatesOfSlotExprs(partitionByExprs, couldBoundForPartitionExpr()), 0, true); } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/AnalyticEvalNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/AnalyticEvalNode.java index 2f4c399f12a518..8de2187403e186 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/AnalyticEvalNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/AnalyticEvalNode.java @@ -312,8 +312,10 @@ public Optional> candidatesOfSlotExpr(Expr expr, Function partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); + DescriptorTable descTbl = context.getDescTbl(); if (!canPushDownRuntimeFilter()) { return false; } @@ -322,7 +324,7 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc return false; } - return pushdownRuntimeFilterForChildOrAccept(descTbl, description, probeExpr, + return pushdownRuntimeFilterForChildOrAccept(context, probeExpr, candidatesOfSlotExpr(probeExpr, couldBound(description, descTbl)), partitionByExprs, candidatesOfSlotExprs(partitionByExprs, couldBoundForPartitionExpr()), 0, true); } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/DecodeNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/DecodeNode.java index 33594c5862a6b8..d7873896d17520 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/DecodeNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/DecodeNode.java @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - package com.starrocks.planner; import com.google.common.collect.Lists; @@ -53,7 +52,7 @@ public DecodeNode(PlanNodeId id, Map dictIdToStringIds, Map stringFunctions, Map slotRefMap - ) { + ) { super(id, tupleDescriptor.getId().asList(), "Decode"); addChild(child); this.dictIdToStringIds = dictIdToStringIds; @@ -113,9 +112,11 @@ public Optional> candidatesOfSlotExpr(Expr expr, Function partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); + DescriptorTable descTbl = context.getDescTbl(); if (!canPushDownRuntimeFilter()) { return false; } @@ -124,7 +125,8 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc return false; } - return pushdownRuntimeFilterForChildOrAccept(descTbl, description, probeExpr, candidatesOfSlotExpr(probeExpr, couldBound(description, descTbl)), + return pushdownRuntimeFilterForChildOrAccept(context, probeExpr, + candidatesOfSlotExpr(probeExpr, couldBound(description, descTbl)), partitionByExprs, candidatesOfSlotExprs(partitionByExprs, couldBoundForPartitionExpr()), 0, true); } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/ExchangeNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/ExchangeNode.java index 02dcf007d79483..013729bcbe1f26 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/ExchangeNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/ExchangeNode.java @@ -39,7 +39,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.starrocks.analysis.Analyzer; -import com.starrocks.analysis.DescriptorTable; import com.starrocks.analysis.Expr; import com.starrocks.analysis.SlotRef; import com.starrocks.analysis.SortInfo; @@ -244,12 +243,13 @@ public boolean canUseRuntimeAdaptiveDop() { } @Override - public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDescription description, Expr probeExpr, + public boolean pushDownRuntimeFilters(RuntimeFilterPushDownContext context, Expr probeExpr, List partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); if (!canPushDownRuntimeFilter()) { return false; } - boolean accept = pushCrossExchange(descTbl, description, probeExpr, partitionByExprs); + boolean accept = pushCrossExchange(context, probeExpr, partitionByExprs); // Add the rf onto ExchangeNode if it can not be pushed down to Exchange's offsprings or // session variable runtime_filter_on_exchange_node is true(in default is false). boolean onExchangeNode = (!accept || ConnectContext.get().getSessionVariable().isRuntimeFilterOnExchangeNode()); @@ -263,7 +263,7 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc // we enable this only when: // - session variable enabled & // - this rf has been accepted by children nodes(global rf). - if (probeExpr.isBoundByTupleIds(getTupleIds()) && description.canAcceptFilter(this)) { + if (probeExpr.isBoundByTupleIds(getTupleIds()) && description.canAcceptFilter(this, context)) { if (onExchangeNode || (description.isLocalApplicable() && description.inLocalFragmentInstance())) { description.addProbeExpr(id.asInt(), probeExpr); description.addPartitionByExprsIfNeeded(id.asInt(), probeExpr, partitionByExprs); @@ -274,9 +274,9 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc return accept; } - private boolean pushCrossExchange(DescriptorTable descTbl, - RuntimeFilterDescription description, Expr probeExpr, + private boolean pushCrossExchange(RuntimeFilterPushDownContext context, Expr probeExpr, List partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); if (!description.canPushAcrossExchangeNode()) { return false; } @@ -310,7 +310,7 @@ private boolean pushCrossExchange(DescriptorTable descTbl, boolean accept = false; description.enterExchangeNode(); for (PlanNode node : children) { - if (node.pushDownRuntimeFilters(descTbl, description, probeExpr, partitionByExprs)) { + if (node.pushDownRuntimeFilters(context, probeExpr, partitionByExprs)) { description.setHasRemoteTargets(true); accept = true; } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroup.java b/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroup.java new file mode 100644 index 00000000000000..6b3800852b8132 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroup.java @@ -0,0 +1,98 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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. + +package com.starrocks.planner; + +import com.google.api.client.util.Sets; +import com.google.common.base.Preconditions; +import com.starrocks.thrift.TExecGroup; + +import java.util.Set; + +public class ExecGroup { + private enum Type { + NORMAL, + COLOCATE + } + + public ExecGroup(ExecGroupId groupId) { + this.groupId = groupId; + } + + public void setColocateGroup() { + type = Type.COLOCATE; + } + + public boolean isColocateExecGroup() { + return type == Type.COLOCATE && !disableColocateGroup; + } + public boolean isDisableColocateGroup() { + return this.disableColocateGroup; + } + public void setDisableColocateGroup() { + this.disableColocateGroup = true; + } + + public void add(PlanNode node) { + nodeIds.add(node.getId().asInt()); + } + + public void add(PlanNode node, boolean disableColocateGroup) { + add(node); + this.disableColocateGroup = this.disableColocateGroup || disableColocateGroup; + } + + public void merge(ExecGroup other) { + Preconditions.checkState(isColocateExecGroup()); + Preconditions.checkState(!other.disableColocateGroup); + if (this != other) { + this.nodeIds.addAll(other.nodeIds); + } + } + + public boolean contains(PlanNode node) { + return nodeIds.contains(node.getId().asInt()); + } + + public boolean contains(int nodeId) { + return nodeIds.contains(nodeId); + } + + public ExecGroupId getGroupId() { + return groupId; + } + + @Override + public String toString() { + return "ExecGroup{" + + "groupId=" + groupId + + ", nodeIds=" + nodeIds + + '}'; + } + + public TExecGroup toThrift() { + TExecGroup tExecGroup = new TExecGroup(); + Preconditions.checkState(isColocateExecGroup()); + tExecGroup.setGroup_id(groupId.asInt()); + for (Integer nodeId : nodeIds) { + tExecGroup.addToPlan_node_ids(nodeId); + } + return tExecGroup; + } + + private final ExecGroupId groupId; + private boolean disableColocateGroup = false; + private final Set nodeIds = Sets.newHashSet(); + private Type type = Type.NORMAL; +} diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroupId.java b/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroupId.java new file mode 100644 index 00000000000000..28aa0d56cbb1ea --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroupId.java @@ -0,0 +1,38 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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. + +package com.starrocks.planner; + +import com.starrocks.common.Id; +import com.starrocks.common.IdGenerator; + +public class ExecGroupId extends Id { + public ExecGroupId(int id) { + super(id); + } + + public static IdGenerator createGenerator() { + return new IdGenerator<>() { + @Override + public ExecGroupId getNextId() { + return new ExecGroupId(nextId++); + } + + @Override + public ExecGroupId getMaxId() { + return new ExecGroupId(nextId - 1); + } + }; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroupSets.java b/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroupSets.java new file mode 100644 index 00000000000000..a1f6c113811e1e --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/planner/ExecGroupSets.java @@ -0,0 +1,51 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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. + +package com.starrocks.planner; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.starrocks.common.IdGenerator; + +import java.util.List; + +// A collection of ExecGroups. +public class ExecGroupSets { + private final IdGenerator execGroupIdIdGenerator = ExecGroupId.createGenerator(); + private final List execGroups = Lists.newArrayList(); + + public List getExecGroups() { + return execGroups; + } + + public ExecGroup newExecGroup() { + ExecGroup execGroup = new ExecGroup(execGroupIdIdGenerator.getNextId()); + execGroups.add(execGroup); + return execGroup; + } + + public void remove(ExecGroup execGroup) { + execGroups.remove(execGroup); + } + + public ExecGroup getExecGroup(int nodeId) { + for (ExecGroup execGroup : execGroups) { + if (execGroup.contains(nodeId)) { + return execGroup; + } + } + Preconditions.checkState(false, "not found exec group node: %d", nodeId); + return null; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/JoinNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/JoinNode.java index c131b4fe6100b1..7a3503c806086d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/JoinNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/JoinNode.java @@ -178,7 +178,7 @@ public List getProbePartitionByExprs() { } @Override - public void buildRuntimeFilters(IdGenerator runtimeFilterIdIdGenerator, DescriptorTable descTbl) { + public void buildRuntimeFilters(IdGenerator runtimeFilterIdIdGenerator, DescriptorTable descTbl, ExecGroupSets execGroupSets) { SessionVariable sessionVariable = ConnectContext.get().getSessionVariable(); JoinOperator joinOp = getJoinOp(); PlanNode inner = getChild(1); @@ -227,7 +227,9 @@ public void buildRuntimeFilters(IdGenerator runtimeFilterIdIdGe // push down rf to left child node, and build it only when it // can be accepted by left child node. rf.setBuildExpr(left); - if (getChild(0).pushDownRuntimeFilters(descTbl, rf, right, probePartitionByExprs)) { + RuntimeFilterPushDownContext rfPushDownCxt = + new RuntimeFilterPushDownContext(rf, descTbl, execGroupSets); + if (getChild(0).pushDownRuntimeFilters(rfPushDownCxt, right, probePartitionByExprs)) { buildRuntimeFilters.add(rf); } } else { @@ -243,7 +245,9 @@ public void buildRuntimeFilters(IdGenerator runtimeFilterIdIdGe rf.setFilterId(runtimeFilterIdIdGenerator.getNextId().asInt()); rf.setBuildExpr(right); rf.setOnlyLocal(true); - if (getChild(0).pushDownRuntimeFilters(descTbl, rf, left, probePartitionByExprs)) { + RuntimeFilterPushDownContext rfPushDownCxt = + new RuntimeFilterPushDownContext(rf, descTbl, execGroupSets); + if (getChild(0).pushDownRuntimeFilters(rfPushDownCxt, left, probePartitionByExprs)) { this.getBuildRuntimeFilters().add(rf); } } @@ -285,17 +289,18 @@ public Optional>> candidatesOfSlotExprsForChild(List exprs return Optional.of(candidateOfPartitionByExprs(candidatesOfSlotExprs)); } - public boolean pushDownRuntimeFiltersForChild(DescriptorTable descTbl, RuntimeFilterDescription description, + public boolean pushDownRuntimeFiltersForChild(RuntimeFilterPushDownContext context, Expr probeExpr, List partitionByExprs, int childIdx) { - return pushdownRuntimeFilterForChildOrAccept(descTbl, description, probeExpr, + return pushdownRuntimeFilterForChildOrAccept(context, probeExpr, candidatesOfSlotExprForChild(probeExpr, childIdx), partitionByExprs, candidatesOfSlotExprsForChild(partitionByExprs, childIdx), childIdx, false); } @Override - public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDescription description, Expr probeExpr, + public boolean pushDownRuntimeFilters(RuntimeFilterPushDownContext context, Expr probeExpr, List partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); if (!canPushDownRuntimeFilter()) { return false; } @@ -307,17 +312,17 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc // SlotRef(b) are equivalent. boolean isInnerOrSemiJoin = joinOp.isSemiJoin() || joinOp.isInnerJoin(); if ((probeExpr instanceof SlotRef) && isInnerOrSemiJoin) { - hasPushedDown |= pushDownRuntimeFiltersForChild(descTbl, description, probeExpr, partitionByExprs, 0); - hasPushedDown |= pushDownRuntimeFiltersForChild(descTbl, description, probeExpr, partitionByExprs, 1); + hasPushedDown |= pushDownRuntimeFiltersForChild(context, probeExpr, partitionByExprs, 0); + hasPushedDown |= pushDownRuntimeFiltersForChild(context, probeExpr, partitionByExprs, 1); } // fall back to PlanNode.pushDownRuntimeFilters for HJ if rf cannot be pushed down via equivalent // equalJoinConjuncts - if (hasPushedDown || super.pushDownRuntimeFilters(descTbl, description, probeExpr, partitionByExprs)) { + if (hasPushedDown || super.pushDownRuntimeFilters(context, probeExpr, partitionByExprs)) { return true; } // use runtime filter at this level if rf can not be pushed down to children. - if (description.canProbeUse(this)) { + if (description.canProbeUse(this, context)) { description.addProbeExpr(id.asInt(), probeExpr); description.addPartitionByExprsIfNeeded(id.asInt(), probeExpr, partitionByExprs); probeRuntimeFilters.add(description); diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/NestLoopJoinNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/NestLoopJoinNode.java index 09049e8aa4a7d0..6e6a29e3164687 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/NestLoopJoinNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/NestLoopJoinNode.java @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - package com.starrocks.planner; import com.google.common.base.Preconditions; @@ -55,7 +54,8 @@ public NestLoopJoinNode(PlanNodeId id, PlanNode outer, PlanNode inner, TableRef * Build the filter if inner table contains only one row, which is a common case for scalar subquery */ @Override - public void buildRuntimeFilters(IdGenerator generator, DescriptorTable descTbl) { + public void buildRuntimeFilters(IdGenerator generator, DescriptorTable descTbl, + ExecGroupSets execGroupSets) { if (!joinOp.isInnerJoin() && !joinOp.isLeftSemiJoin() && !joinOp.isRightJoin() && !joinOp.isCrossJoin()) { return; } @@ -78,7 +78,10 @@ public void buildRuntimeFilters(IdGenerator generator, Descript rf.setOnlyLocal(true); rf.setBuildExpr(right); - if (getChild(0).pushDownRuntimeFilters(descTbl, rf, left, probePartitionByExprs)) { + RuntimeFilterPushDownContext rfPushDownCtx = + new RuntimeFilterPushDownContext(rf, descTbl, execGroupSets); + + if (getChild(0).pushDownRuntimeFilters(rfPushDownCtx, left, probePartitionByExprs)) { this.getBuildRuntimeFilters().add(rf); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/PlanFragment.java b/fe/fe-core/src/main/java/com/starrocks/planner/PlanFragment.java index 2a82139b3b4681..cbe947482a81a5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/PlanFragment.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/PlanFragment.java @@ -51,6 +51,7 @@ import com.starrocks.thrift.TExplainLevel; import com.starrocks.thrift.TExpr; import com.starrocks.thrift.TGlobalDict; +import com.starrocks.thrift.TGroupExecutionParam; import com.starrocks.thrift.TNetworkAddress; import com.starrocks.thrift.TPartitionType; import com.starrocks.thrift.TPlanFragment; @@ -175,6 +176,9 @@ public class PlanFragment extends TreeNode { private boolean isShortCircuit = false; + // Controls whether group execution is used for plan fragment execution. + private List colocateExecGroups = Lists.newArrayList(); + /** * C'tor for fragment with specific partition; the output is by default broadcast. */ @@ -340,6 +344,10 @@ public void setWithLocalShuffleIfTrue(boolean withLocalShuffle) { this.withLocalShuffle |= withLocalShuffle; } + public boolean isUseGroupExecution() { + return !colocateExecGroups.isEmpty(); + } + public boolean isAssignScanRangesPerDriverSeq() { return assignScanRangesPerDriverSeq; } @@ -375,6 +383,23 @@ public void computeLocalRfWaitingSet(PlanNode root, boolean clearGlobalRuntimeFi } } + public void assignColocateExecGroups(PlanNode root, List groups) { + for (ExecGroup group : groups) { + if (group.contains(root)) { + colocateExecGroups.add(group); + groups.remove(group); + break; + } + } + if (!groups.isEmpty()) { + for (PlanNode child : root.getChildren()) { + if (child.getFragment() == this) { + assignColocateExecGroups(child, groups); + } + } + } + } + public boolean isDopEstimated() { return dopEstimated; } @@ -447,6 +472,15 @@ public TPlanFragment toThrift() { } result.setCache_param(cacheParam); } + + if (!colocateExecGroups.isEmpty()) { + TGroupExecutionParam tGroupExecutionParam = new TGroupExecutionParam(); + tGroupExecutionParam.setEnable_group_execution(true); + for (ExecGroup colocateExecGroup : colocateExecGroups) { + tGroupExecutionParam.addToExec_groups(colocateExecGroup.toThrift()); + } + result.setGroup_execution_param(tGroupExecutionParam); + } return result; } @@ -517,9 +551,15 @@ public String getExplainString(TExplainLevel explainLevel) { .collect(Collectors.joining(" | "))); } - str.append(outputBuilder); str.append("\n"); + if (!colocateExecGroups.isEmpty()) { + str.append(" colocate exec groups: "); + for (ExecGroup group : colocateExecGroups) { + str.append(group); + } + str.append("\n"); + } str.append(" PARTITION: ").append(dataPartition.getExplainString(explainLevel)).append("\n"); if (sink != null) { str.append(sink.getExplainString(" ", explainLevel)).append("\n"); @@ -542,6 +582,13 @@ public String getVerboseExplain() { .collect(Collectors.joining(" | "))); } str.append("\n"); + if (!colocateExecGroups.isEmpty()) { + str.append(" colocate exec groups: "); + for (ExecGroup group : colocateExecGroups) { + str.append(group); + } + str.append("\n"); + } str.append(" Input Partition: ").append(dataPartition.getExplainString(TExplainLevel.NORMAL)); if (sink != null) { str.append(sink.getVerboseExplain(" ")).append("\n"); @@ -833,6 +880,7 @@ public void reset() { } public void disablePhysicalPropertyOptimize() { + colocateExecGroups.clear(); forEachNode(planRoot, PlanNode::disablePhysicalPropertyOptimize); } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/PlanNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/PlanNode.java index 02db64d4d11200..119e5c8ed9083b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/PlanNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/PlanNode.java @@ -788,8 +788,10 @@ public Optional>> canPushDownRuntimeFilterCrossExchange(List partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); + DescriptorTable descTbl = context.getDescTbl(); if (!canPushDownRuntimeFilter()) { return false; } @@ -805,13 +807,13 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc boolean accept = false; for (PlanNode node : children) { if (candidatePartitionByExprs.isEmpty()) { - if (node.pushDownRuntimeFilters(descTbl, description, probeExpr, Lists.newArrayList())) { + if (node.pushDownRuntimeFilters(context, probeExpr, Lists.newArrayList())) { accept = true; break; } } else { for (List candidateOfPartitionByExprs : candidatePartitionByExprs) { - if (node.pushDownRuntimeFilters(descTbl, description, probeExpr, candidateOfPartitionByExprs)) { + if (node.pushDownRuntimeFilters(context, probeExpr, candidateOfPartitionByExprs)) { accept = true; break; } @@ -826,7 +828,7 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc if (accept) { return true; } - if (isBound && description.canProbeUse(this)) { + if (isBound && description.canProbeUse(this, context)) { description.addProbeExpr(id.asInt(), probeExpr); description.addPartitionByExprsIfNeeded(id.asInt(), probeExpr, partitionByExprs); probeRuntimeFilters.add(description); @@ -887,11 +889,12 @@ protected boolean canEliminateNull(Expr expr, SlotDescriptor slot) { } return false; } + protected boolean canEliminateNull(SlotDescriptor slot) { return conjuncts.stream().anyMatch(expr -> canEliminateNull(expr, slot)); } - private boolean tryPushdownRuntimeFilterToChild(DescriptorTable descTbl, RuntimeFilterDescription description, + private boolean tryPushdownRuntimeFilterToChild(RuntimeFilterPushDownContext context, Optional> optProbeExprCandidates, Optional>> optPartitionByExprsCandidates, int childIdx) { @@ -903,14 +906,14 @@ private boolean tryPushdownRuntimeFilterToChild(DescriptorTable descTbl, Runtime for (Expr candidateOfProbeExpr : probeExprCandidates) { if (partitionByExprsCandidates.isEmpty()) { - if (children.get(childIdx).pushDownRuntimeFilters(descTbl, description, candidateOfProbeExpr, + if (children.get(childIdx).pushDownRuntimeFilters(context, candidateOfProbeExpr, Lists.newArrayList())) { return true; } } else { for (List candidateOfPartitionByExprs : partitionByExprsCandidates) { if (children.get(childIdx) - .pushDownRuntimeFilters(descTbl, description, candidateOfProbeExpr, + .pushDownRuntimeFilters(context, candidateOfProbeExpr, candidateOfPartitionByExprs)) { return true; } @@ -924,15 +927,16 @@ private boolean tryPushdownRuntimeFilterToChild(DescriptorTable descTbl, Runtime * Push down a runtime filter for the specific child with childIdx. `addProbeInfo` indicates whether * add runtime filter info into this PlanNode. */ - protected boolean pushdownRuntimeFilterForChildOrAccept(DescriptorTable descTbl, - RuntimeFilterDescription description, + protected boolean pushdownRuntimeFilterForChildOrAccept(RuntimeFilterPushDownContext context, Expr probeExpr, Optional> optProbeExprCandidates, List partitionByExprs, Optional>> optPartitionByExprsCandidates, int childIdx, boolean addProbeInfo) { - boolean accept = tryPushdownRuntimeFilterToChild(descTbl, description, optProbeExprCandidates, + RuntimeFilterDescription description = context.getDescription(); + DescriptorTable descTbl = context.getDescTbl(); + boolean accept = tryPushdownRuntimeFilterToChild(context, optProbeExprCandidates, optPartitionByExprsCandidates, childIdx); RoaringBitmap slotIds = getSlotIds(descTbl); boolean isBound = slotIds.contains(probeExpr.getUsedSlotIds()) && @@ -943,7 +947,7 @@ protected boolean pushdownRuntimeFilterForChildOrAccept(DescriptorTable descTbl, if (accept) { return true; } - if (isBound && addProbeInfo && description.canProbeUse(this)) { + if (isBound && addProbeInfo && description.canProbeUse(this, context)) { // can not push down to children. // use runtime filter at this level. description.addProbeExpr(id.asInt(), probeExpr); diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/ProjectNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/ProjectNode.java index ae61858a8bd92b..b7428d6dfdf998 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/ProjectNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/ProjectNode.java @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - package com.starrocks.planner; import com.google.common.base.Preconditions; @@ -147,7 +146,6 @@ public Optional>> candidatesOfSlotExprs(List exprs, Functi return Optional.of(candidateOfPartitionByExprs(candidatesOfSlotExprs)); } - @Override public Optional> candidatesOfSlotExpr(Expr expr, Function couldBound) { if (!(expr instanceof SlotRef)) { @@ -169,10 +167,13 @@ public Optional> candidatesOfSlotExpr(Expr expr, Function 0 ? Optional.of(newExprs) : Optional.empty(); } + @Override - public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDescription description, + public boolean pushDownRuntimeFilters(RuntimeFilterPushDownContext context, Expr probeExpr, List partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); + DescriptorTable descTbl = context.getDescTbl(); if (!canPushDownRuntimeFilter()) { return false; } @@ -181,7 +182,8 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc return false; } - return pushdownRuntimeFilterForChildOrAccept(descTbl, description, probeExpr, candidatesOfSlotExpr(probeExpr, couldBound(description, descTbl)), + return pushdownRuntimeFilterForChildOrAccept(context, probeExpr, + candidatesOfSlotExpr(probeExpr, couldBound(description, descTbl)), partitionByExprs, candidatesOfSlotExprs(partitionByExprs, couldBoundForPartitionExpr()), 0, true); } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterBuildNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterBuildNode.java index 15cbafb46f5d0e..1a16d905bf8141 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterBuildNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterBuildNode.java @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - package com.starrocks.planner; import com.starrocks.analysis.DescriptorTable; @@ -23,7 +22,8 @@ public interface RuntimeFilterBuildNode { List getBuildRuntimeFilters(); - void buildRuntimeFilters(IdGenerator runtimeFilterIdIdGenerator, DescriptorTable descTbl); + void buildRuntimeFilters(IdGenerator runtimeFilterIdIdGenerator, DescriptorTable descTbl, + ExecGroupSets execGroupSets); void clearBuildRuntimeFilters(); } diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterDescription.java b/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterDescription.java index 06daeb4d89ea52..c5d9ddc61881c2 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterDescription.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterDescription.java @@ -30,11 +30,9 @@ import com.starrocks.thrift.TUniqueId; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -77,6 +75,10 @@ public enum RuntimeFilterType { private boolean onlyLocal; + // ExecGroupInfo. used for check build colocate runtime filter + private boolean isBuildFromColocateGroup = false; + private int execGroupId = -1; + private RuntimeFilterType type; int numInstances; @@ -147,8 +149,8 @@ public void setSortInfo(SortInfo sortInfo) { this.sortInfo = sortInfo; } - public boolean canProbeUse(PlanNode node) { - if (!canAcceptFilter(node)) { + public boolean canProbeUse(PlanNode node, RuntimeFilterPushDownContext rfPushCtx) { + if (!canAcceptFilter(node, rfPushCtx)) { return false; } if (RuntimeFilterType.TOPN_FILTER.equals(runtimeFilterType()) && node instanceof OlapScanNode) { @@ -185,7 +187,7 @@ public boolean canProbeUse(PlanNode node) { } // return true if Node could accept the Filter - public boolean canAcceptFilter(PlanNode node) { + public boolean canAcceptFilter(PlanNode node, RuntimeFilterPushDownContext rfPushCtx) { if (RuntimeFilterType.TOPN_FILTER.equals(runtimeFilterType())) { if (node instanceof ScanNode) { ScanNode scanNode = (ScanNode) node; @@ -194,6 +196,12 @@ public boolean canAcceptFilter(PlanNode node) { return false; } } + if (isBuildFromColocateGroup) { + int probeExecGroupId = rfPushCtx.getExecGroup(node.getId().asInt()).getGroupId().asInt(); + if (execGroupId != probeExecGroupId) { + return false; + } + } return true; } @@ -323,18 +331,24 @@ public List getBucketSeqToPartition() { public void setNumInstances(int numInstances) { this.numInstances = numInstances; } + public int getNumInstances() { return numInstances; } - public void setNumDriversPerInstance(int numDriversPerInstance){ + public void setNumDriversPerInstance(int numDriversPerInstance) { this.numDriversPerInstance = numDriversPerInstance; } - public int getNumDriversPerInstance(){ + public int getNumDriversPerInstance() { return numDriversPerInstance; } + public void setExecGroupInfo(boolean buildFromColocateGroup, int buildExecGroupId) { + this.isBuildFromColocateGroup = buildFromColocateGroup; + this.execGroupId = buildExecGroupId; + } + public boolean canPushAcrossExchangeNode() { if (onlyLocal) { return false; diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterPushDownContext.java b/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterPushDownContext.java new file mode 100644 index 00000000000000..d6e6085f46cb74 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/planner/RuntimeFilterPushDownContext.java @@ -0,0 +1,45 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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. + +package com.starrocks.planner; + +import com.starrocks.analysis.DescriptorTable; + +public class RuntimeFilterPushDownContext { + private final RuntimeFilterDescription description; + private final DescriptorTable descTbl; + private final ExecGroupSets execGroups; + + RuntimeFilterPushDownContext(RuntimeFilterDescription description, DescriptorTable descTbl, ExecGroupSets execGroupSets) { + this.description = description; + this.descTbl = descTbl; + this.execGroups = execGroupSets; + // set description + ExecGroup execGroup = this.execGroups.getExecGroup(description.getBuildPlanNodeId()); + this.description.setExecGroupInfo(execGroup.isColocateExecGroup(), execGroup.getGroupId().asInt()); + } + + public DescriptorTable getDescTbl() { + return descTbl; + } + + public RuntimeFilterDescription getDescription() { + return description; + } + + public ExecGroup getExecGroup(int planNodeId) { + return this.execGroups.getExecGroup(planNodeId); + } + +} diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/SetOperationNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/SetOperationNode.java index 39ca6876914ea6..b805f77fde301b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/SetOperationNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/SetOperationNode.java @@ -40,7 +40,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.starrocks.analysis.Analyzer; -import com.starrocks.analysis.DescriptorTable; import com.starrocks.analysis.Expr; import com.starrocks.analysis.SlotDescriptor; import com.starrocks.analysis.SlotId; @@ -308,7 +307,8 @@ public Optional>> candidatesOfSlotExprsForChild(List exprs } @Override - public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDescription description, Expr probeExpr, List partitionByExprs) { + public boolean pushDownRuntimeFilters(RuntimeFilterPushDownContext context, Expr probeExpr, List partitionByExprs) { + RuntimeFilterDescription description = context.getDescription(); if (!canPushDownRuntimeFilter()) { return false; } @@ -321,7 +321,7 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc boolean pushDown = false; // try to push all children if any expr of a child can match `probeExpr` for (int i = 0; i < materializedResultExprLists_.size(); i++) { - pushDown |= pushdownRuntimeFilterForChildOrAccept(descTbl, description, probeExpr, + pushDown |= pushdownRuntimeFilterForChildOrAccept(context, probeExpr, candidatesOfSlotExprForChild(probeExpr, i), partitionByExprs, candidatesOfSlotExprsForChild(partitionByExprs, i), i, false); } @@ -330,7 +330,7 @@ public boolean pushDownRuntimeFilters(DescriptorTable descTbl, RuntimeFilterDesc } } - if (description.canProbeUse(this)) { + if (description.canProbeUse(this, context)) { // can not push down to children. // use runtime filter at this level. description.addProbeExpr(id.asInt(), probeExpr); diff --git a/fe/fe-core/src/main/java/com/starrocks/planner/SortNode.java b/fe/fe-core/src/main/java/com/starrocks/planner/SortNode.java index 9f14c3f0596607..d4d7ac84e448c7 100644 --- a/fe/fe-core/src/main/java/com/starrocks/planner/SortNode.java +++ b/fe/fe-core/src/main/java/com/starrocks/planner/SortNode.java @@ -145,7 +145,8 @@ public List getBuildRuntimeFilters() { } @Override - public void buildRuntimeFilters(IdGenerator generator, DescriptorTable descTbl) { + public void buildRuntimeFilters(IdGenerator generator, DescriptorTable descTbl, + ExecGroupSets execGroupSets) { SessionVariable sessionVariable = ConnectContext.get().getSessionVariable(); // only support the runtime filter in TopN when limit > 0 if (limit < 0 || !sessionVariable.getEnableTopNRuntimeFilter() || @@ -165,9 +166,9 @@ public void buildRuntimeFilters(IdGenerator generator, Descript rf.setSortInfo(getSortInfo()); rf.setBuildExpr(orderBy); rf.setRuntimeFilterType(RuntimeFilterDescription.RuntimeFilterType.TOPN_FILTER); - + RuntimeFilterPushDownContext rfPushDownCtx = new RuntimeFilterPushDownContext(rf, descTbl, execGroupSets); for (PlanNode child : children) { - if (child.pushDownRuntimeFilters(descTbl, rf, orderBy, Lists.newArrayList())) { + if (child.pushDownRuntimeFilters(rfPushDownCtx, orderBy, Lists.newArrayList())) { this.buildRuntimeFilters.add(rf); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java b/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java index 86aa4e28cd7d34..9b4b1ea50a7dbe 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/SessionVariable.java @@ -428,6 +428,10 @@ public class SessionVariable implements Serializable, Writable, Cloneable { public static final String ENABLE_SORT_AGGREGATE = "enable_sort_aggregate"; public static final String ENABLE_PER_BUCKET_OPTIMIZE = "enable_per_bucket_optimize"; public static final String ENABLE_PARTITION_BUCKET_OPTIMIZE = "enable_partition_bucket_optimize"; + public static final String ENABLE_GROUP_EXECUTION = "enable_group_execution"; + public static final String GROUP_EXECUTION_GROUP_SCALE = "group_execution_group_scale"; + public static final String GROUP_EXECUTION_MAX_GROUPS = "group_execution_max_groups"; + public static final String ENABLE_PARALLEL_MERGE = "enable_parallel_merge"; public static final String ENABLE_QUERY_QUEUE = "enable_query_queue"; @@ -1290,6 +1294,15 @@ public static MaterializedViewRewriteMode parse(String str) { @VarAttr(name = ENABLE_PARTITION_BUCKET_OPTIMIZE, flag = VariableMgr.INVISIBLE) private boolean enablePartitionBucketOptimize = false; + + @VarAttr(name = ENABLE_GROUP_EXECUTION) + private boolean enableGroupExecution = false; + + @VarAttr(name = GROUP_EXECUTION_GROUP_SCALE) + private int groupExecutionGroupScale = 64; + + @VarAttr(name = GROUP_EXECUTION_MAX_GROUPS) + private int groupExecutionMaxGroups = 128; @VarAttr(name = ENABLE_PARALLEL_MERGE) private boolean enableParallelMerge = true; @@ -1399,6 +1412,22 @@ public boolean isEnablePartitionBucketOptimize() { public void setEnablePartitionBucketOptimize(boolean enablePartitionBucketOptimize) { this.enablePartitionBucketOptimize = enablePartitionBucketOptimize; } + + public void setEnableGroupExecution(boolean enableGroupExecution) { + this.enableGroupExecution = enableGroupExecution; + } + + public boolean isEnableGroupExecution() { + return enableGroupExecution; + } + + public int getGroupExecutionGroupScale() { + return groupExecutionGroupScale; + } + + public int getGroupExecutionMaxGroups() { + return groupExecutionMaxGroups; + } public int getWindowPartitionMode() { return windowPartitionMode; diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/assignment/LocalFragmentAssignmentStrategy.java b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/assignment/LocalFragmentAssignmentStrategy.java index 0c1b6f45e6d12f..061832d680cf61 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/assignment/LocalFragmentAssignmentStrategy.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/scheduler/assignment/LocalFragmentAssignmentStrategy.java @@ -23,6 +23,7 @@ import com.starrocks.qe.BackendSelector; import com.starrocks.qe.ColocatedBackendSelector; import com.starrocks.qe.ConnectContext; +import com.starrocks.qe.SessionVariable; import com.starrocks.qe.scheduler.WorkerProvider; import com.starrocks.qe.scheduler.dag.ExecutionFragment; import com.starrocks.qe.scheduler.dag.FragmentInstance; @@ -52,7 +53,8 @@ public class LocalFragmentAssignmentStrategy implements FragmentAssignmentStrate private final Set replicatedScanIds = Sets.newHashSet(); - public LocalFragmentAssignmentStrategy(ConnectContext connectContext, WorkerProvider workerProvider, boolean usePipeline, + public LocalFragmentAssignmentStrategy(ConnectContext connectContext, WorkerProvider workerProvider, + boolean usePipeline, boolean isLoadType) { this.connectContext = connectContext; this.workerProvider = workerProvider; @@ -190,9 +192,19 @@ private void assignScanRangesToColocateFragmentInstancePerWorker( }); } else { int expectedDop = Math.max(1, pipelineDop); - List> bucketSeqsPerDriverSeq = ListUtil.splitBySize(bucketSeqsOfInstance, expectedDop); - - instance.setPipelineDop(bucketSeqsPerDriverSeq.size()); + int expectedPhysicalDop = Math.min(expectedDop, bucketSeqsOfInstance.size()); + // For disable group execution logical dop == physical dop + // For enable group execution logical dop >= physical dop + int logicalDop = expectedPhysicalDop; + if (fragment.isUseGroupExecution()) { + // if fragment using group execution + SessionVariable sv = ConnectContext.get().getSessionVariable(); + int maxDop = Math.min(sv.getGroupExecutionGroupScale() * expectedDop, + sv.getGroupExecutionMaxGroups()); + logicalDop = Math.min(bucketSeqsOfInstance.size(), maxDop); + } + List> bucketSeqsPerDriverSeq = ListUtil.splitBySize(bucketSeqsOfInstance, logicalDop); + instance.setPipelineDop(expectedPhysicalDop); for (int driverSeq = 0; driverSeq < bucketSeqsPerDriverSeq.size(); driverSeq++) { int finalDriverSeq = driverSeq; @@ -225,30 +237,42 @@ private void assignScanRangesToNormalFragmentInstancePerWorker(ExecutionFragment } int expectedInstanceNum = Math.max(1, parallelExecInstanceNum); - List> scanRangesPerInstance = ListUtil.splitBySize(scanRangesOfNode, expectedInstanceNum); + List> scanRangesPerInstance = + ListUtil.splitBySize(scanRangesOfNode, expectedInstanceNum); for (List scanRanges : scanRangesPerInstance) { - FragmentInstance instance = new FragmentInstance(workerProvider.getWorkerById(workerId), execFragment); + FragmentInstance instance = + new FragmentInstance(workerProvider.getWorkerById(workerId), execFragment); execFragment.addInstance(instance); if (!enableAssignScanRangesPerDriverSeq(fragment, scanRanges)) { instance.addScanRanges(scanId, scanRanges); fragment.disablePhysicalPropertyOptimize(); } else { - int expectedDop = Math.max(1, Math.min(pipelineDop, scanRanges.size())); + int expectedPhysicalDop = Math.max(1, Math.min(pipelineDop, scanRanges.size())); + int logicalDop = expectedPhysicalDop; + if (fragment.isUseGroupExecution()) { + // if fragment using group execution + SessionVariable sv = ConnectContext.get().getSessionVariable(); + int maxDop = Math.min(sv.getGroupExecutionGroupScale() * expectedPhysicalDop, + sv.getGroupExecutionMaxGroups()); + logicalDop = Math.min(scanRanges.size(), maxDop); + } List> scanRangesPerDriverSeq; if (Config.enable_schedule_insert_query_by_row_count && isLoadType && !scanRanges.isEmpty() && scanRanges.get(0).getScan_range().isSetInternal_scan_range()) { - scanRangesPerDriverSeq = splitScanRangeParamByRowCount(scanRanges, expectedDop); + scanRangesPerDriverSeq = splitScanRangeParamByRowCount(scanRanges, logicalDop); } else { - scanRangesPerDriverSeq = ListUtil.splitBySize(scanRanges, expectedDop); + scanRangesPerDriverSeq = ListUtil.splitBySize(scanRanges, logicalDop); } - - if (fragment.isForceAssignScanRangesPerDriverSeq() && scanRangesPerDriverSeq.size() != pipelineDop) { - fragment.setPipelineDop(scanRangesPerDriverSeq.size()); + // Make pipeline input dop == sink dop to avoid extra local-shuffle. + // TODO: Make XXXSink support group execution to further improve performance. + if (fragment.isForceAssignScanRangesPerDriverSeq() && + expectedPhysicalDop != pipelineDop) { + fragment.setPipelineDop(expectedPhysicalDop); } - instance.setPipelineDop(scanRangesPerDriverSeq.size()); + instance.setPipelineDop(expectedPhysicalDop); for (int driverSeq = 0; driverSeq < scanRangesPerDriverSeq.size(); ++driverSeq) { instance.addScanRanges(scanId, driverSeq, scanRangesPerDriverSeq.get(driverSeq)); @@ -293,7 +317,8 @@ private static List> splitScanRangeParamByRowCount(List nodeIdGenerator = PlanNodeId.createGenerator(); private final IdGenerator fragmentIdGenerator = PlanFragmentId.createGenerator(); private final Map optExpressions = Maps.newHashMap(); + private List execGroups = new ArrayList<>(); private volatile ProfilingExecPlan profilingPlan; @@ -134,6 +136,13 @@ public List getOutputColumns() { return outputColumns; } + public void setExecGroups(List execGroups) { + this.execGroups = execGroups; + } + public List getExecGroups() { + return this.execGroups; + } + public void recordPlanNodeId2OptExpression(int id, OptExpression optExpression) { optExpressions.put(id, optExpression); } diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java b/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java index d0943be28e696c..2871e852e0c3cb 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java @@ -75,6 +75,8 @@ import com.starrocks.planner.EsScanNode; import com.starrocks.planner.ExceptNode; import com.starrocks.planner.ExchangeNode; +import com.starrocks.planner.ExecGroup; +import com.starrocks.planner.ExecGroupSets; import com.starrocks.planner.FileScanNode; import com.starrocks.planner.FileTableScanNode; import com.starrocks.planner.FragmentNormalizer; @@ -352,6 +354,16 @@ private static ExecPlan finalizeFragments(ExecPlan execPlan, TResultSinkType res fragment.createDataSink(resultSinkType); } Collections.reverse(fragments); + // assign colocate groups to plan fragment + if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable().isEnableGroupExecution()) { + List colocateExecGroups = + execPlan.getExecGroups().stream().filter(ExecGroup::isColocateExecGroup).collect( + Collectors.toList()); + for (PlanFragment fragment : fragments) { + fragment.assignColocateExecGroups(fragment.getPlanRoot(), colocateExecGroups); + } + Preconditions.checkState(colocateExecGroups.isEmpty()); + } // compute local_rf_waiting_set for each PlanNode. // when enable_pipeline_engine=true and enable_global_runtime_filter=false, we should clear @@ -383,6 +395,8 @@ private static ExecPlan finalizeFragments(ExecPlan execPlan, TResultSinkType res private static class PhysicalPlanTranslator extends OptExpressionVisitor { private final ColumnRefFactory columnRefFactory; private final IdGenerator runtimeFilterIdIdGenerator = RuntimeFilterId.createGenerator(); + private final ExecGroupSets execGroups = new ExecGroupSets(); + private ExecGroup currentExecGroup = execGroups.newExecGroup(); private boolean canUseLocalShuffleAgg = true; @@ -393,6 +407,7 @@ public PhysicalPlanTranslator(ColumnRefFactory columnRefFactory) { public PlanFragment translate(OptExpression optExpression, ExecPlan context) { PlanFragment fragment = visit(optExpression, context); computeFragmentCost(context, fragment); + context.setExecGroups(execGroups.getExecGroups()); return fragment; } @@ -430,6 +445,8 @@ public PlanFragment visit(OptExpression optExpression, ExecPlan context) { if (projection != null) { fragment = buildProjectNode(optExpression, projection, fragment, context); + // push project to the same group + currentExecGroup.add(fragment.getPlanRoot()); } PlanNode planRoot = fragment.getPlanRoot(); if (!(optExpression.getOp() instanceof PhysicalProjectOperator) && planRoot instanceof ProjectNode) { @@ -557,6 +574,7 @@ public PlanFragment visitPhysicalProject(OptExpression optExpr, ExecPlan context projectNode.setHasNullableGenerateChild(); projectNode.computeStatistics(optExpr.getStatistics()); + currentExecGroup.add(projectNode); for (SlotId sid : projectMap.keySet()) { SlotDescriptor slotDescriptor = tupleDescriptor.getSlot(sid.asInt()); @@ -627,7 +645,6 @@ public PlanFragment buildProjectNode(OptExpression optExpression, Projection nod projectNode.computeStatistics(b.build()); }); - for (SlotId sid : projectMap.keySet()) { SlotDescriptor slotDescriptor = tupleDescriptor.getSlot(sid.asInt()); slotDescriptor.setIsNullable(slotDescriptor.getIsNullable() | projectNode.isHasNullableGenerateChild()); @@ -693,6 +710,7 @@ public PlanFragment visitPhysicalDecode(OptExpression optExpression, ExecPlan co node.getDictIdToStringsId(), projectMap, slotRefMap); decodeNode.computeStatistics(optExpression.getStatistics()); decodeNode.setLimit(node.getLimit()); + currentExecGroup.add(decodeNode); inputFragment.setPlanRoot(decodeNode); return inputFragment; @@ -753,6 +771,7 @@ public PlanFragment visitPhysicalOlapScan(OptExpression optExpr, ExecPlan contex scanNode.setIsSortedByKeyPerTablet(node.needSortedByKeyPerTablet()); scanNode.setIsOutputChunkByBucket(node.needOutputChunkByBucket()); scanNode.setWithoutColocateRequirement(node.isWithoutColocateRequirement()); + currentExecGroup.add(scanNode); // set tablet try { scanNode.updateScanInfo(node.getSelectedPartitionId(), @@ -898,6 +917,7 @@ public PlanFragment visitPhysicalMetaScan(OptExpression optExpression, ExecPlan tupleDescriptor, (OlapTable) scan.getTable(), scan.getAggColumnIdToNames()); scanNode.computeRangeLocations(); scanNode.computeStatistics(optExpression.getStatistics()); + currentExecGroup.add(scanNode, true); for (Map.Entry entry : scan.getColRefToColumnMetaMap().entrySet()) { SlotDescriptor slotDescriptor = @@ -1003,6 +1023,7 @@ public PlanFragment visitPhysicalHudiScan(OptExpression optExpression, ExecPlan new HudiScanNode(context.getNextNodeId(), tupleDescriptor, "HudiScanNode"); hudiScanNode.computeStatistics(optExpression.getStatistics()); hudiScanNode.setScanOptimzeOption(node.getScanOptimzeOption()); + currentExecGroup.add(hudiScanNode, true); try { HDFSScanNodePredicates scanNodePredicates = hudiScanNode.getScanNodePredicates(); scanNodePredicates.setSelectedPartitionIds(predicates.getSelectedPartitionIds()); @@ -1046,6 +1067,7 @@ public PlanFragment visitPhysicalHiveScan(OptExpression optExpression, ExecPlan new HdfsScanNode(context.getNextNodeId(), tupleDescriptor, "HdfsScanNode"); hdfsScanNode.setScanOptimzeOption(node.getScanOptimzeOption()); hdfsScanNode.computeStatistics(optExpression.getStatistics()); + currentExecGroup.add(hdfsScanNode, true); try { HDFSScanNodePredicates scanNodePredicates = hdfsScanNode.getScanNodePredicates(); scanNodePredicates.setSelectedPartitionIds(predicates.getSelectedPartitionIds()); @@ -1087,6 +1109,7 @@ public PlanFragment visitPhysicalFileScan(OptExpression optExpression, ExecPlan new FileTableScanNode(context.getNextNodeId(), tupleDescriptor, "FileTableScanNode"); fileTableScanNode.setScanOptimzeOption(node.getScanOptimzeOption()); fileTableScanNode.computeStatistics(optExpression.getStatistics()); + currentExecGroup.add(fileTableScanNode, true); try { HDFSScanNodePredicates scanNodePredicates = fileTableScanNode.getScanNodePredicates(); @@ -1125,6 +1148,7 @@ public PlanFragment visitPhysicalDeltaLakeScan(OptExpression optExpression, Exec new DeltaLakeScanNode(context.getNextNodeId(), tupleDescriptor, "DeltaLakeScanNode"); deltaLakeScanNode.computeStatistics(optExpression.getStatistics()); deltaLakeScanNode.setScanOptimzeOption(node.getScanOptimzeOption()); + currentExecGroup.add(deltaLakeScanNode, true); try { // set predicate ScalarOperatorToExpr.FormatterContext formatterContext = @@ -1167,6 +1191,7 @@ public PlanFragment visitPhysicalPaimonScan(OptExpression optExpression, ExecPla PaimonScanNode paimonScanNode = new PaimonScanNode(context.getNextNodeId(), tupleDescriptor, "PaimonScanNode"); paimonScanNode.setScanOptimzeOption(node.getScanOptimzeOption()); + currentExecGroup.add(paimonScanNode, true); try { // set predicate ScalarOperatorToExpr.FormatterContext formatterContext = @@ -1211,6 +1236,7 @@ public PlanFragment visitPhysicalOdpsScan(OptExpression optExpression, ExecPlan OdpsScanNode odpsScanNode = new OdpsScanNode(context.getNextNodeId(), tupleDescriptor, "OdpsScanNode"); odpsScanNode.setScanOptimzeOption(node.getScanOptimzeOption()); + currentExecGroup.add(odpsScanNode, true); try { // set predicate ScalarOperatorToExpr.FormatterContext formatterContext = @@ -1259,9 +1285,11 @@ public PlanFragment visitPhysicalIcebergScan(OptExpression optExpression, ExecPl TupleDescriptor equalityDeleteTupleDesc = context.getDescTbl().createTupleDescriptor(); IcebergScanNode icebergScanNode = - new IcebergScanNode(context.getNextNodeId(), tupleDescriptor, "IcebergScanNode", equalityDeleteTupleDesc); + new IcebergScanNode(context.getNextNodeId(), tupleDescriptor, "IcebergScanNode", + equalityDeleteTupleDesc); icebergScanNode.computeStatistics(optExpression.getStatistics()); icebergScanNode.setScanOptimzeOption(node.getScanOptimzeOption()); + currentExecGroup.add(icebergScanNode, true); try { // set predicate ScalarOperatorToExpr.FormatterContext formatterContext = @@ -1322,6 +1350,7 @@ public PlanFragment visitPhysicalSchemaScan(OptExpression optExpression, ExecPla scanNode.setUserIp(context.getConnectContext().getRemoteIP()); scanNode.setLimit(node.getLimit()); scanNode.setScanOptimzeOption(node.getScanOptimzeOption()); + currentExecGroup.add(scanNode, true); // set predicate List predicates = Utils.extractConjuncts(node.getPredicate()); @@ -1473,6 +1502,7 @@ public PlanFragment visitPhysicalMysqlScan(OptExpression optExpression, ExecPlan MysqlScanNode scanNode = new MysqlScanNode(context.getNextNodeId(), tupleDescriptor, (MysqlTable) node.getTable()); + currentExecGroup.add(scanNode, true); if (node.getTemporalClause() != null) { scanNode.setTemporalClause(node.getTemporalClause()); @@ -1517,6 +1547,7 @@ public PlanFragment visitPhysicalEsScan(OptExpression optExpression, ExecPlan co tupleDescriptor.computeMemLayout(); EsScanNode scanNode = new EsScanNode(context.getNextNodeId(), tupleDescriptor, "EsScanNode"); + currentExecGroup.add(scanNode, true); // set predicate List predicates = Utils.extractConjuncts(node.getPredicate()); ScalarOperatorToExpr.FormatterContext formatterContext = @@ -1562,6 +1593,7 @@ public PlanFragment visitPhysicalJDBCScan(OptExpression optExpression, ExecPlan JDBCScanNode scanNode = new JDBCScanNode(context.getNextNodeId(), tupleDescriptor, (JDBCTable) node.getTable()); + currentExecGroup.add(scanNode, true); // set predicate List predicates = Utils.extractConjuncts(node.getPredicate()); @@ -1602,6 +1634,7 @@ public PlanFragment visitPhysicalValues(OptExpression optExpr, ExecPlan context) EmptySetNode emptyNode = new EmptySetNode(context.getNextNodeId(), Lists.newArrayList(tupleDescriptor.getId())); emptyNode.computeStatistics(optExpr.getStatistics()); + currentExecGroup.add(emptyNode, true); PlanFragment fragment = new PlanFragment(context.getNextFragmentId(), emptyNode, DataPartition.UNPARTITIONED); context.getFragments().add(fragment); @@ -1609,6 +1642,7 @@ public PlanFragment visitPhysicalValues(OptExpression optExpr, ExecPlan context) } else { UnionNode unionNode = new UnionNode(context.getNextNodeId(), tupleDescriptor.getId()); unionNode.setLimit(valuesOperator.getLimit()); + currentExecGroup.add(unionNode, true); List> consts = new ArrayList<>(); for (List row : valuesOperator.getRows()) { @@ -1985,6 +2019,16 @@ public PlanFragment visitPhysicalHashAggregate(OptExpression optExpr, ExecPlan c } aggregationNode.getAggInfo().setIntermediateAggrExprs(intermediateAggrExprs); + // enable group execution for: + // any aggregate stage + // disable group execution for local shuffle Agg + boolean useGroupExecution = + ConnectContext.get().getSessionVariable().isEnableGroupExecution() && !groupBys.isEmpty() && + !withLocalShuffle; + if (useGroupExecution) { + currentExecGroup.setColocateGroup(); + } + currentExecGroup.add(aggregationNode); inputFragment.setPlanRoot(aggregationNode); inputFragment.mergeQueryDictExprs(originalInputFragment.getQueryGlobalDictExprs()); inputFragment.mergeQueryGlobalDicts(originalInputFragment.getQueryGlobalDicts()); @@ -2065,12 +2109,15 @@ private void setMergeAggFn(List aggregateExprList, List= 0); if (!topN.isSplit()) { @@ -2139,6 +2191,8 @@ private PlanFragment buildFinalTopNFragment(ExecPlan context, TopNType topNType, sortNode.setTopNType(topNType); exchangeNode.setMergeInfo(sortNode.getSortInfo(), offset); exchangeNode.computeStatistics(optExpr.getStatistics()); + currentExecGroup.add(exchangeNode, true); + if (TopNType.ROW_NUMBER.equals(topNType)) { exchangeNode.setLimit(limit); } else { @@ -2234,8 +2288,9 @@ private PlanFragment buildPartialTopNFragment(OptExpression optExpr, ExecPlan co sortNode.resolvedTupleExprs = resolvedTupleExprs; sortNode.setHasNullableGenerateChild(); sortNode.computeStatistics(optExpr.getStatistics()); + currentExecGroup.add(sortNode, true); if (shouldBuildGlobalRuntimeFilter()) { - sortNode.buildRuntimeFilters(runtimeFilterIdIdGenerator, context.getDescTbl()); + sortNode.buildRuntimeFilters(runtimeFilterIdIdGenerator, context.getDescTbl(), execGroups); } inputFragment.setPlanRoot(sortNode); @@ -2264,8 +2319,10 @@ private boolean shouldBuildGlobalRuntimeFilter() { @Override public PlanFragment visitPhysicalHashJoin(OptExpression optExpr, ExecPlan context) { PlanFragment leftFragment = visit(optExpr.inputAt(0), context); + ExecGroup leftExecGroup = this.currentExecGroup; + this.currentExecGroup = execGroups.newExecGroup(); PlanFragment rightFragment = visit(optExpr.inputAt(1), context); - return visitPhysicalJoin(leftFragment, rightFragment, optExpr, context); + return visitPhysicalJoin(leftFragment, rightFragment, leftExecGroup, currentExecGroup, optExpr, context); } private List extractConjuncts(ScalarOperator predicate, ExecPlan context) { @@ -2299,7 +2356,10 @@ private void setNullableForJoin(JoinOperator joinOperator, public PlanFragment visitPhysicalNestLoopJoin(OptExpression optExpr, ExecPlan context) { PhysicalJoinOperator node = (PhysicalJoinOperator) optExpr.getOp(); PlanFragment leftFragment = visit(optExpr.inputAt(0), context); + ExecGroup leftExecGroup = this.currentExecGroup; + this.currentExecGroup = execGroups.newExecGroup(); PlanFragment rightFragment = visit(optExpr.inputAt(1), context); + this.currentExecGroup = leftExecGroup; List conjuncts = extractConjuncts(node.getPredicate(), context); List joinOnConjuncts = extractConjuncts(node.getOnPredicate(), context); @@ -2323,7 +2383,7 @@ public PlanFragment visitPhysicalNestLoopJoin(OptExpression optExpr, ExecPlan co joinNode.computeStatistics(optExpr.getStatistics()); joinNode.addConjuncts(conjuncts); joinNode.setProbePartitionByExprs(probePartitionByExprs); - + currentExecGroup.add(joinNode); // Connect parent and child fragment rightFragment.getPlanRoot().setFragment(leftFragment); @@ -2343,7 +2403,7 @@ public PlanFragment visitPhysicalNestLoopJoin(OptExpression optExpr, ExecPlan co } if (shouldBuildGlobalRuntimeFilter()) { - joinNode.buildRuntimeFilters(runtimeFilterIdIdGenerator, context.getDescTbl()); + joinNode.buildRuntimeFilters(runtimeFilterIdIdGenerator, context.getDescTbl(), execGroups); } leftFragment.mergeQueryGlobalDicts(rightFragment.getQueryGlobalDicts()); @@ -2354,6 +2414,8 @@ public PlanFragment visitPhysicalNestLoopJoin(OptExpression optExpr, ExecPlan co @Override public PlanFragment visitPhysicalMergeJoin(OptExpression optExpr, ExecPlan context) { PlanFragment leftFragment = visit(optExpr.inputAt(0), context); + ExecGroup leftExecGroup = this.currentExecGroup; + this.currentExecGroup = execGroups.newExecGroup(); PlanFragment rightFragment = visit(optExpr.inputAt(1), context); PlanNode leftPlanRoot = leftFragment.getPlanRoot(); PlanNode rightPlanRoot = rightFragment.getPlanRoot(); @@ -2369,7 +2431,8 @@ public PlanFragment visitPhysicalMergeJoin(OptExpression optExpr, ExecPlan conte rightFragment.setPlanRoot(rightPlanRoot.getChild(0)); } - PlanFragment planFragment = visitPhysicalJoin(leftFragment, rightFragment, optExpr, context); + PlanFragment planFragment = + visitPhysicalJoin(leftFragment, rightFragment, leftExecGroup, currentExecGroup, optExpr, context); if (needDealSort) { leftExpression.setChild(0, optExpr.inputAt(0)); rightExpression.setChild(0, optExpr.inputAt(1)); @@ -2400,6 +2463,7 @@ private List getShuffleExprs(HashDistributionSpec hashDistributionSpec, Ex } private PlanFragment visitPhysicalJoin(PlanFragment leftFragment, PlanFragment rightFragment, + ExecGroup leftExecGroup, ExecGroup rightExecGroup, OptExpression optExpr, ExecPlan context) { PhysicalJoinOperator node = (PhysicalJoinOperator) optExpr.getOp(); @@ -2477,9 +2541,27 @@ private PlanFragment visitPhysicalJoin(PlanFragment leftFragment, PlanFragment r joinNode.setLimit(node.getLimit()); joinNode.computeStatistics(optExpr.getStatistics()); joinNode.setProbePartitionByExprs(probePartitionByExprs); + // enable group execution for colocate join + currentExecGroup = leftExecGroup; + if (ConnectContext.get().getSessionVariable().isEnableGroupExecution()) { + if (distributionMode.equals(JoinNode.DistributionMode.COLOCATE) && + !leftExecGroup.isDisableColocateGroup() && !rightExecGroup.isDisableColocateGroup()) { + currentExecGroup.setColocateGroup(); + currentExecGroup.merge(rightExecGroup); + execGroups.remove(rightExecGroup); + } else if (distributionMode.equals(JoinNode.DistributionMode.BROADCAST) && + !leftExecGroup.isDisableColocateGroup()) { + // do nothing + } else { + // we don't support group execution for other join + currentExecGroup = execGroups.newExecGroup(); + currentExecGroup.setDisableColocateGroup(); + } + } + currentExecGroup.add(joinNode); if (shouldBuildGlobalRuntimeFilter()) { - joinNode.buildRuntimeFilters(runtimeFilterIdIdGenerator, context.getDescTbl()); + joinNode.buildRuntimeFilters(runtimeFilterIdIdGenerator, context.getDescTbl(), execGroups); } return buildJoinFragment(context, leftFragment, rightFragment, distributionMode, joinNode); @@ -3117,7 +3199,7 @@ public PlanFragment visitPhysicalStreamJoin(OptExpression optExpr, ExecPlan cont JoinNode joinNode = new StreamJoinNode(context.getNextNodeId(), leftFragmentPlanRoot, rightFragmentPlanRoot, node.getJoinType(), eqJoinConjuncts, otherJoinConjuncts); - + currentExecGroup.add(joinNode, true); // 4. Build outputColumns fillSlotsInfo(node.getProjection(), joinNode, optExpr, joinExpr.requiredColsForFilter); @@ -3193,7 +3275,6 @@ private PlanFragment buildJoinFragment(ExecPlan context, PlanFragment leftFragme context.getFragments().remove(leftFragment); context.getFragments().add(leftFragment); - leftFragment.mergeQueryGlobalDicts(rightFragment.getQueryGlobalDicts()); leftFragment.mergeQueryDictExprs(rightFragment.getQueryGlobalDictExprs()); @@ -3289,6 +3370,7 @@ public PlanFragment visitPhysicalStreamAgg(OptExpression optExpr, ExecPlan conte aggNode.setHasNullableGenerateChild(); aggNode.computeStatistics(optExpr.getStatistics()); + currentExecGroup.add(aggNode, true); inputFragment.setPlanRoot(aggNode); return inputFragment; } @@ -3305,6 +3387,7 @@ public PlanFragment visitPhysicalStreamScan(OptExpression optExpr, ExecPlan cont BinlogScanNode binlogScanNode = new BinlogScanNode(context.getNextNodeId(), tupleDescriptor); binlogScanNode.computeStatistics(optExpr.getStatistics()); + currentExecGroup.add(binlogScanNode, true); try { binlogScanNode.computeScanRanges(); } catch (UserException e) { @@ -3413,6 +3496,7 @@ public PlanFragment visitPhysicalTableFunctionTableScan(OptExpression optExpress scanNode.setLimit(node.getLimit()); scanNode.computeStatistics(optExpression.getStatistics()); + currentExecGroup.add(scanNode, true); context.getScanNodes().add(scanNode); PlanFragment fragment = diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/plan/ColocateJoinTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/plan/ColocateJoinTest.java index 00b0ff11bd48e0..5c3afca6233bdb 100644 --- a/fe/fe-core/src/test/java/com/starrocks/sql/plan/ColocateJoinTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/sql/plan/ColocateJoinTest.java @@ -28,7 +28,7 @@ import java.util.List; import java.util.stream.Stream; -class ColocateJoinTest extends PlanTestBase { +public class ColocateJoinTest extends PlanTestBase { @BeforeAll public static void beforeClass() throws Exception { @@ -48,7 +48,6 @@ public static void beforeClass() throws Exception { "\"colocate_with\" = \"colocate_group_1\"" + ");"); } - @ParameterizedTest(name = "sql_{index}: {0}.") @MethodSource("colocateJoinOnceSqls") void testColocateJoinOnce(String sql) throws Exception { @@ -231,5 +230,4 @@ private static Stream colocateThenAggSqls() { return pairs.stream().map(e -> Arguments.of(e)); } - } diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/plan/GroupExecutionPlanTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/plan/GroupExecutionPlanTest.java new file mode 100644 index 00000000000000..1525d33a889e0c --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/sql/plan/GroupExecutionPlanTest.java @@ -0,0 +1,95 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// 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 +// +// https://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. + +package com.starrocks.sql.plan; + +import com.google.api.client.util.Lists; +import com.starrocks.common.FeConstants; +import org.junit.AfterClass; +import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; + +import java.util.List; + +public class GroupExecutionPlanTest extends PlanTestBase { + @BeforeAll + public static void beforeClass() throws Exception { + PlanTestBase.beforeClass(); + FeConstants.runningUnitTest = true; + connectContext.getSessionVariable().setEnableGroupExecution(true); + connectContext.getSessionVariable().setOptimizerExecuteTimeout(3000000); + } + + @AfterClass + public static void afterClass() { + connectContext.getSessionVariable().setEnableGroupExecution(false); + } + + @Test + public void testColocateGroupExecutionJoin() throws Exception { + FeConstants.runningUnitTest = true; + boolean enableGroupExecution = connectContext.getSessionVariable().isEnableGroupExecution(); + connectContext.getSessionVariable().setEnableGroupExecution(true); + try { + // normal simple case + List querys = Lists.newArrayList(); + querys.add("select * from colocate1 l join colocate2 r on l.k1=r.k1 and l.k2=r.k2;"); + querys.add("select * from colocate1 l left join colocate2 r on l.k1=r.k1 and l.k2=r.k2;"); + querys.add("select * from colocate1 l right join colocate2 r on l.k1=r.k1 and l.k2=r.k2;"); + querys.add("select l.* from colocate1 l left semi join colocate2 r on l.k1=r.k1 and l.k2=r.k2;"); + querys.add("select l.* from colocate1 l left anti join colocate2 r on l.k1=r.k1 and l.k2=r.k2;"); + // with cross join + querys.add("select l.* from colocate1 l, colocate2 r,colocate1 z where l.k1=r.k1 and l.k2=r.k2 "); + // colocate join with broadcast join + querys.add("select * from (select l.* from colocate1 l, colocate2 r where l.k1=r.k1 and l.k2=r.k2) tb" + + " join [broadcast] colocate1 z on z.k1 = tb.k1 "); + querys.add("select * from (select distinct k1, k2 from colocate1)l join (select k1, k2 from colocate2)r " + + " on l.k1=r.k1 and l.k2=r.k2;"); + + for (String sql : querys) { + String plan = getFragmentPlan(sql); + assertContains(plan, "colocate exec groups:"); + assertContains(plan, "COLOCATE"); + } + + } finally { + FeConstants.runningUnitTest = false; + connectContext.getSessionVariable().setEnableGroupExecution(enableGroupExecution); + } + } + + @Test + public void testGroupExecutionAgg() throws Exception { + FeConstants.runningUnitTest = true; + boolean enableGroupExecution = connectContext.getSessionVariable().isEnableGroupExecution(); + connectContext.getSessionVariable().setEnableGroupExecution(true); + try { + List querys = Lists.newArrayList(); + querys.add("select distinct k1,k2 from colocate1 l"); + querys.add("select distinct k1,k2 from colocate1 l limit 10"); + querys.add("select distinct k1,k2 from colocate1 l where k1 = 1 or k3 = 3"); + querys.add("select distinct k1,k2 from colocate1 l where k1 = 1"); + // for streaming agg + querys.add("select distinct L_SHIPDATE, L_LINENUMBER from lineitem_partition"); + + for (String sql : querys) { + String plan = getFragmentPlan(sql); + assertContains(plan, "colocate exec groups:"); + } + } finally { + FeConstants.runningUnitTest = false; + connectContext.getSessionVariable().setEnableGroupExecution(enableGroupExecution); + } + } +} diff --git a/gensrc/thrift/Planner.thrift b/gensrc/thrift/Planner.thrift index e1d84f87afd563..982f42954bb0de 100644 --- a/gensrc/thrift/Planner.thrift +++ b/gensrc/thrift/Planner.thrift @@ -57,6 +57,16 @@ struct TCacheParam { 11: optional set cached_plan_node_ids; } +struct TExecGroup { + 1: optional i32 group_id + 2: list plan_node_ids +} + +struct TGroupExecutionParam { + 1: optional bool enable_group_execution + 2: optional list exec_groups +} + // TPlanFragment encapsulates info needed to execute a particular // plan fragment, including how to produce and how to partition its output. // It leaves out node-specific parameters neede for the actual execution. @@ -99,6 +109,7 @@ struct TPlanFragment { 21: optional list load_global_dicts 22: optional TCacheParam cache_param 23: optional map query_global_dict_exprs + 24: optional TGroupExecutionParam group_execution_param } // location information for a single scan range diff --git a/gensrc/thrift/RuntimeFilter.thrift b/gensrc/thrift/RuntimeFilter.thrift index 3c78c5635fac96..e913b60bf09982 100644 --- a/gensrc/thrift/RuntimeFilter.thrift +++ b/gensrc/thrift/RuntimeFilter.thrift @@ -79,7 +79,7 @@ enum TRuntimeFilterLayoutMode { // global-two-level grf formed by concatenating PIPELINE_BUCKET rf. GLOBAL_BUCKET_2L, // global-one-level grf(old-fashioned) formed by concatenating SINGLETON rf generated by bucket-shuffle/colocate Join. - GLOBAL_BUCKET_1L + GLOBAL_BUCKET_1L, } struct TRuntimeFilterLayout { diff --git a/test/sql/test_group_execution/T/test_group_execution_agg b/test/sql/test_group_execution/T/test_group_execution_agg new file mode 100644 index 00000000000000..ca311d0f976708 --- /dev/null +++ b/test/sql/test_group_execution/T/test_group_execution_agg @@ -0,0 +1,14 @@ +-- name: test_group_execution_agg + +-- no partition table +-- partition table +-- colocate partition table +-- single tablet table agg case +-- local exchange process case +-- query cache with group execution case +-- streaming agg case +-- force_streaming +-- force_pregg +-- sorted streaming agg +-- tablet_sink_case +-- black hole case \ No newline at end of file diff --git a/test/sql/test_group_execution/T/test_group_execution_combine b/test/sql/test_group_execution/T/test_group_execution_combine new file mode 100644 index 00000000000000..e69de29bb2d1d6 diff --git a/test/sql/test_group_execution/T/test_group_execution_join b/test/sql/test_group_execution/T/test_group_execution_join new file mode 100644 index 00000000000000..9608424e43ee6a --- /dev/null +++ b/test/sql/test_group_execution/T/test_group_execution_join @@ -0,0 +1,66 @@ +-- name: test_group_execution_join + +CREATE TABLE `t0` ( + `c0` int(11) NULL COMMENT "", + `c1` varchar(20) NULL COMMENT "", + `c2` varchar(200) NULL COMMENT "", + `c3` int(11) NULL COMMENT "" +) ENGINE=OLAP +DUPLICATE KEY(`c0`, `c1`) +COMMENT "OLAP" +DISTRIBUTED BY HASH(`c0`, `c1`) BUCKETS 16 +PROPERTIES ( +"colocate_with" = "${uuid0}", +"replication_num" = "1", +"in_memory" = "false", +"storage_format" = "DEFAULT", +"enable_persistent_index" = "false", +"replicated_storage" = "true", +"compression" = "LZ4" +); + +CREATE TABLE `t1` ( + `c0` int(11) NULL COMMENT "", + `c1` varchar(20) NULL COMMENT "", + `c2` varchar(200) NULL COMMENT "", + `c3` int(11) NULL COMMENT "" +) ENGINE=OLAP +DUPLICATE KEY(`c0`, `c1`) +COMMENT "OLAP" +DISTRIBUTED BY HASH(`c0`, `c1`) BUCKETS 16 +PROPERTIES ( +"colocate_with" = "${uuid0}", +"replication_num" = "1", +"in_memory" = "false", +"storage_format" = "DEFAULT", +"enable_persistent_index" = "false", +"replicated_storage" = "true", +"compression" = "LZ4" +); + +insert into t0 SELECT generate_series, generate_series, generate_series, generate_series FROM TABLE(generate_series(1, 40960)); +insert into t0 values (null,null,null,null); +insert into t1 SELECT * FROM t0; + +select count(*) from t0; +select count(*) from t1; +select count(*) from empty_t; + +-- hash join case +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where l.c3 > 100; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l left join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where l.c3 > 100; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l right join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where r.c3 < 1024; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where r.c3 < 1024; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where r.c3 < 1024; +-- probe side empty +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where l.c3 < 0; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l left join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where l.c3 < 0; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l right join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where l.c3 < 0; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l full outer join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where l.c3 < 0; +-- build side empty +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where r.c3 < 0; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l left join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where r.c3 < 0; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l right join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where r.c3 < 0; +select count(l.c0), avg(l.c0), count(l.c1), count(l.c0), count(r.c1) from t0 l full outer join [colocate] t1 r on l.c0 = r.c0 and l.c1 = r.c1 where r.c3 < 0; +-- with other condition +-- with other conjuncts