This is an automated email from the ASF dual-hosted git repository. gabriellee pushed a commit to branch branch-2.1 in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-2.1 by this push: new 5bd33fc88cb [pick](branch-2.1) pick #41292 #41350 #41589 #41628 #41743 #41601 #41667 #41751 (#41927) 5bd33fc88cb is described below commit 5bd33fc88cb281795237d7f67b0717c904f037d2 Author: Gabriel <gabrielleeb...@gmail.com> AuthorDate: Wed Oct 16 15:41:28 2024 +0800 [pick](branch-2.1) pick #41292 #41350 #41589 #41628 #41743 #41601 #41667 #41751 (#41927) ## Proposed changes pick #41292 #41350 #41589 #41628 #41743 #41601 #41667 #41751 <!--Describe your changes.--> --------- Co-authored-by: Pxl <pxl...@qq.com> --- be/src/exprs/bloom_filter_func.h | 4 +-- be/src/exprs/runtime_filter.cpp | 9 +++-- be/src/exprs/runtime_filter_slots.h | 12 ++++--- be/src/pipeline/exec/exchange_sink_operator.cpp | 35 ++++++++++-------- be/src/pipeline/exec/exchange_sink_operator.h | 4 ++- be/src/pipeline/exec/hashjoin_build_sink.cpp | 42 ++++++++++++++-------- be/src/pipeline/exec/multi_cast_data_stream_sink.h | 15 +++++--- .../exec/partitioned_aggregation_sink_operator.h | 1 - be/src/pipeline/exec/spill_sort_sink_operator.h | 1 - be/src/pipeline/pipeline.cpp | 13 ++++++- be/src/pipeline/pipeline.h | 17 ++++++++- be/src/pipeline/pipeline_fragment_context.cpp | 4 +-- be/src/pipeline/pipeline_fragment_context.h | 2 +- be/src/pipeline/pipeline_task.h | 4 +++ .../local_exchange_sink_operator.cpp | 18 ++-------- .../local_exchange/local_exchange_sink_operator.h | 2 -- be/src/pipeline/pipeline_x/operator.h | 3 ++ .../pipeline_x/pipeline_x_fragment_context.cpp | 24 ++++++++++--- .../pipeline_x/pipeline_x_fragment_context.h | 3 ++ be/src/pipeline/pipeline_x/pipeline_x_task.cpp | 40 ++++++++++++++++----- be/src/pipeline/pipeline_x/pipeline_x_task.h | 6 +++- be/src/pipeline/task_scheduler.cpp | 2 +- be/src/runtime/runtime_filter_mgr.cpp | 4 ++- be/src/runtime/runtime_state.h | 5 +++ be/src/vec/exec/join/vhash_join_node.h | 2 +- be/src/vec/runtime/shared_hash_table_controller.h | 1 + be/test/exprs/runtime_filter_test.cpp | 5 --- 27 files changed, 188 insertions(+), 90 deletions(-) diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h index 6d452bbe992..ad43b78cc6a 100644 --- a/be/src/exprs/bloom_filter_func.h +++ b/be/src/exprs/bloom_filter_func.h @@ -151,19 +151,19 @@ public: } Status merge(BloomFilterFuncBase* bloomfilter_func) { + DCHECK(bloomfilter_func != nullptr); + DCHECK(bloomfilter_func->_bloom_filter != nullptr); // If `_inited` is false, there is no memory allocated in bloom filter and this is the first // call for `merge` function. So we just reuse this bloom filter, and we don't need to // allocate memory again. if (!_inited) { auto* other_func = static_cast<BloomFilterFuncBase*>(bloomfilter_func); DCHECK(_bloom_filter == nullptr); - DCHECK(bloomfilter_func != nullptr); _bloom_filter = bloomfilter_func->_bloom_filter; _bloom_filter_alloced = other_func->_bloom_filter_alloced; _inited = true; return Status::OK(); } - DCHECK(bloomfilter_func != nullptr); auto* other_func = static_cast<BloomFilterFuncBase*>(bloomfilter_func); if (_bloom_filter_alloced != other_func->_bloom_filter_alloced) { return Status::InternalError( diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 4e9a12b5bc5..1d1446fa63d 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -468,10 +468,10 @@ public: const TExpr& probe_expr); Status merge(const RuntimePredicateWrapper* wrapper) { - if (is_ignored() || wrapper->is_ignored()) { - _context->ignored = true; + if (wrapper->is_ignored()) { return Status::OK(); } + _context->ignored = false; bool can_not_merge_in_or_bloom = _filter_type == RuntimeFilterType::IN_OR_BLOOM_FILTER && @@ -489,7 +489,10 @@ public: switch (_filter_type) { case RuntimeFilterType::IN_FILTER: { - // try insert set + if (!_context->hybrid_set) { + _context->ignored = true; + return Status::OK(); + } _context->hybrid_set->insert(wrapper->_context->hybrid_set.get()); if (_max_in_num >= 0 && _context->hybrid_set->size() >= _max_in_num) { _context->ignored = true; diff --git a/be/src/exprs/runtime_filter_slots.h b/be/src/exprs/runtime_filter_slots.h index ebda4b56fcc..b6ca31c7272 100644 --- a/be/src/exprs/runtime_filter_slots.h +++ b/be/src/exprs/runtime_filter_slots.h @@ -98,12 +98,16 @@ public: return Status::OK(); } + Status ignore_all_filters() { + for (auto filter : _runtime_filters) { + filter->set_ignored(); + } + return Status::OK(); + } + Status init_filters(RuntimeState* state, uint64_t local_hash_table_size) { // process IN_OR_BLOOM_FILTER's real type for (auto* filter : _runtime_filters) { - if (filter->get_ignored()) { - continue; - } if (filter->type() == RuntimeFilterType::IN_OR_BLOOM_FILTER && get_real_size(filter, local_hash_table_size) > state->runtime_filter_max_in_num()) { RETURN_IF_ERROR(filter->change_to_bloom_filter()); @@ -140,7 +144,7 @@ public: } // publish runtime filter - Status publish(bool publish_local = false) { + Status publish(bool publish_local) { for (auto& pair : _runtime_filters_map) { for (auto& filter : pair.second) { RETURN_IF_ERROR(filter->publish(publish_local)); diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index ada5d5455b0..9958a0b6fc1 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -189,13 +189,17 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { PUniqueId id; id.set_hi(_state->query_id().hi); id.set_lo(_state->query_id().lo); - _sink_buffer = std::make_unique<ExchangeSinkBuffer<ExchangeSinkLocalState>>( - id, p._dest_node_id, _sender_id, _state->be_number(), state, this); - register_channels(_sink_buffer.get()); - _queue_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), - "ExchangeSinkQueueDependency", true); - _sink_buffer->set_dependency(_queue_dependency, _finish_dependency); + if (!only_local_exchange) { + _sink_buffer = std::make_unique<ExchangeSinkBuffer<ExchangeSinkLocalState>>( + id, p._dest_node_id, _sender_id, _state->be_number(), state, this); + register_channels(_sink_buffer.get()); + _queue_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), + "ExchangeSinkQueueDependency", true); + _sink_buffer->set_dependency(_queue_dependency, _finish_dependency); + _finish_dependency->block(); + } + if ((_part_type == TPartitionType::UNPARTITIONED || channels.size() == 1) && !only_local_exchange) { _broadcast_dependency = Dependency::create_shared( @@ -304,7 +308,6 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { fmt::format("Crc32HashPartitioner({})", _partition_count)); } - _finish_dependency->block(); if (_part_type == TPartitionType::HASH_PARTITIONED || _part_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED || _part_type == TPartitionType::TABLE_SINK_HASH_PARTITIONED) { @@ -638,8 +641,9 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block final_st = st; } } - local_state._sink_buffer->set_should_stop(); - return final_st; + if (local_state._sink_buffer) { + local_state._sink_buffer->set_should_stop(); + } } return final_st; } @@ -725,11 +729,14 @@ Status ExchangeSinkOperatorX::channel_add_rows_with_idx( std::string ExchangeSinkLocalState::debug_string(int indentation_level) const { fmt::memory_buffer debug_string_buffer; fmt::format_to(debug_string_buffer, "{}", Base::debug_string(indentation_level)); - fmt::format_to(debug_string_buffer, - ", Sink Buffer: (_should_stop = {}, _busy_channels = {}, _is_finishing = {}), " - "_reach_limit: {}", - _sink_buffer->_should_stop.load(), _sink_buffer->_busy_channels.load(), - _sink_buffer->_is_finishing.load(), _reach_limit.load()); + if (_sink_buffer) { + fmt::format_to( + debug_string_buffer, + ", Sink Buffer: (_should_stop = {}, _busy_channels = {}, _is_finishing = {}), " + "_reach_limit: {}", + _sink_buffer->_should_stop.load(), _sink_buffer->_busy_channels.load(), + _sink_buffer->_is_finishing.load(), _reach_limit.load()); + } return fmt::to_string(debug_string_buffer); } diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index aeb6a1503b7..8d9382dadd0 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -96,7 +96,9 @@ public: std::vector<Dependency*> dependencies() const override { std::vector<Dependency*> dep_vec; - dep_vec.push_back(_queue_dependency.get()); + if (_queue_dependency) { + dep_vec.push_back(_queue_dependency.get()); + } if (_broadcast_dependency) { dep_vec.push_back(_broadcast_dependency.get()); } diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 33e64017d50..a177c1b59e1 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -22,6 +22,7 @@ #include "exprs/bloom_filter_func.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/operator.h" +#include "pipeline/pipeline_x/pipeline_x_task.h" #include "vec/data_types/data_type_nullable.h" #include "vec/exec/join/vhash_join_node.h" #include "vec/utils/template_helpers.hpp" @@ -122,6 +123,9 @@ Status HashJoinBuildSinkLocalState::open(RuntimeState* state) { } Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_status) { + if (_closed) { + return Status::OK(); + } auto p = _parent->cast<HashJoinBuildSinkOperatorX>(); Defer defer {[&]() { if (_should_build_hash_table && p._shared_hashtable_controller) { @@ -130,25 +134,30 @@ Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_statu }}; if (!_runtime_filter_slots || _runtime_filters.empty() || state->is_cancelled()) { - return Status::OK(); + return Base::close(state, exec_status); } - auto* block = _shared_state->build_block.get(); - uint64_t hash_table_size = block ? block->rows() : 0; - { - SCOPED_TIMER(_runtime_filter_init_timer); - if (_should_build_hash_table) { - RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); + + if (state->get_task()->wake_up_by_downstream()) { + RETURN_IF_ERROR(_runtime_filter_slots->send_filter_size(state, 0, _finish_dependency)); + RETURN_IF_ERROR(_runtime_filter_slots->ignore_all_filters()); + } else { + auto* block = _shared_state->build_block.get(); + uint64_t hash_table_size = block ? block->rows() : 0; + { + SCOPED_TIMER(_runtime_filter_init_timer); + if (_should_build_hash_table) { + RETURN_IF_ERROR(_runtime_filter_slots->init_filters(state, hash_table_size)); + } + RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); + } + if (_should_build_hash_table && hash_table_size > 1) { + SCOPED_TIMER(_runtime_filter_compute_timer); + _runtime_filter_slots->insert(block); } - RETURN_IF_ERROR(_runtime_filter_slots->ignore_filters(state)); - } - if (_should_build_hash_table && hash_table_size > 1) { - SCOPED_TIMER(_runtime_filter_compute_timer); - _runtime_filter_slots->insert(block); } - SCOPED_TIMER(_publish_runtime_filter_timer); RETURN_IF_ERROR(_runtime_filter_slots->publish(!_should_build_hash_table)); - return Status::OK(); + return Base::close(state, exec_status); } bool HashJoinBuildSinkLocalState::build_unique() const { @@ -519,6 +528,7 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); + local_state._eos = eos; if (local_state._should_build_hash_table) { // If eos or have already met a null value using short-circuit strategy, we do not need to pull // data from probe side. @@ -582,6 +592,7 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* local_state.process_build_block(state, (*local_state._shared_state->build_block))); if (_shared_hashtable_controller) { _shared_hash_table_context->status = Status::OK(); + _shared_hash_table_context->complete_build_stage = true; // arena will be shared with other instances. _shared_hash_table_context->arena = local_state._shared_state->arena; _shared_hash_table_context->hash_table_variants = @@ -594,7 +605,8 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* local_state._runtime_filter_slots->copy_to_shared_context(_shared_hash_table_context); _shared_hashtable_controller->signal(node_id()); } - } else if (!local_state._should_build_hash_table) { + } else if (!local_state._should_build_hash_table && + _shared_hash_table_context->complete_build_stage) { DCHECK(_shared_hashtable_controller != nullptr); DCHECK(_shared_hash_table_context != nullptr); // the instance which is not build hash table, it's should wait the signal of hash table build finished. diff --git a/be/src/pipeline/exec/multi_cast_data_stream_sink.h b/be/src/pipeline/exec/multi_cast_data_stream_sink.h index b4886f089ef..f36edd3aaf7 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.h @@ -62,15 +62,15 @@ class MultiCastDataStreamSinkOperatorX final using Base = DataSinkOperatorX<MultiCastDataStreamSinkLocalState>; public: - MultiCastDataStreamSinkOperatorX(int sink_id, std::vector<int>& sources, - const int cast_sender_count, ObjectPool* pool, + MultiCastDataStreamSinkOperatorX(int sink_id, std::vector<int>& sources, ObjectPool* pool, const TMultiCastDataStreamSink& sink, const RowDescriptor& row_desc) : Base(sink_id, -1, sources), _pool(pool), _row_desc(row_desc), - _cast_sender_count(cast_sender_count), - _sink(sink) {} + _cast_sender_count(sources.size()), + _sink(sink), + _num_dests(sources.size()) {} ~MultiCastDataStreamSinkOperatorX() override = default; Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos) override { @@ -103,14 +103,19 @@ public: } const TMultiCastDataStreamSink& sink_node() { return _sink; } + bool count_down_destination() override { + DCHECK_GT(_num_dests, 0); + return _num_dests.fetch_sub(1) == 1; + } private: friend class MultiCastDataStreamSinkLocalState; ObjectPool* _pool; RowDescriptor _row_desc; - const int _cast_sender_count; + const size_t _cast_sender_count; const TMultiCastDataStreamSink& _sink; friend class MultiCastDataStreamSinkLocalState; + std::atomic<size_t> _num_dests; }; } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h index 8046c68ca45..dacebe52390 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h @@ -258,7 +258,6 @@ public: std::unique_ptr<RuntimeState> _runtime_state; - bool _eos = false; std::shared_ptr<Dependency> _finish_dependency; // temp structures during spilling diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.h b/be/src/pipeline/exec/spill_sort_sink_operator.h index fae5fe3270f..7e8b7b36aab 100644 --- a/be/src/pipeline/exec/spill_sort_sink_operator.h +++ b/be/src/pipeline/exec/spill_sort_sink_operator.h @@ -55,7 +55,6 @@ private: RuntimeProfile::Counter* _spill_merge_sort_timer = nullptr; - bool _eos = false; vectorized::SpillStreamSPtr _spilling_stream; std::shared_ptr<Dependency> _finish_dependency; }; diff --git a/be/src/pipeline/pipeline.cpp b/be/src/pipeline/pipeline.cpp index 8b2123bcce2..2431e64d158 100644 --- a/be/src/pipeline/pipeline.cpp +++ b/be/src/pipeline/pipeline.cpp @@ -22,6 +22,7 @@ #include <utility> #include "pipeline/exec/operator.h" +#include "pipeline/pipeline_task.h" namespace doris::pipeline { @@ -99,4 +100,14 @@ Status Pipeline::set_sink(DataSinkOperatorXPtr& sink) { return Status::OK(); } -} // namespace doris::pipeline \ No newline at end of file +void Pipeline::make_all_runnable() { + if (_sink_x->count_down_destination()) { + for (auto* task : _tasks) { + if (task) { + task->clear_blocking_state(true); + } + } + } +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline.h b/be/src/pipeline/pipeline.h index 4693f8343fd..db93e8dfe35 100644 --- a/be/src/pipeline/pipeline.h +++ b/be/src/pipeline/pipeline.h @@ -50,6 +50,7 @@ public: std::weak_ptr<PipelineFragmentContext> context) : _pipeline_id(pipeline_id), _context(std::move(context)), _num_tasks(num_tasks) { _init_profile(); + _tasks.resize(_num_tasks, nullptr); } void add_dependency(std::shared_ptr<Pipeline>& pipeline) { @@ -155,14 +156,24 @@ public: void set_children(std::shared_ptr<Pipeline> child) { _children.push_back(child); } void set_children(std::vector<std::shared_ptr<Pipeline>> children) { _children = children; } - void incr_created_tasks() { _num_tasks_created++; } + void incr_created_tasks(int i, PipelineTask* task) { + _num_tasks_created++; + _num_tasks_running++; + DCHECK_LT(i, _tasks.size()); + _tasks[i] = task; + } + + void make_all_runnable(); + void set_num_tasks(int num_tasks) { _num_tasks = num_tasks; + _tasks.resize(_num_tasks, nullptr); for (auto& op : operatorXs) { op->set_parallel_tasks(_num_tasks); } } int num_tasks() const { return _num_tasks; } + bool close_task() { return _num_tasks_running.fetch_sub(1) == 1; } std::string debug_string() { fmt::memory_buffer debug_string_buffer; @@ -243,6 +254,10 @@ private: int _num_tasks = 1; // How many tasks are already created? std::atomic<int> _num_tasks_created = 0; + // How many tasks are already created and not finished? + std::atomic<int> _num_tasks_running = 0; + // Tasks in this pipeline. + std::vector<PipelineTask*> _tasks; }; } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index 550e60c210c..e514a5d6b97 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -813,7 +813,7 @@ void PipelineFragmentContext::close_if_prepare_failed(Status /*st*/) { DCHECK(!task->is_pending_finish()); WARN_IF_ERROR(task->close(Status::OK()), fmt::format("Query {} closed since prepare failed", print_id(_query_id))); - close_a_pipeline(); + close_a_pipeline(task->pipeline_id()); } } @@ -960,7 +960,7 @@ void PipelineFragmentContext::_close_fragment_instance() { std::dynamic_pointer_cast<PipelineFragmentContext>(shared_from_this())); } -void PipelineFragmentContext::close_a_pipeline() { +void PipelineFragmentContext::close_a_pipeline(PipelineId pipeline_id) { std::lock_guard<std::mutex> l(_task_mutex); g_pipeline_tasks_count << -1; ++_closed_tasks; diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h index f60cb80359a..ce9b10be4ff 100644 --- a/be/src/pipeline/pipeline_fragment_context.h +++ b/be/src/pipeline/pipeline_fragment_context.h @@ -110,7 +110,7 @@ public: [[nodiscard]] int get_fragment_id() const { return _fragment_id; } - void close_a_pipeline(); + virtual void close_a_pipeline(PipelineId pipeline_id); virtual void clear_finished_tasks() {} diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index 47f96c5f8fa..661b16c99ed 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -291,6 +291,10 @@ public: std::string task_name() const { return fmt::format("task{}({})", _index, _pipeline->_name); } + PipelineId pipeline_id() const { return _pipeline->id(); } + + virtual void clear_blocking_state(bool wake_up_by_downstream = false) {} + protected: void _finish_p_dependency() { for (const auto& p : _pipeline->_parents) { diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.cpp b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.cpp index c6f675f3c1b..473e6625ee2 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.cpp +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.cpp @@ -52,28 +52,16 @@ Status LocalExchangeSinkLocalState::open(RuntimeState* state) { return Status::OK(); } -Status LocalExchangeSinkLocalState::close(RuntimeState* state, Status exec_status) { - if (_closed) { - return Status::OK(); - } - RETURN_IF_ERROR(Base::close(state, exec_status)); - if (exec_status.ok()) { - DCHECK(_release_count) << "Do not finish correctly! " << debug_string(0); - } - return Status::OK(); -} - std::string LocalExchangeSinkLocalState::debug_string(int indentation_level) const { fmt::memory_buffer debug_string_buffer; fmt::format_to(debug_string_buffer, "{}, _use_global_shuffle: {}, _channel_id: {}, _num_partitions: {}, " "_num_senders: {}, _num_sources: {}, " - "_running_sink_operators: {}, _running_source_operators: {}, _release_count: {}", + "_running_sink_operators: {}, _running_source_operators: {}", Base::debug_string(indentation_level), _parent->cast<LocalExchangeSinkOperatorX>()._use_global_shuffle, _channel_id, _exchanger->_num_partitions, _exchanger->_num_senders, _exchanger->_num_sources, - _exchanger->_running_sink_operators, _exchanger->_running_source_operators, - _release_count); + _exchanger->_running_sink_operators, _exchanger->_running_source_operators); return fmt::to_string(debug_string_buffer); } @@ -121,13 +109,11 @@ Status LocalExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* // If all exchange sources ended due to limit reached, current task should also finish if (local_state._exchanger->_running_source_operators == 0) { - local_state._release_count = true; local_state._shared_state->sub_running_sink_operators(); return Status::EndOfFile("receiver eof"); } if (eos) { local_state._shared_state->sub_running_sink_operators(); - local_state._release_count = true; } return Status::OK(); diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h index 9b72402abce..737fb90541d 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h @@ -38,7 +38,6 @@ public: Status init(RuntimeState* state, LocalSinkStateInfo& info) override; Status open(RuntimeState* state) override; - Status close(RuntimeState* state, Status exec_status) override; std::string debug_string(int indentation_level) const override; private: @@ -62,7 +61,6 @@ private: // Used by random passthrough exchanger int _channel_id = 0; - bool _release_count = false; }; // A single 32-bit division on a recent x64 processor has a throughput of one instruction every six cycles with a latency of 26 cycles. diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index e32176a08d1..0230085763a 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -534,6 +534,7 @@ protected: // Set to true after close() has been called. subclasses should check and set this in // close(). bool _closed = false; + std::atomic<bool> _eos = false; //NOTICE: now add a faker profile, because sometimes the profile record is useless //so we want remove some counters and timers, eg: in join node, if it's broadcast_join //and shared hash table, some counter/timer about build hash table is useless, @@ -683,6 +684,8 @@ public: virtual bool should_dry_run(RuntimeState* state) { return false; } + [[nodiscard]] virtual bool count_down_destination() { return true; } + protected: template <typename Writer, typename Parent> requires(std::is_base_of_v<vectorized::AsyncResultWriter, Writer>) diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index 53dae142a6d..91812bac925 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -127,6 +127,8 @@ PipelineXFragmentContext::~PipelineXFragmentContext() { } else { _call_back(nullptr, &st); } + _dag.clear(); + _pip_id_to_pipeline.clear(); _runtime_state.reset(); _runtime_filter_states.clear(); _runtime_filter_mgr_map.clear(); @@ -465,8 +467,7 @@ Status PipelineXFragmentContext::_create_data_sink(ObjectPool* pool, const TData } _sink.reset(new MultiCastDataStreamSinkOperatorX( - sink_id, sources, thrift_sink.multi_cast_stream_sink.sinks.size(), pool, - thrift_sink.multi_cast_stream_sink, row_desc)); + sink_id, sources, pool, thrift_sink.multi_cast_stream_sink, row_desc)); for (int i = 0; i < sender_size; ++i) { auto new_pipeline = add_pipeline(); RowDescriptor* _row_desc = nullptr; @@ -525,6 +526,7 @@ Status PipelineXFragmentContext::_build_pipeline_x_tasks( _task_runtime_states.resize(_pipelines.size()); for (size_t pip_idx = 0; pip_idx < _pipelines.size(); pip_idx++) { _task_runtime_states[pip_idx].resize(_pipelines[pip_idx]->num_tasks()); + _pip_id_to_pipeline[_pipelines[pip_idx]->id()] = _pipelines[pip_idx].get(); } auto& pipeline_id_to_profile = _runtime_state->pipeline_id_to_profile(); DCHECK(pipeline_id_to_profile.empty()); @@ -638,6 +640,8 @@ Status PipelineXFragmentContext::_build_pipeline_x_tasks( task_runtime_state.get(), ctx, pipeline_id_to_profile[pip_idx].get(), get_local_exchange_state(pipeline), i); + pipeline->incr_created_tasks(i, task.get()); + task_runtime_state->set_task(task.get()); pipeline_id_to_task.insert({pipeline->id(), task.get()}); _tasks[i].emplace_back(std::move(task)); } @@ -737,7 +741,6 @@ Status PipelineXFragmentContext::_build_pipeline_x_tasks( } } _pipeline_parent_map.clear(); - _dag.clear(); _op_id_to_le_state.clear(); return Status::OK(); @@ -1498,7 +1501,7 @@ void PipelineXFragmentContext::close_if_prepare_failed(Status st) { for (auto& t : task) { DCHECK(!t->is_pending_finish()); WARN_IF_ERROR(t->close(Status::OK()), "close_if_prepare_failed failed: "); - close_a_pipeline(); + close_a_pipeline(t->pipeline_id()); } } _query_ctx->cancel(st.to_string(), st, _fragment_id); @@ -1587,4 +1590,17 @@ std::string PipelineXFragmentContext::debug_string() { return fmt::to_string(debug_string_buffer); } + +void PipelineXFragmentContext::close_a_pipeline(PipelineId pipeline_id) { + // If all tasks of this pipeline has been closed, upstream tasks is never needed, and we just make those runnable here + DCHECK(_pip_id_to_pipeline.contains(pipeline_id)); + if (_pip_id_to_pipeline[pipeline_id]->close_task()) { + if (_dag.contains(pipeline_id)) { + for (auto dep : _dag[pipeline_id]) { + _pip_id_to_pipeline[dep]->make_all_runnable(); + } + } + } + PipelineFragmentContext::close_a_pipeline(pipeline_id); +} } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h index 8a09f64ac6b..60ac22a429e 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.h @@ -116,6 +116,8 @@ public: std::string debug_string() override; + void close_a_pipeline(PipelineId pipeline_id) override; + private: void _close_fragment_instance() override; Status _build_pipeline_x_tasks(const doris::TPipelineFragmentParams& request, @@ -222,6 +224,7 @@ private: std::map<int, std::pair<std::shared_ptr<LocalExchangeSharedState>, std::shared_ptr<Dependency>>> _op_id_to_le_state; + std::map<PipelineId, Pipeline*> _pip_id_to_pipeline; // UniqueId -> runtime mgr std::map<UniqueId, std::unique_ptr<RuntimeFilterMgr>> _runtime_filter_mgr_map; diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp index 00464e59f21..6be1c6a1492 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp @@ -66,7 +66,6 @@ PipelineXTask::PipelineXTask( if (shared_state) { _sink_shared_state = shared_state; } - pipeline->incr_created_tasks(); } Status PipelineXTask::prepare(const TPipelineInstanceParams& local_params, const TDataSink& tsink, @@ -248,7 +247,7 @@ Status PipelineXTask::execute(bool* eos) { cpu_qs->add_cpu_nanos(delta_cpu_time); } }}; - *eos = _sink->is_finished(_state); + *eos = _sink->is_finished(_state) || _wake_up_by_downstream || is_final_state(_cur_state); if (*eos) { return Status::OK(); } @@ -256,10 +255,18 @@ Status PipelineXTask::execute(bool* eos) { set_state(PipelineTaskState::BLOCKED_FOR_DEPENDENCY); return Status::OK(); } + if (_wake_up_by_downstream) { + *eos = true; + return Status::OK(); + } if (_runtime_filter_blocked_dependency() != nullptr) { set_state(PipelineTaskState::BLOCKED_FOR_RF); return Status::OK(); } + if (_wake_up_by_downstream) { + *eos = true; + return Status::OK(); + } // The status must be runnable if (!_opened) { { @@ -270,10 +277,18 @@ Status PipelineXTask::execute(bool* eos) { set_state(PipelineTaskState::BLOCKED_FOR_SOURCE); return Status::OK(); } + if (_wake_up_by_downstream) { + *eos = true; + return Status::OK(); + } if (!sink_can_write()) { set_state(PipelineTaskState::BLOCKED_FOR_SINK); return Status::OK(); } + if (_wake_up_by_downstream) { + *eos = true; + return Status::OK(); + } } Status status = Status::OK(); @@ -283,10 +298,18 @@ Status PipelineXTask::execute(bool* eos) { set_state(PipelineTaskState::BLOCKED_FOR_SOURCE); break; } + if (_wake_up_by_downstream) { + *eos = true; + return Status::OK(); + } if (!sink_can_write()) { set_state(PipelineTaskState::BLOCKED_FOR_SINK); break; } + if (_wake_up_by_downstream) { + *eos = true; + return Status::OK(); + } /// When a task is cancelled, /// its blocking state will be cleared and it will transition to a ready state (though it is not truly ready). @@ -445,12 +468,13 @@ std::string PipelineXTask::debug_string() { // If at the same time FE cancel this pipeline task and logging debug_string before _blocked_dep is cleared, // it will think _blocked_dep is not nullptr and call _blocked_dep->debug_string(). auto* cur_blocked_dep = _blocked_dep; - fmt::format_to(debug_string_buffer, - "PipelineTask[this = {}, state = {}, dry run = {}, elapse time " - "= {}s], block dependency = {}, is running = {}\noperators: ", - (void*)this, get_state_name(_cur_state), _dry_run, elapsed, - cur_blocked_dep && !_finished ? cur_blocked_dep->debug_string() : "NULL", - is_running()); + fmt::format_to( + debug_string_buffer, + "PipelineTask[this = {}, state = {}, dry run = {}, elapse time " + "= {}s], _wake_up_by_downstream = {}, block dependency = {}, is running = " + "{}\noperators: ", + (void*)this, get_state_name(_cur_state), _dry_run, elapsed, _wake_up_by_downstream, + cur_blocked_dep && !_finished ? cur_blocked_dep->debug_string() : "NULL", is_running()); for (size_t i = 0; i < _operators.size(); i++) { fmt::format_to(debug_string_buffer, "\n{}", _opened && !_finished ? _operators[i]->debug_string(_state, i) diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.h b/be/src/pipeline/pipeline_x/pipeline_x_task.h index 47746b76fb0..42dca15076a 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.h @@ -138,7 +138,8 @@ public: int task_id() const { return _index; }; - void clear_blocking_state() { + void clear_blocking_state(bool wake_up_by_downstream = false) override { + _wake_up_by_downstream = _wake_up_by_downstream || wake_up_by_downstream; _state->get_query_ctx()->get_execution_dependency()->set_always_ready(); // We use a lock to assure all dependencies are not deconstructed here. std::unique_lock<std::mutex> lc(_dependency_lock); @@ -176,6 +177,8 @@ public: static bool should_revoke_memory(RuntimeState* state, int64_t revocable_mem_bytes); + bool wake_up_by_downstream() const { return _wake_up_by_downstream; } + private: friend class RuntimeFilterDependency; Dependency* _write_blocked_dependency() { @@ -252,6 +255,7 @@ private: std::atomic<bool> _finished {false}; std::mutex _dependency_lock; + std::atomic<bool> _wake_up_by_downstream = false; }; } // namespace doris::pipeline diff --git a/be/src/pipeline/task_scheduler.cpp b/be/src/pipeline/task_scheduler.cpp index de697469575..93cf2d9dd99 100644 --- a/be/src/pipeline/task_scheduler.cpp +++ b/be/src/pipeline/task_scheduler.cpp @@ -258,7 +258,7 @@ void _close_task(PipelineTask* task, PipelineTaskState state, Status exec_status task->set_close_pipeline_time(); task->finalize(); task->set_running(false); - task->fragment_context()->close_a_pipeline(); + task->fragment_context()->close_a_pipeline(task->pipeline_id()); } void TaskScheduler::_do_work(size_t index) { diff --git a/be/src/runtime/runtime_filter_mgr.cpp b/be/src/runtime/runtime_filter_mgr.cpp index 24baf9b6c97..640cece8fb3 100644 --- a/be/src/runtime/runtime_filter_mgr.cpp +++ b/be/src/runtime/runtime_filter_mgr.cpp @@ -145,6 +145,7 @@ Status RuntimeFilterMgr::register_local_merge_producer_filter( RETURN_IF_ERROR(IRuntimeFilter::create(_state, &_pool, &desc, &options, RuntimeFilterRole::PRODUCER, -1, &merge_filter, build_bf_exactly, true)); + merge_filter->set_ignored(); iter->second.filters.emplace_back(merge_filter); } iter->second.merge_time++; @@ -166,7 +167,6 @@ Status RuntimeFilterMgr::get_local_merge_producer_filters( } *local_merge_filters = &iter->second; DCHECK(!iter->second.filters.empty()); - DCHECK_GT(iter->second.merge_time, 0); return Status::OK(); } @@ -253,6 +253,7 @@ Status RuntimeFilterMergeControllerEntity::_init_with_desc( auto filter_id = runtime_filter_desc->filter_id; RETURN_IF_ERROR(cnt_val->filter->init_with_desc(&cnt_val->runtime_filter_desc, query_options, -1, false)); + cnt_val->filter->set_ignored(); _filter_map.emplace(filter_id, cnt_val); return Status::OK(); } @@ -272,6 +273,7 @@ Status RuntimeFilterMergeControllerEntity::_init_with_desc( new IRuntimeFilter(_state, &_state->get_query_ctx()->obj_pool, runtime_filter_desc)); auto filter_id = runtime_filter_desc->filter_id; RETURN_IF_ERROR(cnt_val->filter->init_with_desc(&cnt_val->runtime_filter_desc, query_options)); + cnt_val->filter->set_ignored(); std::unique_lock<std::shared_mutex> guard(_filter_map_mutex); _filter_map.emplace(filter_id, cnt_val); diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 8243faa37ae..3c8d181f16f 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -568,6 +568,10 @@ public: : 0; } + void set_task(pipeline::PipelineXTask* task) { _task = task; } + + pipeline::PipelineXTask* get_task() const { return _task; } + inline bool enable_delete_sub_pred_v2() const { return _query_options.__isset.enable_delete_sub_predicate_v2 && _query_options.enable_delete_sub_predicate_v2; @@ -784,6 +788,7 @@ private: // prohibit copies RuntimeState(const RuntimeState&); + pipeline::PipelineXTask* _task; vectorized::ColumnInt64* _partial_update_auto_inc_column; }; diff --git a/be/src/vec/exec/join/vhash_join_node.h b/be/src/vec/exec/join/vhash_join_node.h index 20245be06c2..f23a992049f 100644 --- a/be/src/vec/exec/join/vhash_join_node.h +++ b/be/src/vec/exec/join/vhash_join_node.h @@ -91,7 +91,7 @@ Status process_runtime_filter_build(RuntimeState* state, Block* block, Parent* p } { SCOPED_TIMER(parent->_publish_runtime_filter_timer); - RETURN_IF_ERROR(parent->_runtime_filter_slots->publish()); + RETURN_IF_ERROR(parent->_runtime_filter_slots->publish(false)); } return Status::OK(); diff --git a/be/src/vec/runtime/shared_hash_table_controller.h b/be/src/vec/runtime/shared_hash_table_controller.h index b685b984bf3..4581bb762e8 100644 --- a/be/src/vec/runtime/shared_hash_table_controller.h +++ b/be/src/vec/runtime/shared_hash_table_controller.h @@ -66,6 +66,7 @@ struct SharedHashTableContext { std::map<int, RuntimeFilterContextSPtr> runtime_filters; std::atomic<bool> signaled = false; bool short_circuit_for_null_in_probe_side = false; + std::atomic<bool> complete_build_stage = false; }; using SharedHashTableContextPtr = std::shared_ptr<SharedHashTableContext>; diff --git a/be/test/exprs/runtime_filter_test.cpp b/be/test/exprs/runtime_filter_test.cpp index 36d7cd885dd..554438fc6bf 100644 --- a/be/test/exprs/runtime_filter_test.cpp +++ b/be/test/exprs/runtime_filter_test.cpp @@ -103,11 +103,6 @@ IRuntimeFilter* create_runtime_filter(TRuntimeFilterType::type type, TQueryOptio EXPECT_TRUE(status.ok()) << status.to_string(); - if (auto bf = runtime_filter->get_bloomfilter()) { - status = bf->init_with_fixed_length(); - EXPECT_TRUE(status.ok()) << status.to_string(); - } - return status.ok() ? runtime_filter : nullptr; } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org