From 5442e8d1fcc87d184b54dd45249b341ef5534926 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Wed, 22 Nov 2023 12:50:39 +0800 Subject: [PATCH] [pipelineX](dependency) split different dependencies (#27366) --- .../exec/aggregation_sink_operator.cpp | 106 +++- .../pipeline/exec/aggregation_sink_operator.h | 43 +- .../exec/aggregation_source_operator.cpp | 196 ++++--- .../exec/aggregation_source_operator.h | 43 +- .../pipeline/exec/analytic_sink_operator.cpp | 131 ++++- be/src/pipeline/exec/analytic_sink_operator.h | 29 +- .../exec/analytic_source_operator.cpp | 136 ++++- .../pipeline/exec/analytic_source_operator.h | 26 +- be/src/pipeline/exec/data_queue.cpp | 25 +- be/src/pipeline/exec/data_queue.h | 18 +- ...ct_streaming_aggregation_sink_operator.cpp | 4 +- ...inct_streaming_aggregation_sink_operator.h | 9 +- ..._streaming_aggregation_source_operator.cpp | 4 +- be/src/pipeline/exec/exchange_sink_buffer.cpp | 6 +- be/src/pipeline/exec/exchange_sink_buffer.h | 6 +- .../pipeline/exec/exchange_sink_operator.cpp | 9 +- be/src/pipeline/exec/exchange_sink_operator.h | 18 +- .../exec/exchange_source_operator.cpp | 2 +- .../pipeline/exec/exchange_source_operator.h | 20 +- be/src/pipeline/exec/hashjoin_build_sink.cpp | 73 ++- be/src/pipeline/exec/hashjoin_build_sink.h | 23 +- .../pipeline/exec/hashjoin_probe_operator.cpp | 53 +- .../pipeline/exec/hashjoin_probe_operator.h | 15 +- .../exec/join_build_sink_operator.cpp | 5 +- be/src/pipeline/exec/join_probe_operator.cpp | 4 +- .../exec/multi_cast_data_stream_sink.cpp | 2 +- .../exec/multi_cast_data_stream_sink.h | 12 +- .../exec/multi_cast_data_stream_source.cpp | 2 +- .../exec/multi_cast_data_stream_source.h | 15 +- .../exec/multi_cast_data_streamer.cpp | 7 +- .../pipeline/exec/multi_cast_data_streamer.h | 6 +- .../exec/nested_loop_join_build_operator.cpp | 6 +- .../exec/nested_loop_join_build_operator.h | 10 +- .../exec/nested_loop_join_probe_operator.cpp | 6 +- .../exec/nested_loop_join_probe_operator.h | 10 +- .../exec/partition_sort_sink_operator.cpp | 4 +- .../exec/partition_sort_sink_operator.h | 12 +- .../exec/partition_sort_source_operator.cpp | 4 +- .../exec/partition_sort_source_operator.h | 20 +- be/src/pipeline/exec/result_sink_operator.cpp | 2 +- be/src/pipeline/exec/result_sink_operator.h | 6 +- be/src/pipeline/exec/scan_operator.cpp | 4 +- be/src/pipeline/exec/scan_operator.h | 23 +- .../pipeline/exec/set_probe_sink_operator.cpp | 9 +- .../pipeline/exec/set_probe_sink_operator.h | 21 +- be/src/pipeline/exec/set_sink_operator.cpp | 7 +- be/src/pipeline/exec/set_sink_operator.h | 15 +- be/src/pipeline/exec/set_source_operator.cpp | 10 +- be/src/pipeline/exec/set_source_operator.h | 11 +- be/src/pipeline/exec/sort_sink_operator.cpp | 4 +- be/src/pipeline/exec/sort_sink_operator.h | 11 +- be/src/pipeline/exec/sort_source_operator.cpp | 2 +- be/src/pipeline/exec/sort_source_operator.h | 9 +- .../streaming_aggregation_sink_operator.cpp | 7 +- .../streaming_aggregation_sink_operator.h | 14 +- .../streaming_aggregation_source_operator.cpp | 4 +- be/src/pipeline/exec/union_sink_operator.h | 13 +- .../pipeline/exec/union_source_operator.cpp | 15 +- be/src/pipeline/exec/union_source_operator.h | 24 +- be/src/pipeline/pipeline_x/dependency.cpp | 431 +------------- be/src/pipeline/pipeline_x/dependency.h | 542 ++++-------------- .../local_exchange_sink_operator.h | 15 +- .../local_exchange_source_operator.cpp | 2 +- .../local_exchange_source_operator.h | 15 +- be/src/pipeline/pipeline_x/operator.cpp | 114 ++-- be/src/pipeline/pipeline_x/operator.h | 32 +- .../pipeline/pipeline_x/pipeline_x_task.cpp | 11 +- be/src/pipeline/pipeline_x/pipeline_x_task.h | 19 +- be/src/runtime/buffer_control_block.cpp | 4 +- be/src/vec/exec/scan/pip_scanner_context.h | 8 +- be/src/vec/exec/scan/scanner_context.cpp | 2 +- be/src/vec/exec/scan/scanner_context.h | 6 +- .../runtime/shared_hash_table_controller.cpp | 4 +- be/src/vec/runtime/vdata_stream_recvr.cpp | 18 +- .../vec/sink/writer/async_result_writer.cpp | 18 +- be/src/vec/sink/writer/async_result_writer.h | 10 +- 76 files changed, 1307 insertions(+), 1275 deletions(-) diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index 9eb49d234f..dfde3a0299 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -72,10 +72,10 @@ Status AggSinkLocalState::init(RuntimeState* state, _agg_data = Base::_shared_state->agg_data.get(); _agg_arena_pool = Base::_shared_state->agg_arena_pool.get(); auto& p = Base::_parent->template cast(); - Base::_dependency->set_align_aggregate_states(p._align_aggregate_states); - Base::_dependency->set_total_size_of_aggregate_states(p._total_size_of_aggregate_states); - Base::_dependency->set_offsets_of_aggregate_states(p._offsets_of_aggregate_states); - Base::_dependency->set_make_nullable_keys(p._make_nullable_keys); + Base::_shared_state->align_aggregate_states = p._align_aggregate_states; + Base::_shared_state->total_size_of_aggregate_states = p._total_size_of_aggregate_states; + Base::_shared_state->offsets_of_aggregate_states = p._offsets_of_aggregate_states; + Base::_shared_state->make_nullable_keys = p._make_nullable_keys; Base::_shared_state->init_spill_partition_helper(p._spill_partition_count_bits); for (auto& evaluator : p._aggregate_evaluators) { Base::_shared_state->aggregate_evaluators.push_back(evaluator->clone(state, p._pool)); @@ -169,8 +169,26 @@ Status AggSinkLocalState::open(RuntimeState* state) { // this could cause unable to get JVM if (Base::_shared_state->probe_expr_ctxs.empty()) { // _create_agg_status may acquire a lot of memory, may allocate failed when memory is very few - RETURN_IF_CATCH_EXCEPTION( - static_cast(Base::_dependency->create_agg_status(_agg_data->without_key))); + RETURN_IF_CATCH_EXCEPTION(static_cast(_create_agg_status(_agg_data->without_key))); + } + return Status::OK(); +} + +template +Status AggSinkLocalState::_create_agg_status( + vectorized::AggregateDataPtr data) { + auto& shared_state = *Base::_shared_state; + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + try { + shared_state.aggregate_evaluators[i]->create( + data + shared_state.offsets_of_aggregate_states[i]); + } catch (...) { + for (int j = 0; j < i; ++j) { + shared_state.aggregate_evaluators[j]->destroy( + data + shared_state.offsets_of_aggregate_states[j]); + } + throw; + } } return Status::OK(); } @@ -221,24 +239,35 @@ void AggSinkLocalState::_update_memusage_with_serialize auto arena_memory_usage = _agg_arena_pool->size() + Base::_shared_state->aggregate_data_container->memory_usage() - - Base::_dependency->mem_usage_record().used_in_arena; - Base::_dependency->mem_tracker()->consume(arena_memory_usage); - Base::_dependency->mem_tracker()->consume( + Base::_shared_state->mem_usage_record.used_in_arena; + Base::_shared_state->mem_tracker->consume(arena_memory_usage); + Base::_shared_state->mem_tracker->consume( data.get_buffer_size_in_bytes() - - Base::_dependency->mem_usage_record().used_in_state); + Base::_shared_state->mem_usage_record.used_in_state); _serialize_key_arena_memory_usage->add(arena_memory_usage); COUNTER_UPDATE(_hash_table_memory_usage, data.get_buffer_size_in_bytes() - - Base::_dependency->mem_usage_record().used_in_state); - Base::_dependency->mem_usage_record().used_in_state = + Base::_shared_state->mem_usage_record.used_in_state); + Base::_shared_state->mem_usage_record.used_in_state = data.get_buffer_size_in_bytes(); - Base::_dependency->mem_usage_record().used_in_arena = + Base::_shared_state->mem_usage_record.used_in_arena = _agg_arena_pool->size() + Base::_shared_state->aggregate_data_container->memory_usage(); }, _agg_data->method_variant); } +template +Status AggSinkLocalState::_destroy_agg_status( + vectorized::AggregateDataPtr data) { + auto& shared_state = *Base::_shared_state; + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + shared_state.aggregate_evaluators[i]->function()->destroy( + data + shared_state.offsets_of_aggregate_states[i]); + } + return Status::OK(); +} + template template Status AggSinkLocalState::_merge_with_serialized_key_helper( @@ -405,10 +434,10 @@ Status AggSinkLocalState::_merge_without_key(vectorized template void AggSinkLocalState::_update_memusage_without_key() { auto arena_memory_usage = - _agg_arena_pool->size() - Base::_dependency->mem_usage_record().used_in_arena; - Base::_dependency->mem_tracker()->consume(arena_memory_usage); + _agg_arena_pool->size() - Base::_shared_state->mem_usage_record.used_in_arena; + Base::_shared_state->mem_tracker->consume(arena_memory_usage); _serialize_key_arena_memory_usage->add(arena_memory_usage); - Base::_dependency->mem_usage_record().used_in_arena = _agg_arena_pool->size(); + Base::_shared_state->mem_usage_record.used_in_arena = _agg_arena_pool->size(); } template @@ -475,7 +504,7 @@ Status AggSinkLocalState::_execute_with_serialized_key_ Base::_parent->template cast()._limit; if (_reach_limit && Base::_parent->template cast()._can_short_circuit) { - Base::_dependency->set_ready_for_read(); + Base::_dependency->set_ready_to_read(); return Status::Error(""); } } @@ -506,7 +535,7 @@ void AggSinkLocalState::_emplace_into_hash_table( HashMethodType::try_presis_key(key, origin, *_agg_arena_pool); auto mapped = Base::_shared_state->aggregate_data_container->append_data(origin); - auto st = Base::_dependency->create_agg_status(mapped); + auto st = _create_agg_status(mapped); if (!st) { throw Exception(st.code(), st.to_string()); } @@ -519,7 +548,7 @@ void AggSinkLocalState::_emplace_into_hash_table( ._total_size_of_aggregate_states, Base::_parent->template cast() ._align_aggregate_states); - auto st = Base::_dependency->create_agg_status(mapped); + auto st = _create_agg_status(mapped); if (!st) { throw Exception(st.code(), st.to_string()); } @@ -631,6 +660,35 @@ void AggSinkLocalState::_init_hash_method( } } +template +Status AggSinkLocalState::_reset_hash_table() { + auto& ss = *Base::_shared_state; + return std::visit( + [&](auto&& agg_method) { + auto& hash_table = *agg_method.hash_table; + using HashTableType = std::decay_t; + + agg_method.reset(); + + hash_table.for_each_mapped([&](auto& mapped) { + if (mapped) { + static_cast(_destroy_agg_status(mapped)); + mapped = nullptr; + } + }); + + ss.aggregate_data_container.reset(new vectorized::AggregateDataContainer( + sizeof(typename HashTableType::key_type), + ((ss.total_size_of_aggregate_states + ss.align_aggregate_states - 1) / + ss.align_aggregate_states) * + ss.align_aggregate_states)); + hash_table = HashTableType(); + ss.agg_arena_pool.reset(new vectorized::Arena); + return Status::OK(); + }, + ss.agg_data->method_variant); +} + template Status AggSinkLocalState::try_spill_disk(bool eos) { if (Base::_parent->template cast()._external_agg_bytes_threshold == @@ -651,7 +709,7 @@ Status AggSinkLocalState::try_spill_disk(bool eos) { } RETURN_IF_ERROR(_spill_hash_table(agg_method, hash_table)); - return Base::_dependency->reset_hash_table(); + return _reset_hash_table(); }, _agg_data->method_variant); } @@ -797,7 +855,7 @@ Status AggSinkOperatorX::sink(doris::RuntimeState* state, static_cast(local_state.try_spill_disk(true)); RETURN_IF_ERROR(local_state._shared_state->spill_context.prepare_for_reading()); } - local_state._dependency->set_ready_for_read(); + local_state._dependency->set_ready_to_read(); } return Status::OK(); } @@ -825,7 +883,7 @@ class DistinctStreamingAggSinkLocalState; template class AggSinkOperatorX; template class AggSinkOperatorX; template class AggSinkOperatorX; -template class AggSinkLocalState; -template class AggSinkLocalState; -template class AggSinkLocalState; +template class AggSinkLocalState; +template class AggSinkLocalState; +template class AggSinkLocalState; } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h index 9615228ca1..3f1ce26036 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -45,6 +45,40 @@ public: bool can_write() override { return true; } }; +class AggSinkDependency final : public Dependency { +public: + using SharedState = AggSharedState; + AggSinkDependency(int id, int node_id) : Dependency(id, node_id, "AggSinkDependency", true) {} + ~AggSinkDependency() override = default; + + void set_ready() override { + if (_is_streaming_agg_state()) { + if (((SharedState*)Dependency::_shared_state.get()) + ->data_queue->has_enough_space_to_push()) { + Dependency::set_ready(); + } + } else { + Dependency::set_ready(); + } + } + + void block() override { + if (_is_streaming_agg_state()) { + if (!((SharedState*)Dependency::_shared_state.get()) + ->data_queue->has_enough_space_to_push()) { + Dependency::block(); + } + } else { + Dependency::block(); + } + } + +private: + bool _is_streaming_agg_state() { + return ((SharedState*)Dependency::_shared_state.get())->data_queue != nullptr; + } +}; + template class AggSinkOperatorX; @@ -145,7 +179,7 @@ protected: for (size_t i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { Base::_shared_state->aggregate_evaluators[i]->function()->serialize_to_column( Base::_shared_state->values, - Base::_dependency->offsets_of_aggregate_states()[i], value_columns[i], + Base::_shared_state->offsets_of_aggregate_states[i], value_columns[i], num_rows); } @@ -167,6 +201,7 @@ protected: return Status::OK(); } + Status _destroy_agg_status(vectorized::AggregateDataPtr data); template Status _spill_hash_table(HashTableCtxType& agg_method, HashTableType& hash_table) { vectorized::Block block; @@ -254,6 +289,8 @@ protected: return Status::OK(); } + Status _create_agg_status(vectorized::AggregateDataPtr data); + Status _reset_hash_table(); // We should call this function only at 1st phase. // 1st phase: is_merge=true, only have one SlotRef. // 2nd phase: is_merge=false, maybe have multiple exprs. @@ -299,13 +336,13 @@ protected: }; class BlockingAggSinkLocalState - : public AggSinkLocalState { + : public AggSinkLocalState { public: ENABLE_FACTORY_CREATOR(BlockingAggSinkLocalState); using Parent = AggSinkOperatorX; BlockingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : AggSinkLocalState(parent, state) {} + : AggSinkLocalState(parent, state) {} ~BlockingAggSinkLocalState() override = default; }; diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index bb290cfbd2..dc09e8a268 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -24,8 +24,7 @@ #include "pipeline/exec/streaming_aggregation_source_operator.h" #include "vec//utils/util.hpp" -namespace doris { -namespace pipeline { +namespace doris::pipeline { OPERATOR_CODE_GENERATOR(AggSourceOperator, SourceOperator) @@ -50,6 +49,11 @@ Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { _serialize_data_timer = ADD_TIMER(profile(), "SerializeDataTime"); _hash_table_size_counter = ADD_COUNTER(profile(), "HashTableSize", TUnit::UNIT); auto& p = _parent->template cast(); + if (p._is_streaming) { + _shared_state->data_queue.reset(new DataQueue(1)); + _shared_state->data_queue->set_dependency(_dependency, + info.upstream_dependencies.front().get()); + } if (p._without_key) { if (p._needs_finalize) { _executor.get_result = std::bind(&AggLocalState::_get_without_key_result, this, @@ -79,18 +83,27 @@ Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { return Status::OK(); } +Status AggLocalState::_destroy_agg_status(vectorized::AggregateDataPtr data) { + auto& shared_state = *Base::_shared_state; + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + shared_state.aggregate_evaluators[i]->function()->destroy( + data + shared_state.offsets_of_aggregate_states[i]); + } + return Status::OK(); +} + void AggLocalState::_close_with_serialized_key() { std::visit( [&](auto&& agg_method) -> void { auto& data = *agg_method.hash_table; data.for_each_mapped([&](auto& mapped) { if (mapped) { - static_cast(_dependency->destroy_agg_status(mapped)); + static_cast(_destroy_agg_status(mapped)); mapped = nullptr; } }); if (data.has_null_key_data()) { - auto st = _dependency->destroy_agg_status( + auto st = _destroy_agg_status( data.template get_null_key_data()); if (!st) { throw Exception(st.code(), st.to_string()); @@ -98,7 +111,7 @@ void AggLocalState::_close_with_serialized_key() { } }, _agg_data->method_variant); - _dependency->release_tracker(); + _release_tracker(); } void AggLocalState::_close_without_key() { @@ -106,10 +119,10 @@ void AggLocalState::_close_without_key() { //but finally call close to destory agg data, if agg data has bitmapValue //will be core dump, it's not initialized if (_agg_data_created_without_key) { - static_cast(_dependency->destroy_agg_status(_agg_data->without_key)); + static_cast(_destroy_agg_status(_agg_data->without_key)); _agg_data_created_without_key = false; } - _dependency->release_tracker(); + _release_tracker(); } Status AggLocalState::_serialize_with_serialized_key_result(RuntimeState* state, @@ -134,8 +147,8 @@ Status AggLocalState::_serialize_with_serialized_key_result_with_spilt_data( _shared_state->spill_partition_helper->partition_count) { break; } - RETURN_IF_ERROR(_dependency->reset_hash_table()); - RETURN_IF_ERROR(_dependency->merge_spilt_data()); + RETURN_IF_ERROR(_reset_hash_table()); + RETURN_IF_ERROR(_merge_spilt_data()); _shared_state->aggregate_data_container->init_once(); } @@ -150,17 +163,46 @@ Status AggLocalState::_serialize_with_serialized_key_result_with_spilt_data( return Status::OK(); } +Status AggLocalState::_reset_hash_table() { + auto& ss = *Base::_shared_state; + return std::visit( + [&](auto&& agg_method) { + auto& hash_table = *agg_method.hash_table; + using HashTableType = std::decay_t; + + agg_method.reset(); + + hash_table.for_each_mapped([&](auto& mapped) { + if (mapped) { + static_cast(_destroy_agg_status(mapped)); + mapped = nullptr; + } + }); + + ss.aggregate_data_container.reset(new vectorized::AggregateDataContainer( + sizeof(typename HashTableType::key_type), + ((ss.total_size_of_aggregate_states + ss.align_aggregate_states - 1) / + ss.align_aggregate_states) * + ss.align_aggregate_states)); + hash_table = HashTableType(); + ss.agg_arena_pool.reset(new vectorized::Arena); + return Status::OK(); + }, + ss.agg_data->method_variant); +} + Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { SCOPED_TIMER(_serialize_result_timer); + auto& shared_state = *_shared_state; int key_size = _shared_state->probe_expr_ctxs.size(); int agg_size = _shared_state->aggregate_evaluators.size(); vectorized::MutableColumns value_columns(agg_size); vectorized::DataTypes value_data_types(agg_size); // non-nullable column(id in `_make_nullable_keys`) will be converted to nullable. - bool mem_reuse = _dependency->make_nullable_keys().empty() && block->mem_reuse(); + bool mem_reuse = shared_state.make_nullable_keys.empty() && block->mem_reuse(); vectorized::MutableColumns key_columns; for (int i = 0; i < key_size; ++i) { @@ -168,7 +210,7 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta key_columns.emplace_back(std::move(*block->get_by_position(i).column).mutate()); } else { key_columns.emplace_back( - _shared_state->probe_expr_ctxs[i]->root()->data_type()->create_column()); + shared_state.probe_expr_ctxs[i]->root()->data_type()->create_column()); } } @@ -180,20 +222,20 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta const auto size = std::min(data.size(), size_t(state->batch_size())); using KeyType = std::decay_tget_first())>; std::vector keys(size); - if (_shared_state->values.size() < size + 1) { - _shared_state->values.resize(size + 1); + if (shared_state.values.size() < size + 1) { + shared_state.values.resize(size + 1); } size_t num_rows = 0; - _shared_state->aggregate_data_container->init_once(); - auto& iter = _shared_state->aggregate_data_container->iterator; + shared_state.aggregate_data_container->init_once(); + auto& iter = shared_state.aggregate_data_container->iterator; { SCOPED_TIMER(_hash_table_iterate_timer); - while (iter != _shared_state->aggregate_data_container->end() && + while (iter != shared_state.aggregate_data_container->end() && num_rows < state->batch_size()) { keys[num_rows] = iter.template get_key(); - _shared_state->values[num_rows] = iter.get_aggregate_data(); + shared_state.values[num_rows] = iter.get_aggregate_data(); ++iter; ++num_rows; } @@ -204,7 +246,7 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta agg_method.insert_keys_into_columns(keys, key_columns, num_rows); } - if (iter == _shared_state->aggregate_data_container->end()) { + if (iter == shared_state.aggregate_data_container->end()) { if (agg_method.hash_table->has_null_key_data()) { // only one key of group by support wrap null key // here need additional processing logic on the null key / value @@ -212,7 +254,7 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta DCHECK(key_columns[0]->is_nullable()); if (agg_method.hash_table->has_null_key_data()) { key_columns[0]->insert_data(nullptr, 0); - _shared_state->values[num_rows] = + shared_state.values[num_rows] = agg_method.hash_table->template get_null_key_data< vectorized::AggregateDataPtr>(); ++num_rows; @@ -225,8 +267,8 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta { SCOPED_TIMER(_serialize_data_timer); - for (size_t i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - value_data_types[i] = _shared_state->aggregate_evaluators[i] + for (size_t i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + value_data_types[i] = shared_state.aggregate_evaluators[i] ->function() ->get_serialized_type(); if (mem_reuse) { @@ -234,14 +276,13 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta std::move(*block->get_by_position(i + key_size).column) .mutate(); } else { - value_columns[i] = _shared_state->aggregate_evaluators[i] + value_columns[i] = shared_state.aggregate_evaluators[i] ->function() ->create_serialize_column(); } - _shared_state->aggregate_evaluators[i]->function()->serialize_to_column( - _shared_state->values, - _dependency->offsets_of_aggregate_states()[i], value_columns[i], - num_rows); + shared_state.aggregate_evaluators[i]->function()->serialize_to_column( + shared_state.values, shared_state.offsets_of_aggregate_states[i], + value_columns[i], num_rows); } } }, @@ -250,10 +291,9 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta if (!mem_reuse) { vectorized::ColumnsWithTypeAndName columns_with_schema; for (int i = 0; i < key_size; ++i) { - columns_with_schema.emplace_back( - std::move(key_columns[i]), - _shared_state->probe_expr_ctxs[i]->root()->data_type(), - _shared_state->probe_expr_ctxs[i]->root()->expr_name()); + columns_with_schema.emplace_back(std::move(key_columns[i]), + shared_state.probe_expr_ctxs[i]->root()->data_type(), + shared_state.probe_expr_ctxs[i]->root()->expr_name()); } for (int i = 0; i < agg_size; ++i) { columns_with_schema.emplace_back(std::move(value_columns[i]), value_data_types[i], ""); @@ -285,8 +325,8 @@ Status AggLocalState::_get_result_with_spilt_data(RuntimeState* state, vectorize _shared_state->spill_partition_helper->partition_count) { break; } - RETURN_IF_ERROR(_dependency->reset_hash_table()); - RETURN_IF_ERROR(_dependency->merge_spilt_data()); + RETURN_IF_ERROR(_reset_hash_table()); + RETURN_IF_ERROR(_merge_spilt_data()); _shared_state->aggregate_data_container->init_once(); } @@ -301,15 +341,37 @@ Status AggLocalState::_get_result_with_spilt_data(RuntimeState* state, vectorize return Status::OK(); } +Status AggLocalState::_merge_spilt_data() { + CHECK(!_shared_state->spill_context.stream_ids.empty()); + + for (auto& reader : _shared_state->spill_context.readers) { + CHECK_LT(_shared_state->spill_context.read_cursor, reader->block_count()); + reader->seek(_shared_state->spill_context.read_cursor); + vectorized::Block block; + bool eos = false; + RETURN_IF_ERROR(reader->read(&block, &eos)); + + // TODO + // if (!block.empty()) { + // auto st = _merge_with_serialized_key_helper( + // &block); + // RETURN_IF_ERROR(st); + // } + } + _shared_state->spill_context.read_cursor++; + return Status::OK(); +} + Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { + auto& shared_state = *_shared_state; // non-nullable column(id in `_make_nullable_keys`) will be converted to nullable. - bool mem_reuse = _dependency->make_nullable_keys().empty() && block->mem_reuse(); + bool mem_reuse = shared_state.make_nullable_keys.empty() && block->mem_reuse(); auto columns_with_schema = vectorized::VectorizedUtils::create_columns_with_type_and_name( _parent->cast()._row_descriptor); - int key_size = _shared_state->probe_expr_ctxs.size(); + int key_size = shared_state.probe_expr_ctxs.size(); vectorized::MutableColumns key_columns; for (int i = 0; i < key_size; ++i) { @@ -336,20 +398,20 @@ Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* st const auto size = std::min(data.size(), size_t(state->batch_size())); using KeyType = std::decay_tget_first())>; std::vector keys(size); - if (_shared_state->values.size() < size) { - _shared_state->values.resize(size); + if (shared_state.values.size() < size) { + shared_state.values.resize(size); } size_t num_rows = 0; - _shared_state->aggregate_data_container->init_once(); - auto& iter = _shared_state->aggregate_data_container->iterator; + shared_state.aggregate_data_container->init_once(); + auto& iter = shared_state.aggregate_data_container->iterator; { SCOPED_TIMER(_hash_table_iterate_timer); - while (iter != _shared_state->aggregate_data_container->end() && + while (iter != shared_state.aggregate_data_container->end() && num_rows < state->batch_size()) { keys[num_rows] = iter.template get_key(); - _shared_state->values[num_rows] = iter.get_aggregate_data(); + shared_state.values[num_rows] = iter.get_aggregate_data(); ++iter; ++num_rows; } @@ -360,13 +422,13 @@ Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* st agg_method.insert_keys_into_columns(keys, key_columns, num_rows); } - for (size_t i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - _shared_state->aggregate_evaluators[i]->insert_result_info_vec( - _shared_state->values, _dependency->offsets_of_aggregate_states()[i], + for (size_t i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + shared_state.aggregate_evaluators[i]->insert_result_info_vec( + shared_state.values, shared_state.offsets_of_aggregate_states[i], value_columns[i].get(), num_rows); } - if (iter == _shared_state->aggregate_data_container->end()) { + if (iter == shared_state.aggregate_data_container->end()) { if (agg_method.hash_table->has_null_key_data()) { // only one key of group by support wrap null key // here need additional processing logic on the null key / value @@ -376,9 +438,9 @@ Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* st key_columns[0]->insert_data(nullptr, 0); auto mapped = agg_method.hash_table->template get_null_key_data< vectorized::AggregateDataPtr>(); - for (size_t i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) - _shared_state->aggregate_evaluators[i]->insert_result_info( - mapped + _dependency->offsets_of_aggregate_states()[i], + for (size_t i = 0; i < shared_state.aggregate_evaluators.size(); ++i) + shared_state.aggregate_evaluators[i]->insert_result_info( + mapped + shared_state.offsets_of_aggregate_states[i], value_columns[i].get()); source_state = SourceState::FINISHED; } @@ -407,6 +469,7 @@ Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* st Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { + auto& shared_state = *_shared_state; // 1. `child(0)->rows_returned() == 0` mean not data from child // in level two aggregation node should return NULL result // level one aggregation node set `eos = true` return directly @@ -418,26 +481,26 @@ Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Bl block->clear(); DCHECK(_agg_data->without_key != nullptr); - int agg_size = _shared_state->aggregate_evaluators.size(); + int agg_size = shared_state.aggregate_evaluators.size(); vectorized::MutableColumns value_columns(agg_size); std::vector data_types(agg_size); // will serialize data to string column - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - data_types[i] = _shared_state->aggregate_evaluators[i]->function()->get_serialized_type(); + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + data_types[i] = shared_state.aggregate_evaluators[i]->function()->get_serialized_type(); value_columns[i] = - _shared_state->aggregate_evaluators[i]->function()->create_serialize_column(); + shared_state.aggregate_evaluators[i]->function()->create_serialize_column(); } - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - _shared_state->aggregate_evaluators[i]->function()->serialize_without_key_to_column( - _agg_data->without_key + _dependency->offsets_of_aggregate_states()[i], + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + shared_state.aggregate_evaluators[i]->function()->serialize_without_key_to_column( + _agg_data->without_key + shared_state.offsets_of_aggregate_states[i], *value_columns[i]); } { vectorized::ColumnsWithTypeAndName data_with_schema; - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { vectorized::ColumnWithTypeAndName column_with_schema = {nullptr, data_types[i], ""}; data_with_schema.push_back(std::move(column_with_schema)); } @@ -451,24 +514,25 @@ Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Bl Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { + auto& shared_state = *_shared_state; DCHECK(_agg_data->without_key != nullptr); block->clear(); auto& p = _parent->cast(); *block = vectorized::VectorizedUtils::create_empty_columnswithtypename(p._row_descriptor); - int agg_size = _shared_state->aggregate_evaluators.size(); + int agg_size = shared_state.aggregate_evaluators.size(); vectorized::MutableColumns columns(agg_size); std::vector data_types(agg_size); - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - data_types[i] = _shared_state->aggregate_evaluators[i]->function()->get_return_type(); + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + data_types[i] = shared_state.aggregate_evaluators[i]->function()->get_return_type(); columns[i] = data_types[i]->create_column(); } - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { auto column = columns[i].get(); - _shared_state->aggregate_evaluators[i]->insert_result_info( - _agg_data->without_key + _dependency->offsets_of_aggregate_states()[i], column); + shared_state.aggregate_evaluators[i]->insert_result_info( + _agg_data->without_key + shared_state.offsets_of_aggregate_states[i], column); } const auto& block_schema = block->get_columns_with_type_and_name(); @@ -489,7 +553,7 @@ Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::B vectorized::ColumnPtr ptr = std::move(columns[i]); // unless `count`, other aggregate function dispose empty set should be null // so here check the children row return - ptr = make_nullable(ptr, _shared_state->input_num_rows == 0); + ptr = make_nullable(ptr, shared_state.input_num_rows == 0); columns[i] = ptr->assume_mutable(); } } @@ -501,8 +565,9 @@ Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::B } AggSourceOperatorX::AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, - const DescriptorTbl& descs) + const DescriptorTbl& descs, bool is_streaming) : Base(pool, tnode, operator_id, descs), + _is_streaming(is_streaming), _needs_finalize(tnode.agg_node.need_finalize), _without_key(tnode.agg_node.grouping_exprs.empty()) {} @@ -520,7 +585,7 @@ Status AggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* blo void AggLocalState::make_nullable_output_key(vectorized::Block* block) { if (block->rows() != 0) { - for (auto cid : _dependency->make_nullable_keys()) { + for (auto cid : _shared_state->make_nullable_keys) { block->get_by_position(cid).column = make_nullable(block->get_by_position(cid).column); block->get_by_position(cid).type = make_nullable(block->get_by_position(cid).type); } @@ -552,5 +617,4 @@ Status AggLocalState::close(RuntimeState* state) { return Base::close(state); } -} // namespace pipeline -} // namespace doris +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h index b7bbabd9a3..79671fb9c7 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.h +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -48,11 +48,29 @@ public: Status open(RuntimeState*) override { return Status::OK(); } }; +class AggSourceDependency final : public Dependency { +public: + using SharedState = AggSharedState; + AggSourceDependency(int id, int node_id) : Dependency(id, node_id, "AggSourceDependency") {} + ~AggSourceDependency() override = default; + + void block() override { + if (_is_streaming_agg_state()) { + Dependency::block(); + } + } + +private: + bool _is_streaming_agg_state() { + return ((SharedState*)Dependency::_shared_state.get())->data_queue != nullptr; + } +}; + class AggSourceOperatorX; -class AggLocalState final : public PipelineXLocalState { +class AggLocalState final : public PipelineXLocalState { public: - using Base = PipelineXLocalState; + using Base = PipelineXLocalState; ENABLE_FACTORY_CREATOR(AggLocalState); AggLocalState(RuntimeState* state, OperatorXBase* parent); ~AggLocalState() override = default; @@ -90,6 +108,24 @@ protected: Status _serialize_with_serialized_key_result_with_spilt_data(RuntimeState* state, vectorized::Block* block, SourceState& source_state); + Status _destroy_agg_status(vectorized::AggregateDataPtr data); + Status _reset_hash_table(); + Status _merge_spilt_data(); + void _make_nullable_output_key(vectorized::Block* block) { + if (block->rows() != 0) { + auto& shared_state = *Base ::_shared_state; + for (auto cid : shared_state.make_nullable_keys) { + block->get_by_position(cid).column = + make_nullable(block->get_by_position(cid).column); + block->get_by_position(cid).type = make_nullable(block->get_by_position(cid).type); + } + } + } + void _release_tracker() { + Base::_shared_state->mem_tracker->release( + Base::_shared_state->mem_usage_record.used_in_state + + Base::_shared_state->mem_usage_record.used_in_arena); + } RuntimeProfile::Counter* _get_results_timer; RuntimeProfile::Counter* _serialize_result_timer; @@ -117,7 +153,7 @@ class AggSourceOperatorX : public OperatorX { public: using Base = OperatorX; AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, - const DescriptorTbl& descs); + const DescriptorTbl& descs, bool is_streaming = false); ~AggSourceOperatorX() = default; Status get_block(RuntimeState* state, vectorized::Block* block, @@ -127,6 +163,7 @@ public: private: friend class AggLocalState; + const bool _is_streaming; bool _needs_finalize; bool _without_key; diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index 23a4838dff..84afab8344 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -27,7 +27,7 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(AnalyticSinkOperator, StreamingOperator) Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); @@ -66,6 +66,126 @@ Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf return Status::OK(); } +bool AnalyticSinkLocalState::_whether_need_next_partition( + vectorized::BlockRowPos& found_partition_end) { + auto& shared_state = *_shared_state; + if (shared_state.input_eos || + (shared_state.current_row_position < + shared_state.partition_by_end.pos)) { //now still have partition data + return false; + } + if ((shared_state.partition_by_eq_expr_ctxs.empty() && !shared_state.input_eos) || + (found_partition_end.pos == 0)) { //no partition, get until fetch to EOS + return true; + } + if (!shared_state.partition_by_eq_expr_ctxs.empty() && + found_partition_end.pos == shared_state.all_block_end.pos && + !shared_state.input_eos) { //current partition data calculate done + return true; + } + return false; +} + +//_partition_by_columns,_order_by_columns save in blocks, so if need to calculate the boundary, may find in which blocks firstly +vectorized::BlockRowPos AnalyticSinkLocalState::_compare_row_to_find_end( + int idx, vectorized::BlockRowPos start, vectorized::BlockRowPos end, + bool need_check_first) { + auto& shared_state = *_shared_state; + int64_t start_init_row_num = start.row_num; + vectorized::ColumnPtr start_column = + shared_state.input_blocks[start.block_num].get_by_position(idx).column; + vectorized::ColumnPtr start_next_block_column = start_column; + + DCHECK_LE(start.block_num, end.block_num); + DCHECK_LE(start.block_num, shared_state.input_blocks.size() - 1); + int64_t start_block_num = start.block_num; + int64_t end_block_num = end.block_num; + int64_t mid_blcok_num = end.block_num; + // To fix this problem: https://github.com/apache/doris/issues/15951 + // in this case, the partition by column is last row of block, so it's pointed to a new block at row = 0, range is: [left, right) + // From the perspective of order by column, the two values are exactly equal. + // so the range will be get wrong because it's compare_at == 0 with next block at row = 0 + if (need_check_first && end.block_num > 0 && end.row_num == 0) { + end.block_num--; + end_block_num--; + end.row_num = shared_state.input_blocks[end_block_num].rows(); + } + //binary search find in which block + while (start_block_num < end_block_num) { + mid_blcok_num = (start_block_num + end_block_num + 1) >> 1; + start_next_block_column = + shared_state.input_blocks[mid_blcok_num].get_by_position(idx).column; + //Compares (*this)[n] and rhs[m], this: start[init_row] rhs: mid[0] + if (start_column->compare_at(start_init_row_num, 0, *start_next_block_column, 1) == 0) { + start_block_num = mid_blcok_num; + } else { + end_block_num = mid_blcok_num - 1; + } + } + + // have check the start.block_num: start_column[start_init_row_num] with mid_blcok_num start_next_block_column[0] + // now next block must not be result, so need check with end_block_num: start_next_block_column[last_row] + if (end_block_num == mid_blcok_num - 1) { + start_next_block_column = + shared_state.input_blocks[end_block_num].get_by_position(idx).column; + int64_t block_size = shared_state.input_blocks[end_block_num].rows(); + if ((start_column->compare_at(start_init_row_num, block_size - 1, *start_next_block_column, + 1) == 0)) { + start.block_num = end_block_num + 1; + start.row_num = 0; + return start; + } + } + + //check whether need get column again, maybe same as first init + // if the start_block_num have move to forword, so need update start block num and compare it from row_num=0 + if (start_block_num != start.block_num) { + start_init_row_num = 0; + start.block_num = start_block_num; + start_column = shared_state.input_blocks[start.block_num].get_by_position(idx).column; + } + //binary search, set start and end pos + int64_t start_pos = start_init_row_num; + int64_t end_pos = shared_state.input_blocks[start.block_num].rows(); + //if end_block_num haven't moved, only start_block_num go to the end block + //so could use the end.row_num for binary search + if (start.block_num == end.block_num) { + end_pos = end.row_num; + } + while (start_pos < end_pos) { + int64_t mid_pos = (start_pos + end_pos) >> 1; + if (start_column->compare_at(start_init_row_num, mid_pos, *start_column, 1)) { + end_pos = mid_pos; + } else { + start_pos = mid_pos + 1; + } + } + start.row_num = start_pos; //update row num, return the find end + return start; +} + +vectorized::BlockRowPos AnalyticSinkLocalState::_get_partition_by_end() { + auto& shared_state = *_shared_state; + if (shared_state.current_row_position < + shared_state.partition_by_end.pos) { //still have data, return partition_by_end directly + return shared_state.partition_by_end; + } + + if (shared_state.partition_by_eq_expr_ctxs.empty() || + (shared_state.input_total_rows == 0)) { //no partition_by, the all block is end + return shared_state.all_block_end; + } + + vectorized::BlockRowPos cal_end = shared_state.all_block_end; + for (size_t i = 0; i < shared_state.partition_by_eq_expr_ctxs.size(); + ++i) { //have partition_by, binary search the partiton end + cal_end = _compare_row_to_find_end(shared_state.partition_by_column_idxs[i], + shared_state.partition_by_end, cal_end); + } + cal_end.pos = shared_state.input_block_first_row_positions[cal_end.block_num] + cal_end.row_num; + return cal_end; +} + AnalyticSinkOperatorX::AnalyticSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, const DescriptorTbl& descs) : DataSinkOperatorX(operator_id, tnode.node_id), @@ -138,8 +258,8 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)input_block->rows()); local_state._shared_state->input_eos = source_state == SourceState::FINISHED; if (local_state._shared_state->input_eos && input_block->rows() == 0) { - local_state._dependency->set_ready_for_read(); - local_state._dependency->block_writing(); + local_state._dependency->set_ready_to_read(); + local_state._dependency->block(); return Status::OK(); } @@ -192,10 +312,9 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block local_state._shared_state->input_blocks.emplace_back(std::move(*input_block)); { SCOPED_TIMER(local_state._evaluation_timer); - local_state._shared_state->found_partition_end = - local_state._dependency->get_partition_by_end(); + local_state._shared_state->found_partition_end = local_state._get_partition_by_end(); } - local_state._dependency->refresh_need_more_input(); + local_state._refresh_need_more_input(); return Status::OK(); } diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index a6c0fa745a..a90321f5e3 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -45,20 +45,45 @@ public: bool can_write() override { return _node->can_write(); } }; +class AnalyticSinkDependency final : public Dependency { +public: + using SharedState = AnalyticSharedState; + AnalyticSinkDependency(int id, int node_id) + : Dependency(id, node_id, "AnalyticSinkDependency", true) {} + ~AnalyticSinkDependency() override = default; +}; + class AnalyticSinkOperatorX; -class AnalyticSinkLocalState : public PipelineXSinkLocalState { +class AnalyticSinkLocalState : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(AnalyticSinkLocalState); public: AnalyticSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state) {} + : PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; private: friend class AnalyticSinkOperatorX; + bool _refresh_need_more_input() { + auto need_more_input = _whether_need_next_partition(_shared_state->found_partition_end); + if (need_more_input) { + _shared_state->source_dep->block(); + _dependency->set_ready(); + } else { + _dependency->block(); + _shared_state->source_dep->set_ready(); + } + return need_more_input; + } + vectorized::BlockRowPos _get_partition_by_end(); + vectorized::BlockRowPos _compare_row_to_find_end(int idx, vectorized::BlockRowPos start, + vectorized::BlockRowPos end, + bool need_check_first = false); + bool _whether_need_next_partition(vectorized::BlockRowPos& found_partition_end); + RuntimeProfile::Counter* _memory_usage_counter; RuntimeProfile::Counter* _evaluation_timer; RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage; diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp index 0d679d4adc..3c99d5bbdd 100644 --- a/be/src/pipeline/exec/analytic_source_operator.cpp +++ b/be/src/pipeline/exec/analytic_source_operator.cpp @@ -27,7 +27,7 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(AnalyticSourceOperator, SourceOperator) AnalyticLocalState::AnalyticLocalState(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState(state, parent), + : PipelineXLocalState(state, parent), _output_block_index(0), _window_end_position(0), _next_partition(false), @@ -37,8 +37,129 @@ AnalyticLocalState::AnalyticLocalState(RuntimeState* state, OperatorXBase* paren _agg_functions_size(0), _agg_functions_created(false) {} +//_partition_by_columns,_order_by_columns save in blocks, so if need to calculate the boundary, may find in which blocks firstly +vectorized::BlockRowPos AnalyticLocalState::_compare_row_to_find_end(int idx, + vectorized::BlockRowPos start, + vectorized::BlockRowPos end, + bool need_check_first) { + auto& shared_state = *_shared_state; + int64_t start_init_row_num = start.row_num; + vectorized::ColumnPtr start_column = + shared_state.input_blocks[start.block_num].get_by_position(idx).column; + vectorized::ColumnPtr start_next_block_column = start_column; + + DCHECK_LE(start.block_num, end.block_num); + DCHECK_LE(start.block_num, shared_state.input_blocks.size() - 1); + int64_t start_block_num = start.block_num; + int64_t end_block_num = end.block_num; + int64_t mid_blcok_num = end.block_num; + // To fix this problem: https://github.com/apache/doris/issues/15951 + // in this case, the partition by column is last row of block, so it's pointed to a new block at row = 0, range is: [left, right) + // From the perspective of order by column, the two values are exactly equal. + // so the range will be get wrong because it's compare_at == 0 with next block at row = 0 + if (need_check_first && end.block_num > 0 && end.row_num == 0) { + end.block_num--; + end_block_num--; + end.row_num = shared_state.input_blocks[end_block_num].rows(); + } + //binary search find in which block + while (start_block_num < end_block_num) { + mid_blcok_num = (start_block_num + end_block_num + 1) >> 1; + start_next_block_column = + shared_state.input_blocks[mid_blcok_num].get_by_position(idx).column; + //Compares (*this)[n] and rhs[m], this: start[init_row] rhs: mid[0] + if (start_column->compare_at(start_init_row_num, 0, *start_next_block_column, 1) == 0) { + start_block_num = mid_blcok_num; + } else { + end_block_num = mid_blcok_num - 1; + } + } + + // have check the start.block_num: start_column[start_init_row_num] with mid_blcok_num start_next_block_column[0] + // now next block must not be result, so need check with end_block_num: start_next_block_column[last_row] + if (end_block_num == mid_blcok_num - 1) { + start_next_block_column = + shared_state.input_blocks[end_block_num].get_by_position(idx).column; + int64_t block_size = shared_state.input_blocks[end_block_num].rows(); + if ((start_column->compare_at(start_init_row_num, block_size - 1, *start_next_block_column, + 1) == 0)) { + start.block_num = end_block_num + 1; + start.row_num = 0; + return start; + } + } + + //check whether need get column again, maybe same as first init + // if the start_block_num have move to forword, so need update start block num and compare it from row_num=0 + if (start_block_num != start.block_num) { + start_init_row_num = 0; + start.block_num = start_block_num; + start_column = shared_state.input_blocks[start.block_num].get_by_position(idx).column; + } + //binary search, set start and end pos + int64_t start_pos = start_init_row_num; + int64_t end_pos = shared_state.input_blocks[start.block_num].rows(); + //if end_block_num haven't moved, only start_block_num go to the end block + //so could use the end.row_num for binary search + if (start.block_num == end.block_num) { + end_pos = end.row_num; + } + while (start_pos < end_pos) { + int64_t mid_pos = (start_pos + end_pos) >> 1; + if (start_column->compare_at(start_init_row_num, mid_pos, *start_column, 1)) { + end_pos = mid_pos; + } else { + start_pos = mid_pos + 1; + } + } + start.row_num = start_pos; //update row num, return the find end + return start; +} + +vectorized::BlockRowPos AnalyticLocalState::_get_partition_by_end() { + auto& shared_state = *_shared_state; + if (shared_state.current_row_position < + shared_state.partition_by_end.pos) { //still have data, return partition_by_end directly + return shared_state.partition_by_end; + } + + if (shared_state.partition_by_eq_expr_ctxs.empty() || + (shared_state.input_total_rows == 0)) { //no partition_by, the all block is end + return shared_state.all_block_end; + } + + vectorized::BlockRowPos cal_end = shared_state.all_block_end; + for (size_t i = 0; i < shared_state.partition_by_eq_expr_ctxs.size(); + ++i) { //have partition_by, binary search the partiton end + cal_end = _compare_row_to_find_end(shared_state.partition_by_column_idxs[i], + shared_state.partition_by_end, cal_end); + } + cal_end.pos = shared_state.input_block_first_row_positions[cal_end.block_num] + cal_end.row_num; + return cal_end; +} + +bool AnalyticLocalState::_whether_need_next_partition( + vectorized::BlockRowPos& found_partition_end) { + auto& shared_state = *_shared_state; + if (shared_state.input_eos || + (shared_state.current_row_position < + shared_state.partition_by_end.pos)) { //now still have partition data + return false; + } + if ((shared_state.partition_by_eq_expr_ctxs.empty() && !shared_state.input_eos) || + (found_partition_end.pos == 0)) { //no partition, get until fetch to EOS + return true; + } + if (!shared_state.partition_by_eq_expr_ctxs.empty() && + found_partition_end.pos == shared_state.all_block_end.pos && + !shared_state.input_eos) { //current partition data calculate done + return true; + } + return false; +} + Status AnalyticLocalState::init(RuntimeState* state, LocalStateInfo& info) { - RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); _agg_arena_pool = std::make_unique(); @@ -282,8 +403,8 @@ void AnalyticLocalState::_update_order_by_range() { _order_by_start = _order_by_end; _order_by_end = _shared_state->partition_by_end; for (size_t i = 0; i < _shared_state->order_by_eq_expr_ctxs.size(); ++i) { - _order_by_end = _dependency->compare_row_to_find_end(_shared_state->ordey_by_column_idxs[i], - _order_by_start, _order_by_end, true); + _order_by_end = _compare_row_to_find_end(_shared_state->ordey_by_column_idxs[i], + _order_by_start, _order_by_end, true); } _order_by_start.pos = _shared_state->input_block_first_row_positions[_order_by_start.block_num] + @@ -410,10 +531,9 @@ Status AnalyticSourceOperatorX::get_block(RuntimeState* state, vectorized::Block local_state._output_block_index < local_state._shared_state->input_blocks.size()) { { SCOPED_TIMER(local_state._evaluation_timer); - local_state._shared_state->found_partition_end = - local_state._dependency->get_partition_by_end(); + local_state._shared_state->found_partition_end = local_state._get_partition_by_end(); } - if (local_state._dependency->refresh_need_more_input()) { + if (local_state._refresh_need_more_input()) { return Status::OK(); } local_state._next_partition = @@ -448,7 +568,7 @@ Status AnalyticLocalState::close(RuntimeState* state) { std::vector tmp_result_window_columns; _result_window_columns.swap(tmp_result_window_columns); - return PipelineXLocalState::close(state); + return PipelineXLocalState::close(state); } Status AnalyticSourceOperatorX::prepare(RuntimeState* state) { diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 4015a49113..0a741181f8 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -46,8 +46,16 @@ public: Status open(RuntimeState*) override { return Status::OK(); } }; +class AnalyticSourceDependency final : public Dependency { +public: + using SharedState = AnalyticSharedState; + AnalyticSourceDependency(int id, int node_id) + : Dependency(id, node_id, "AnalyticSourceDependency") {} + ~AnalyticSourceDependency() override = default; +}; + class AnalyticSourceOperatorX; -class AnalyticLocalState final : public PipelineXLocalState { +class AnalyticLocalState final : public PipelineXLocalState { public: ENABLE_FACTORY_CREATOR(AnalyticLocalState); AnalyticLocalState(RuntimeState* state, OperatorXBase* parent); @@ -71,6 +79,22 @@ private: void _insert_result_info(int64_t current_block_rows); void _update_order_by_range(); + bool _refresh_need_more_input() { + auto need_more_input = _whether_need_next_partition(_shared_state->found_partition_end); + if (need_more_input) { + _dependency->block(); + _shared_state->sink_dep->set_ready(); + } else { + _shared_state->sink_dep->block(); + _dependency->set_ready(); + } + return need_more_input; + } + vectorized::BlockRowPos _get_partition_by_end(); + vectorized::BlockRowPos _compare_row_to_find_end(int idx, vectorized::BlockRowPos start, + vectorized::BlockRowPos end, + bool need_check_first = false); + bool _whether_need_next_partition(vectorized::BlockRowPos& found_partition_end); Status _reset_agg_status(); Status _create_agg_status(); diff --git a/be/src/pipeline/exec/data_queue.cpp b/be/src/pipeline/exec/data_queue.cpp index fa522b68c6..656db9cbe7 100644 --- a/be/src/pipeline/exec/data_queue.cpp +++ b/be/src/pipeline/exec/data_queue.cpp @@ -30,7 +30,7 @@ namespace doris { namespace pipeline { -DataQueue::DataQueue(int child_count, WriteDependency* dependency) +DataQueue::DataQueue(int child_count) : _queue_blocks_lock(child_count), _queue_blocks(child_count), _free_blocks_lock(child_count), @@ -41,7 +41,8 @@ DataQueue::DataQueue(int child_count, WriteDependency* dependency) _cur_bytes_in_queue(child_count), _cur_blocks_nums_in_queue(child_count), _flag_queue_idx(0), - _dependency(dependency) { + _source_dependency(nullptr), + _sink_dependency(nullptr) { for (int i = 0; i < child_count; ++i) { _queue_blocks_lock[i].reset(new std::mutex()); _free_blocks_lock[i].reset(new std::mutex()); @@ -117,11 +118,11 @@ Status DataQueue::get_block_from_queue(std::unique_ptr* outpu } _cur_bytes_in_queue[_flag_queue_idx] -= (*output_block)->allocated_bytes(); _cur_blocks_nums_in_queue[_flag_queue_idx] -= 1; - if (_dependency) { + if (_sink_dependency) { if (!_is_finished[_flag_queue_idx]) { - _dependency->block_reading(); + _source_dependency->block(); } - _dependency->set_ready_for_write(); + _sink_dependency->set_ready(); } } else { if (_is_finished[_flag_queue_idx]) { @@ -141,9 +142,9 @@ void DataQueue::push_block(std::unique_ptr block, int child_i _cur_bytes_in_queue[child_idx] += block->allocated_bytes(); _queue_blocks[child_idx].emplace_back(std::move(block)); _cur_blocks_nums_in_queue[child_idx] += 1; - if (_dependency) { - _dependency->set_ready_for_read(); - _dependency->block_writing(); + if (_sink_dependency) { + _source_dependency->set_ready(); + _sink_dependency->block(); } //this only use to record the queue[0] for profile _max_bytes_in_queue = std::max(_max_bytes_in_queue, _cur_bytes_in_queue[0].load()); @@ -154,8 +155,8 @@ void DataQueue::push_block(std::unique_ptr block, int child_i void DataQueue::set_finish(int child_idx) { std::lock_guard l(*_queue_blocks_lock[child_idx]); _is_finished[child_idx] = true; - if (_dependency) { - _dependency->set_ready_for_read(); + if (_source_dependency) { + _source_dependency->set_ready(); } } @@ -164,8 +165,8 @@ void DataQueue::set_canceled(int child_idx) { DCHECK(!_is_finished[child_idx]); _is_canceled[child_idx] = true; _is_finished[child_idx] = true; - if (_dependency) { - _dependency->set_ready_for_read(); + if (_source_dependency) { + _source_dependency->set_ready(); } } diff --git a/be/src/pipeline/exec/data_queue.h b/be/src/pipeline/exec/data_queue.h index 39ab87c5ac..f756ca7e62 100644 --- a/be/src/pipeline/exec/data_queue.h +++ b/be/src/pipeline/exec/data_queue.h @@ -30,12 +30,12 @@ namespace doris { namespace pipeline { -class WriteDependency; +class Dependency; class DataQueue { public: //always one is enough, but in union node it's has more children - DataQueue(int child_count = 1, WriteDependency* dependency = nullptr); + DataQueue(int child_count = 1); ~DataQueue() = default; Status get_block_from_queue(std::unique_ptr* block, @@ -60,11 +60,16 @@ public: int64_t max_size_of_queue() const { return _max_size_of_queue; } bool data_exhausted() const { return _data_exhausted; } - void set_dependency(WriteDependency* dependency) { _dependency = dependency; } + void set_dependency(Dependency* source_dependency, Dependency* sink_dependency) { + _source_dependency = source_dependency; + _sink_dependency = sink_dependency; + } private: - friend class AggDependency; - friend class UnionDependency; + friend class AggSourceDependency; + friend class UnionSourceDependency; + friend class AggSinkDependency; + friend class UnionSinkDependency; std::vector> _queue_blocks_lock; std::vector>> _queue_blocks; @@ -90,7 +95,8 @@ private: int64_t _max_size_of_queue = 0; static constexpr int64_t MAX_BYTE_OF_QUEUE = 1024l * 1024 * 1024 / 10; - WriteDependency* _dependency = nullptr; + Dependency* _source_dependency = nullptr; + Dependency* _sink_dependency = nullptr; }; } // namespace pipeline diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.cpp index 8a240d6336..74cf655c36 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.cpp @@ -97,7 +97,7 @@ OperatorPtr DistinctStreamingAggSinkOperatorBuilder::build_operator() { DistinctStreamingAggSinkLocalState::DistinctStreamingAggSinkLocalState( DataSinkOperatorXBase* parent, RuntimeState* state) - : AggSinkLocalState(parent, state), + : AggSinkLocalState(parent, state), dummy_mapped_data(std::make_shared('A')) {} Status DistinctStreamingAggSinkLocalState::_distinct_pre_agg_with_serialized_key( @@ -127,7 +127,7 @@ Status DistinctStreamingAggSinkLocalState::_distinct_pre_agg_with_serialized_key RETURN_IF_CATCH_EXCEPTION( _emplace_into_hash_table_to_distinct(_distinct_row, key_columns, rows)); - bool mem_reuse = _dependency->make_nullable_keys().empty() && out_block->mem_reuse(); + bool mem_reuse = _shared_state->make_nullable_keys.empty() && out_block->mem_reuse(); if (mem_reuse) { for (int i = 0; i < key_size; ++i) { auto dst = out_block->get_by_position(i).column->assume_mutable(); diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h index 60af15179d..c156a83e1d 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_sink_operator.h @@ -78,17 +78,12 @@ private: class DistinctStreamingAggSinkOperatorX; class DistinctStreamingAggSinkLocalState final - : public AggSinkLocalState { + : public AggSinkLocalState { public: using Parent = DistinctStreamingAggSinkOperatorX; - using Base = AggSinkLocalState; + using Base = AggSinkLocalState; ENABLE_FACTORY_CREATOR(DistinctStreamingAggSinkLocalState); DistinctStreamingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); - Status init(RuntimeState* state, LocalSinkStateInfo& info) override { - RETURN_IF_ERROR(Base::init(state, info)); - _shared_state->data_queue.reset(new DataQueue(1, _dependency)); - return Status::OK(); - } Status close(RuntimeState* state, Status exec_status) override; Status _distinct_pre_agg_with_serialized_key(vectorized::Block* in_block, diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.cpp index ffdbab0127..722cc5b795 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_source_operator.cpp @@ -92,7 +92,7 @@ DistinctStreamingAggSourceOperatorX::DistinctStreamingAggSourceOperatorX(ObjectP const TPlanNode& tnode, int operator_id, const DescriptorTbl& descs) - : Base(pool, tnode, operator_id, descs) { + : Base(pool, tnode, operator_id, descs, true) { if (tnode.agg_node.__isset.use_streaming_preaggregation) { _is_streaming_preagg = tnode.agg_node.use_streaming_preaggregation; if (_is_streaming_preagg) { @@ -116,7 +116,7 @@ Status DistinctStreamingAggSourceOperatorX::get_block(RuntimeState* state, vecto local_state._shared_state->data_queue->push_free_block(std::move(agg_block)); } - local_state._dependency->_make_nullable_output_key(block); + local_state._make_nullable_output_key(block); if (_is_streaming_preagg == false) { // dispose the having clause, should not be execute in prestreaming agg RETURN_IF_ERROR( diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 68f34ffc82..1a4a786654 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -93,7 +93,7 @@ bool ExchangeSinkBuffer::can_write() const { template void ExchangeSinkBuffer::_set_ready_to_finish(bool all_done) { if (_finish_dependency && _should_stop && all_done) { - _finish_dependency->set_ready_to_finish(); + _finish_dependency->set_ready(); } } @@ -171,7 +171,7 @@ Status ExchangeSinkBuffer::add_block(TransmitInfo&& request) { _instance_to_package_queue[ins_id.lo].emplace(std::move(request)); _total_queue_size++; if (_queue_dependency && _total_queue_size > _queue_capacity) { - _queue_dependency->block_writing(); + _queue_dependency->block(); } } if (send_now) { @@ -292,7 +292,7 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { q.pop(); _total_queue_size--; if (_queue_dependency && _total_queue_size <= _queue_capacity) { - _queue_dependency->set_ready_for_write(); + _queue_dependency->set_ready(); } } else if (!broadcast_q.empty()) { // If we have data to shuffle which is broadcasted diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index edcfa80bc2..a04b3b29b3 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -47,7 +47,7 @@ using InstanceLoId = int64_t; namespace pipeline { class BroadcastDependency; class ExchangeSinkQueueDependency; -class FinishDependency; +class Dependency; } // namespace pipeline namespace vectorized { @@ -189,7 +189,7 @@ public: void update_profile(RuntimeProfile* profile); void set_dependency(std::shared_ptr queue_dependency, - std::shared_ptr finish_dependency) { + std::shared_ptr finish_dependency) { _queue_dependency = queue_dependency; _finish_dependency = finish_dependency; } @@ -249,7 +249,7 @@ private: std::atomic _total_queue_size = 0; static constexpr int QUEUE_CAPACITY_FACTOR = 64; std::shared_ptr _queue_dependency = nullptr; - std::shared_ptr _finish_dependency = nullptr; + std::shared_ptr _finish_dependency = nullptr; QueryStatistics* _statistics = nullptr; std::atomic _should_stop {false}; }; diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 42ab71a454..616a1bd76f 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -230,7 +230,7 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf fmt::format("Crc32HashPartitioner({})", _partition_count)); } - _finish_dependency->should_finish_after_check(); + _finish_dependency->block(); return Status::OK(); } @@ -524,14 +524,13 @@ Status ExchangeSinkLocalState::close(RuntimeState* state, Status exec_status) { } SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_close_timer); - COUNTER_UPDATE(_wait_queue_timer, _queue_dependency->write_watcher_elapse_time()); + COUNTER_UPDATE(_wait_queue_timer, _queue_dependency->watcher_elapse_time()); if (_broadcast_dependency) { - COUNTER_UPDATE(_wait_broadcast_buffer_timer, - _broadcast_dependency->write_watcher_elapse_time()); + COUNTER_UPDATE(_wait_broadcast_buffer_timer, _broadcast_dependency->watcher_elapse_time()); } for (size_t i = 0; i < _local_channels_dependency.size(); i++) { COUNTER_UPDATE(_wait_channel_timer[i], - _local_channels_dependency[i]->write_watcher_elapse_time()); + _local_channels_dependency[i]->watcher_elapse_time()); } _sink_buffer->update_profile(profile()); _sink_buffer->close(); diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index bc9c26e36e..c27681738a 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -64,32 +64,32 @@ private: int _mult_cast_id = -1; }; -class ExchangeSinkQueueDependency final : public WriteDependency { +class ExchangeSinkQueueDependency final : public Dependency { public: ENABLE_FACTORY_CREATOR(ExchangeSinkQueueDependency); ExchangeSinkQueueDependency(int id, int node_id) - : WriteDependency(id, node_id, "ResultQueueDependency") {} + : Dependency(id, node_id, "ResultQueueDependency", true) {} ~ExchangeSinkQueueDependency() override = default; }; -class BroadcastDependency final : public WriteDependency { +class BroadcastDependency final : public Dependency { public: ENABLE_FACTORY_CREATOR(BroadcastDependency); BroadcastDependency(int id, int node_id) - : WriteDependency(id, node_id, "BroadcastDependency"), _available_block(0) {} + : Dependency(id, node_id, "BroadcastDependency", true), _available_block(0) {} ~BroadcastDependency() override = default; void set_available_block(int available_block) { _available_block = available_block; } void return_available_block() { if (_available_block.fetch_add(1) == 0) { - WriteDependency::set_ready_for_write(); + Dependency::set_ready(); } } void take_available_block() { if (_available_block.fetch_sub(1) == 1) { - WriteDependency::block_writing(); + Dependency::block(); } } @@ -117,11 +117,11 @@ private: * | ExchangeSource1 | | ExchangeSource2 | * +-----------------+ +------------------+ */ -class LocalExchangeChannelDependency final : public WriteDependency { +class LocalExchangeChannelDependency final : public Dependency { public: ENABLE_FACTORY_CREATOR(LocalExchangeChannelDependency); LocalExchangeChannelDependency(int id, int node_id) - : WriteDependency(id, node_id, "LocalExchangeChannelDependency") {} + : Dependency(id, node_id, "LocalExchangeChannelDependency", true) {} ~LocalExchangeChannelDependency() override = default; // TODO(gabriel): blocked by memory }; @@ -139,7 +139,7 @@ public: Status init(RuntimeState* state, LocalSinkStateInfo& info) override; Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; - WriteDependency* dependency() override { return _exchange_sink_dependency.get(); } + Dependency* dependency() override { return _exchange_sink_dependency.get(); } Status serialize_block(vectorized::Block* src, PBlock* dest, int num_receivers = 1); void register_channels(pipeline::ExchangeSinkBuffer* buffer); Status get_next_available_buffer(vectorized::BroadcastPBlockHolder** holder); diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp index 4a29694a37..00cead5b2c 100644 --- a/be/src/pipeline/exec/exchange_source_operator.cpp +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -176,7 +176,7 @@ Status ExchangeLocalState::close(RuntimeState* state) { } const auto& queues = stream_recvr->sender_queues(); for (size_t i = 0; i < deps.size(); i++) { - COUNTER_SET(metrics[i], deps[i]->read_watcher_elapse_time()); + COUNTER_SET(metrics[i], deps[i]->watcher_elapse_time()); } if (stream_recvr != nullptr) { stream_recvr->close(); diff --git a/be/src/pipeline/exec/exchange_source_operator.h b/be/src/pipeline/exec/exchange_source_operator.h index c00319c1e9..4a2bae8298 100644 --- a/be/src/pipeline/exec/exchange_source_operator.h +++ b/be/src/pipeline/exec/exchange_source_operator.h @@ -55,25 +55,7 @@ public: ENABLE_FACTORY_CREATOR(ExchangeDataDependency); ExchangeDataDependency(int id, int node_id, vectorized::VDataStreamRecvr::SenderQueue* sender_queue) - : Dependency(id, node_id, "DataDependency"), _always_done(false) {} - - void set_always_done() { - if (_always_done) { - return; - } - _always_done = true; - Dependency::set_ready_for_read(); - } - - void block_reading() override { - if (_always_done) { - return; - } - Dependency::block_reading(); - } - -private: - std::atomic _always_done; + : Dependency(id, node_id, "DataDependency") {} }; class ExchangeSourceOperatorX; diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 3d9827a27b..53b20f53cd 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -81,7 +81,7 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo } } if (!_should_build_hash_table) { - _shared_hash_table_dependency->block_writing(); + _shared_hash_table_dependency->block(); p._shared_hashtable_controller->append_dependency(p.node_id(), _shared_hash_table_dependency); } else if (p._is_broadcast_join) { @@ -175,6 +175,68 @@ void HashJoinBuildSinkLocalState::init_short_circuit_for_probe() { p._join_op == TJoinOp::LEFT_ANTI_JOIN); } +Status HashJoinBuildSinkLocalState::_do_evaluate(vectorized::Block& block, + vectorized::VExprContextSPtrs& exprs, + RuntimeProfile::Counter& expr_call_timer, + std::vector& res_col_ids) { + for (size_t i = 0; i < exprs.size(); ++i) { + int result_col_id = -1; + // execute build column + { + SCOPED_TIMER(&expr_call_timer); + RETURN_IF_ERROR(exprs[i]->execute(&block, &result_col_id)); + } + + // TODO: opt the column is const + block.get_by_position(result_col_id).column = + block.get_by_position(result_col_id).column->convert_to_full_column_if_const(); + res_col_ids[i] = result_col_id; + } + return Status::OK(); +} + +std::vector HashJoinBuildSinkLocalState::_convert_block_to_null( + vectorized::Block& block) { + std::vector results; + for (int i = 0; i < block.columns(); ++i) { + if (auto& column_type = block.safe_get_by_position(i); !column_type.type->is_nullable()) { + DCHECK(!column_type.column->is_nullable()); + column_type.column = make_nullable(column_type.column); + column_type.type = make_nullable(column_type.type); + results.emplace_back(i); + } + } + return results; +} + +Status HashJoinBuildSinkLocalState::_extract_join_column( + vectorized::Block& block, vectorized::ColumnUInt8::MutablePtr& null_map, + vectorized::ColumnRawPtrs& raw_ptrs, const std::vector& res_col_ids) { + auto& shared_state = *_shared_state; + for (size_t i = 0; i < shared_state.build_exprs_size; ++i) { + if (shared_state.is_null_safe_eq_join[i]) { + raw_ptrs[i] = block.get_by_position(res_col_ids[i]).column.get(); + } else { + auto column = block.get_by_position(res_col_ids[i]).column.get(); + if (auto* nullable = check_and_get_column(*column)) { + auto& col_nested = nullable->get_nested_column(); + auto& col_nullmap = nullable->get_null_map_data(); + + if (shared_state.store_null_in_hash_table[i]) { + raw_ptrs[i] = nullable; + } else { + DCHECK(null_map != nullptr); + vectorized::VectorizedUtils::update_null_map(null_map->get_data(), col_nullmap); + raw_ptrs[i] = &col_nested; + } + } else { + raw_ptrs[i] = column; + } + } + } + return Status::OK(); +} + Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, vectorized::Block& block, uint8_t offset) { auto& p = _parent->cast(); @@ -189,10 +251,9 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, vectorized::ColumnUInt8::MutablePtr null_map_val; std::vector res_col_ids(_build_expr_ctxs.size()); - RETURN_IF_ERROR(_dependency->do_evaluate(block, _build_expr_ctxs, *_build_expr_call_timer, - res_col_ids)); + RETURN_IF_ERROR(_do_evaluate(block, _build_expr_ctxs, *_build_expr_call_timer, res_col_ids)); if (p._join_op == TJoinOp::LEFT_OUTER_JOIN || p._join_op == TJoinOp::FULL_OUTER_JOIN) { - _dependency->convert_block_to_null(block); + _convert_block_to_null(block); } // TODO: Now we are not sure whether a column is nullable only by ExecNode's `row_desc` // so we have to initialize this flag by the first build block. @@ -206,7 +267,7 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, } // Get the key column that needs to be built - Status st = _dependency->extract_join_column(block, null_map_val, raw_ptrs, res_col_ids); + Status st = _extract_join_column(block, null_map_val, raw_ptrs, res_col_ids); st = std::visit( Overload { @@ -560,7 +621,7 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* _join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { local_state._shared_state->probe_ignore_null = true; } - local_state._dependency->set_ready_for_read(); + local_state._dependency->set_ready_to_read(); } return Status::OK(); diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index 9771ae43e8..e3f10a1feb 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -46,16 +46,24 @@ public: class HashJoinBuildSinkOperatorX; -class SharedHashTableDependency final : public WriteDependency { +class SharedHashTableDependency final : public Dependency { public: ENABLE_FACTORY_CREATOR(SharedHashTableDependency); SharedHashTableDependency(int id, int node_id) - : WriteDependency(id, node_id, "SharedHashTableDependency") {} + : Dependency(id, node_id, "SharedHashTableDependency", true) {} ~SharedHashTableDependency() override = default; }; +class HashJoinBuildSinkDependency final : public Dependency { +public: + using SharedState = HashJoinSharedState; + HashJoinBuildSinkDependency(int id, int node_id) + : Dependency(id, node_id, "HashJoinBuildSinkDependency", true) {} + ~HashJoinBuildSinkDependency() override = default; +}; + class HashJoinBuildSinkLocalState final - : public JoinBuildSinkLocalState { + : public JoinBuildSinkLocalState { public: ENABLE_FACTORY_CREATOR(HashJoinBuildSinkLocalState); using Parent = HashJoinBuildSinkOperatorX; @@ -78,11 +86,18 @@ public: void add_hash_buckets_filled_info(const std::string& info) const { _profile->add_info_string("HashTableFilledBuckets", info); } - WriteDependency* dependency() override { return _shared_hash_table_dependency.get(); } + Dependency* dependency() override { return _shared_hash_table_dependency.get(); } protected: void _hash_table_init(RuntimeState* state); void _set_build_ignore_flag(vectorized::Block& block, const std::vector& res_col_ids); + Status _do_evaluate(vectorized::Block& block, vectorized::VExprContextSPtrs& exprs, + RuntimeProfile::Counter& expr_call_timer, std::vector& res_col_ids); + std::vector _convert_block_to_null(vectorized::Block& block); + Status _extract_join_column(vectorized::Block& block, + vectorized::ColumnUInt8::MutablePtr& null_map, + vectorized::ColumnRawPtrs& raw_ptrs, + const std::vector& res_col_ids); friend class HashJoinBuildSinkOperatorX; template friend struct vectorized::ProcessHashTableBuild; diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index b459d8eddd..e4e5873fa7 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -28,7 +28,7 @@ namespace pipeline { OPERATOR_CODE_GENERATOR(HashJoinProbeOperator, StatefulOperator) HashJoinProbeLocalState::HashJoinProbeLocalState(RuntimeState* state, OperatorXBase* parent) - : JoinProbeLocalState(state, parent) {} + : JoinProbeLocalState(state, parent) {} Status HashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(JoinProbeLocalState::init(state, info)); @@ -147,7 +147,7 @@ Status HashJoinProbeLocalState::close(RuntimeState* state) { _tuple_is_null_left_flag_column = nullptr; _tuple_is_null_right_flag_column = nullptr; _probe_block.clear(); - return JoinProbeLocalState::close(state); + return JoinProbeLocalState::close(state); } bool HashJoinProbeLocalState::_need_probe_null_map(vectorized::Block& block, @@ -402,6 +402,48 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc return Status::OK(); } +Status HashJoinProbeLocalState::_extract_join_column(vectorized::Block& block, + vectorized::ColumnUInt8::MutablePtr& null_map, + vectorized::ColumnRawPtrs& raw_ptrs, + const std::vector& res_col_ids) { + auto& shared_state = *_shared_state; + for (size_t i = 0; i < shared_state.build_exprs_size; ++i) { + if (shared_state.is_null_safe_eq_join[i]) { + raw_ptrs[i] = block.get_by_position(res_col_ids[i]).column.get(); + } else { + auto column = block.get_by_position(res_col_ids[i]).column.get(); + if (auto* nullable = check_and_get_column(*column)) { + auto& col_nested = nullable->get_nested_column(); + auto& col_nullmap = nullable->get_null_map_data(); + + DCHECK(null_map != nullptr); + vectorized::VectorizedUtils::update_null_map(null_map->get_data(), col_nullmap); + if (shared_state.store_null_in_hash_table[i]) { + raw_ptrs[i] = nullable; + } else { + raw_ptrs[i] = &col_nested; + } + } else { + raw_ptrs[i] = column; + } + } + } + return Status::OK(); +} + +std::vector HashJoinProbeLocalState::_convert_block_to_null(vectorized::Block& block) { + std::vector results; + for (int i = 0; i < block.columns(); ++i) { + if (auto& column_type = block.safe_get_by_position(i); !column_type.type->is_nullable()) { + DCHECK(!column_type.column->is_nullable()); + column_type.column = make_nullable(column_type.column); + column_type.type = make_nullable(column_type.type); + results.emplace_back(i); + } + } + return results; +} + Status HashJoinProbeLocalState::filter_data_and_build_output(RuntimeState* state, vectorized::Block* output_block, SourceState& source_state, @@ -467,7 +509,7 @@ Status HashJoinProbeOperatorX::push(RuntimeState* state, vectorized::Block* inpu std::vector res_col_ids(probe_expr_ctxs_sz); if (_join_op == TJoinOp::RIGHT_OUTER_JOIN || _join_op == TJoinOp::FULL_OUTER_JOIN) { local_state._probe_column_convert_to_null = - local_state._dependency->convert_block_to_null(*input_block); + local_state._convert_block_to_null(*input_block); } RETURN_IF_ERROR(_do_evaluate(*input_block, local_state._probe_expr_ctxs, *local_state._probe_expr_call_timer, res_col_ids)); @@ -486,9 +528,8 @@ Status HashJoinProbeOperatorX::push(RuntimeState* state, vectorized::Block* inpu local_state._null_map_column->get_data().assign(input_block->rows(), (uint8_t)0); } - RETURN_IF_ERROR(local_state._dependency->extract_join_column( - *input_block, local_state._null_map_column, local_state._probe_columns, - res_col_ids)); + RETURN_IF_ERROR(local_state._extract_join_column(*input_block, local_state._null_map_column, + local_state._probe_columns, res_col_ids)); if (&local_state._probe_block != input_block) { input_block->swap(local_state._probe_block); } diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index 923f7dd7b9..64f3cce2d7 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -61,9 +61,17 @@ using HashTableCtxVariants = std::variant< vectorized::ProcessHashTableProbe>; +class HashJoinProbeDependency final : public Dependency { +public: + using SharedState = HashJoinSharedState; + HashJoinProbeDependency(int id, int node_id) + : Dependency(id, node_id, "HashJoinProbeDependency") {} + ~HashJoinProbeDependency() override = default; +}; + class HashJoinProbeOperatorX; class HashJoinProbeLocalState final - : public JoinProbeLocalState { + : public JoinProbeLocalState { public: using Parent = HashJoinProbeOperatorX; ENABLE_FACTORY_CREATOR(HashJoinProbeLocalState); @@ -96,6 +104,11 @@ public: private: void _prepare_probe_block(); bool _need_probe_null_map(vectorized::Block& block, const std::vector& res_col_ids); + std::vector _convert_block_to_null(vectorized::Block& block); + Status _extract_join_column(vectorized::Block& block, + vectorized::ColumnUInt8::MutablePtr& null_map, + vectorized::ColumnRawPtrs& raw_ptrs, + const std::vector& res_col_ids); friend class HashJoinProbeOperatorX; template friend struct vectorized::ProcessHashTableProbe; diff --git a/be/src/pipeline/exec/join_build_sink_operator.cpp b/be/src/pipeline/exec/join_build_sink_operator.cpp index 2ed4ebfb0d..fe790d9032 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.cpp +++ b/be/src/pipeline/exec/join_build_sink_operator.cpp @@ -115,8 +115,9 @@ void JoinBuildSinkOperatorX::_init_join_op() { } template class JoinBuildSinkOperatorX; -template class JoinBuildSinkLocalState; +template class JoinBuildSinkLocalState; template class JoinBuildSinkOperatorX; -template class JoinBuildSinkLocalState; +template class JoinBuildSinkLocalState; } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 5dd2d5c0cc..6b6796767e 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -225,10 +225,10 @@ Status JoinProbeOperatorX::open(doris::RuntimeState* state) { return vectorized::VExpr::open(_output_expr_ctxs, state); } -template class JoinProbeLocalState; +template class JoinProbeLocalState; template class JoinProbeOperatorX; -template class JoinProbeLocalState; +template class JoinProbeLocalState; template class JoinProbeOperatorX; } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp b/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp index 2631a4de77..ad6b3c8cdd 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp @@ -40,7 +40,7 @@ Status MultiCastDataStreamSinkLocalState::init(RuntimeState* state, LocalSinkSta ->create_multi_cast_data_streamer(); auto& deps = info.dependencys; for (auto dep : deps) { - ((MultiCastDependency*)dep.get())->set_shared_state(multi_cast_data_streamer); + ((MultiCastSinkDependency*)dep.get())->set_shared_state(multi_cast_data_streamer); } RETURN_IF_ERROR(Base::init(state, info)); return Status::OK(); 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 008238577f..c9b3dcb479 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.h @@ -41,15 +41,23 @@ public: bool can_write() override { return _sink->can_write(); } }; +class MultiCastSinkDependency final : public Dependency { +public: + using SharedState = MultiCastSharedState; + MultiCastSinkDependency(int id, int node_id) + : Dependency(id, node_id, "MultiCastSinkDependency", true) {} + ~MultiCastSinkDependency() override = default; +}; + class MultiCastDataStreamSinkOperatorX; class MultiCastDataStreamSinkLocalState final - : public PipelineXSinkLocalState { + : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(MultiCastDataStreamSinkLocalState); MultiCastDataStreamSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : Base(parent, state) {} friend class MultiCastDataStreamSinkOperatorX; friend class DataSinkOperatorX; - using Base = PipelineXSinkLocalState; + using Base = PipelineXSinkLocalState; using Parent = MultiCastDataStreamSinkOperatorX; Status init(RuntimeState* state, LocalSinkStateInfo& info) override; std::string id_name() override; diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp index 26eee2161c..4c73f6ecb2 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp @@ -139,7 +139,7 @@ Status MultiCastDataStreamSourceLocalState::init(RuntimeState* state, LocalState SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); _shared_state->multi_cast_data_streamer.set_dep_by_sender_idx( - p._consumer_id, static_cast(_dependency)); + p._consumer_id, static_cast(_dependency)); _output_expr_contexts.resize(p._output_expr_contexts.size()); for (size_t i = 0; i < p._output_expr_contexts.size(); i++) { RETURN_IF_ERROR(p._output_expr_contexts[i]->clone(state, _output_expr_contexts[i])); diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.h b/be/src/pipeline/exec/multi_cast_data_stream_source.h index aafacf9d87..6c3a4cfbbc 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -92,13 +92,22 @@ private: vectorized::VExprContextSPtrs _conjuncts; }; +class MultiCastSourceDependency final : public Dependency { +public: + using SharedState = MultiCastSharedState; + MultiCastSourceDependency(int id, int node_id) + : Dependency(id, node_id, "MultiCastSourceDependency") {} + ~MultiCastSourceDependency() override = default; +}; + class MultiCastDataStreamerSourceOperatorX; -class MultiCastDataStreamSourceLocalState final : public PipelineXLocalState, - public vectorized::RuntimeFilterConsumer { +class MultiCastDataStreamSourceLocalState final + : public PipelineXLocalState, + public vectorized::RuntimeFilterConsumer { public: ENABLE_FACTORY_CREATOR(MultiCastDataStreamSourceLocalState); - using Base = PipelineXLocalState; + using Base = PipelineXLocalState; using Parent = MultiCastDataStreamerSourceOperatorX; MultiCastDataStreamSourceLocalState(RuntimeState* state, OperatorXBase* parent); Status init(RuntimeState* state, LocalStateInfo& info) override; diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.cpp b/be/src/pipeline/exec/multi_cast_data_streamer.cpp index 8d7a745a04..175a21469b 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.cpp +++ b/be/src/pipeline/exec/multi_cast_data_streamer.cpp @@ -17,6 +17,7 @@ #include "multi_cast_data_streamer.h" +#include "pipeline/exec/multi_cast_data_stream_source.h" #include "pipeline/pipeline_x/dependency.h" #include "runtime/runtime_state.h" @@ -105,13 +106,13 @@ void MultiCastDataStreamer::_set_ready_for_read(int sender_idx) { } auto* dep = _dependencies[sender_idx]; DCHECK(dep); - dep->set_ready_for_read(); + dep->set_ready(); } void MultiCastDataStreamer::_set_ready_for_read() { for (auto* dep : _dependencies) { DCHECK(dep); - dep->set_ready_for_read(); + dep->set_ready(); } } @@ -121,7 +122,7 @@ void MultiCastDataStreamer::_block_reading(int sender_idx) { } auto* dep = _dependencies[sender_idx]; DCHECK(dep); - dep->block_reading(); + dep->block(); } } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.h b/be/src/pipeline/exec/multi_cast_data_streamer.h index 4d03fe53b8..973b364410 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.h +++ b/be/src/pipeline/exec/multi_cast_data_streamer.h @@ -21,7 +21,7 @@ namespace doris::pipeline { -class MultiCastDependency; +class MultiCastSourceDependency; struct MultiCastBlock { MultiCastBlock(vectorized::Block* block, int used_count, size_t mem_size); @@ -74,7 +74,7 @@ public: _set_ready_for_read(); } - void set_dep_by_sender_idx(int sender_idx, MultiCastDependency* dep) { + void set_dep_by_sender_idx(int sender_idx, MultiCastSourceDependency* dep) { _dependencies[sender_idx] = dep; _block_reading(sender_idx); } @@ -97,6 +97,6 @@ private: RuntimeProfile::Counter* _process_rows; RuntimeProfile::Counter* _peak_mem_usage; - std::vector _dependencies; + std::vector _dependencies; }; } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp index f525a1abf1..aec93c66b6 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp @@ -27,8 +27,8 @@ OPERATOR_CODE_GENERATOR(NestLoopJoinBuildOperator, StreamingOperator) NestedLoopJoinBuildSinkLocalState::NestedLoopJoinBuildSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : JoinBuildSinkLocalState( - parent, state) {} + : JoinBuildSinkLocalState(parent, state) {} Status NestedLoopJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(JoinBuildSinkLocalState::init(state, info)); @@ -120,7 +120,7 @@ Status NestedLoopJoinBuildSinkOperatorX::sink(doris::RuntimeState* state, vector !local_state._shared_state->build_blocks.empty()))) { local_state._shared_state->left_side_eos = true; } - local_state._dependency->set_ready_for_read(); + local_state._dependency->set_ready_to_read(); } return Status::OK(); diff --git a/be/src/pipeline/exec/nested_loop_join_build_operator.h b/be/src/pipeline/exec/nested_loop_join_build_operator.h index bd94c9e0e9..a02fb8ec1a 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.h @@ -44,10 +44,18 @@ public: bool can_write() override { return true; } }; +class NestedLoopJoinBuildSinkDependency final : public Dependency { +public: + using SharedState = NestedLoopJoinSharedState; + NestedLoopJoinBuildSinkDependency(int id, int node_id) + : Dependency(id, node_id, "NestedLoopJoinBuildSinkDependency", true) {} + ~NestedLoopJoinBuildSinkDependency() override = default; +}; + class NestedLoopJoinBuildSinkOperatorX; class NestedLoopJoinBuildSinkLocalState final - : public JoinBuildSinkLocalState { public: ENABLE_FACTORY_CREATOR(NestedLoopJoinBuildSinkLocalState); diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp index f73b84c538..371e010614 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp @@ -45,8 +45,8 @@ Status NestLoopJoinProbeOperator::close(doris::RuntimeState* state) { NestedLoopJoinProbeLocalState::NestedLoopJoinProbeLocalState(RuntimeState* state, OperatorXBase* parent) - : JoinProbeLocalState(state, - parent), + : JoinProbeLocalState(state, + parent), _matched_rows_done(false), _left_block_pos(0) {} @@ -75,7 +75,7 @@ Status NestedLoopJoinProbeLocalState::close(RuntimeState* state) { _tuple_is_null_left_flag_column = nullptr; _tuple_is_null_right_flag_column = nullptr; - return JoinProbeLocalState::close( + return JoinProbeLocalState::close( state); } diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.h b/be/src/pipeline/exec/nested_loop_join_probe_operator.h index 1a18d419b8..34f123ba32 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.h @@ -51,9 +51,17 @@ public: Status close(RuntimeState* state) override; }; +class NestedLoopJoinProbeDependency final : public Dependency { +public: + using SharedState = NestedLoopJoinSharedState; + NestedLoopJoinProbeDependency(int id, int node_id) + : Dependency(id, node_id, "NestedLoopJoinProbeDependency") {} + ~NestedLoopJoinProbeDependency() override = default; +}; + class NestedLoopJoinProbeOperatorX; class NestedLoopJoinProbeLocalState final - : public JoinProbeLocalState { + : public JoinProbeLocalState { public: using Parent = NestedLoopJoinProbeOperatorX; ENABLE_FACTORY_CREATOR(NestedLoopJoinProbeLocalState); diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.cpp b/be/src/pipeline/exec/partition_sort_sink_operator.cpp index d0c3774a58..168d499e66 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_sink_operator.cpp @@ -29,7 +29,7 @@ OperatorPtr PartitionSortSinkOperatorBuilder::build_operator() { } Status PartitionSortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); auto& p = _parent->cast(); RETURN_IF_ERROR(p._vsort_exec_exprs.clone(state, _vsort_exec_exprs)); @@ -117,7 +117,7 @@ Status PartitionSortSinkOperatorX::sink(RuntimeState* state, vectorized::Block* std::lock_guard lock(local_state._shared_state->buffer_mutex); local_state._shared_state->blocks_buffer.push(std::move(*input_block)); // buffer have data, source could read this. - local_state._dependency->set_ready_for_read(); + local_state._dependency->set_ready_to_read(); } } else { RETURN_IF_ERROR( diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.h b/be/src/pipeline/exec/partition_sort_sink_operator.h index 93771ac8e2..a77df27cac 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.h +++ b/be/src/pipeline/exec/partition_sort_sink_operator.h @@ -50,13 +50,21 @@ public: bool can_write() override { return true; } }; +class PartitionSortSinkDependency final : public Dependency { +public: + using SharedState = PartitionSortNodeSharedState; + PartitionSortSinkDependency(int id, int node_id) + : Dependency(id, node_id, "PartitionSortSinkDependency", true) {} + ~PartitionSortSinkDependency() override = default; +}; + class PartitionSortSinkOperatorX; -class PartitionSortSinkLocalState : public PipelineXSinkLocalState { +class PartitionSortSinkLocalState : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(PartitionSortSinkLocalState); public: PartitionSortSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state) {} + : PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; diff --git a/be/src/pipeline/exec/partition_sort_source_operator.cpp b/be/src/pipeline/exec/partition_sort_source_operator.cpp index dfbe1fd40f..0a2f28b19f 100644 --- a/be/src/pipeline/exec/partition_sort_source_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_source_operator.cpp @@ -30,7 +30,7 @@ OperatorPtr PartitionSortSourceOperatorBuilder::build_operator() { } Status PartitionSortSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { - RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); _get_next_timer = ADD_TIMER(profile(), "GetResultTime"); @@ -55,7 +55,7 @@ Status PartitionSortSourceOperatorX::get_block(RuntimeState* state, vectorized:: RETURN_IF_ERROR(vectorized::VExprContext::filter_block( local_state._conjuncts, output_block, output_block->columns())); if (local_state._shared_state->blocks_buffer.empty()) { - local_state._dependency->block_reading(); + local_state._dependency->block(); } return Status::OK(); } diff --git a/be/src/pipeline/exec/partition_sort_source_operator.h b/be/src/pipeline/exec/partition_sort_source_operator.h index 9720e55efa..0ef89c5068 100644 --- a/be/src/pipeline/exec/partition_sort_source_operator.h +++ b/be/src/pipeline/exec/partition_sort_source_operator.h @@ -49,14 +49,22 @@ public: Status open(RuntimeState*) override { return Status::OK(); } }; -class PartitionSortSourceOperatorX; -class PartitionSortSourceLocalState final : public PipelineXLocalState { - ENABLE_FACTORY_CREATOR(PartitionSortSourceLocalState); - +class PartitionSortSourceDependency final : public Dependency { public: - using Base = PipelineXLocalState; + using SharedState = PartitionSortNodeSharedState; + PartitionSortSourceDependency(int id, int node_id) + : Dependency(id, node_id, "PartitionSortSourceDependency") {} + ~PartitionSortSourceDependency() override = default; +}; + +class PartitionSortSourceOperatorX; +class PartitionSortSourceLocalState final + : public PipelineXLocalState { +public: + ENABLE_FACTORY_CREATOR(PartitionSortSourceLocalState); + using Base = PipelineXLocalState; PartitionSortSourceLocalState(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState(state, parent), + : PipelineXLocalState(state, parent), _get_sorted_timer(nullptr), _get_next_timer(nullptr) {} diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index e9a51d56ad..7f6d1673e9 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -167,7 +167,7 @@ Status ResultSinkLocalState::close(RuntimeState* state, Status exec_status) { } SCOPED_TIMER(_close_timer); SCOPED_TIMER(exec_time_counter()); - COUNTER_SET(_wait_for_dependency_timer, _result_sink_dependency->write_watcher_elapse_time()); + COUNTER_SET(_wait_for_dependency_timer, _result_sink_dependency->watcher_elapse_time()); Status final_status = exec_status; if (_writer) { // close the writer diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index 93ce397b84..e7af819e62 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -43,11 +43,11 @@ public: bool can_write() override; }; -class ResultSinkDependency final : public WriteDependency { +class ResultSinkDependency final : public Dependency { public: ENABLE_FACTORY_CREATOR(ResultSinkDependency); ResultSinkDependency(int id, int node_id) - : WriteDependency(id, node_id, "ResultSinkDependency") {} + : Dependency(id, node_id, "ResultSinkDependency", true) {} ~ResultSinkDependency() override = default; }; @@ -61,7 +61,7 @@ public: Status init(RuntimeState* state, LocalSinkStateInfo& info) override; Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; - WriteDependency* dependency() override { return _result_sink_dependency.get(); } + Dependency* dependency() override { return _result_sink_dependency.get(); } RuntimeProfile::Counter* blocks_sent_counter() { return _blocks_sent_counter; } RuntimeProfile::Counter* rows_sent_counter() { return _rows_sent_counter; } diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index fbbf2e05d7..15b77af6ff 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -173,7 +173,7 @@ Status ScanLocalState::open(RuntimeState* state) { auto status = _scan_dependency->eos() ? Status::OK() : _prepare_scanners(); if (_scanner_ctx) { - _finish_dependency->should_finish_after_check(); + _finish_dependency->block(); DCHECK(!_scan_dependency->eos() && _num_scanners->value() > 0); RETURN_IF_ERROR(_scanner_ctx->init()); RETURN_IF_ERROR(state->exec_env()->scanner_scheduler()->submit(_scanner_ctx.get())); @@ -1341,7 +1341,7 @@ Status ScanLocalState::close(RuntimeState* state) { if (_scanner_ctx.get()) { _scanner_ctx->clear_and_join(reinterpret_cast(this), state); } - COUNTER_SET(_wait_for_dependency_timer, _scan_dependency->read_watcher_elapse_time()); + COUNTER_SET(_wait_for_dependency_timer, _scan_dependency->watcher_elapse_time()); return PipelineXLocalState<>::close(state); } diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index f4dd2c45d5..06a6c61129 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -63,33 +63,21 @@ public: : Dependency(id, node_id, "ScanDependency"), _scanner_ctx(nullptr) {} // TODO(gabriel): - [[nodiscard]] Dependency* read_blocked_by(PipelineXTask* task) override { + [[nodiscard]] Dependency* is_blocked_by(PipelineXTask* task) override { if (_scanner_ctx && _scanner_ctx->get_num_running_scanners() == 0 && _scanner_ctx->should_be_scheduled()) { _scanner_ctx->reschedule_scanner_ctx(); } - return Dependency::read_blocked_by(task); + return Dependency::is_blocked_by(task); } bool push_to_blocking_queue() override { return true; } - void block_reading() override { - if (_eos) { - return; - } + void block() override { if (_scanner_done) { return; } - Dependency::block_reading(); - } - - bool eos() const { return _eos.load(); } - void set_eos() { - if (_eos) { - return; - } - _eos = true; - Dependency::set_ready_for_read(); + Dependency::block(); } void set_scanner_done() { @@ -97,14 +85,13 @@ public: return; } _scanner_done = true; - Dependency::set_ready_for_read(); + Dependency::set_ready(); } void set_scanner_ctx(vectorized::ScannerContext* scanner_ctx) { _scanner_ctx = scanner_ctx; } private: vectorized::ScannerContext* _scanner_ctx; - std::atomic _eos {false}; std::atomic _scanner_done {false}; }; diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index de9b48362c..f1615df60d 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -136,11 +136,11 @@ Status SetProbeSinkOperatorX::sink(RuntimeState* state, vectorized template Status SetProbeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); Parent& parent = _parent->cast(); - static_cast(_dependency)->set_cur_child_id(parent._cur_child_id); + _dependency->set_cur_child_id(parent._cur_child_id); _child_exprs.resize(parent._child_exprs.size()); for (size_t i = 0; i < _child_exprs.size(); i++) { RETURN_IF_ERROR(parent._child_exprs[i]->clone(state, _child_exprs[i])); @@ -215,10 +215,11 @@ void SetProbeSinkOperatorX::_finalize_probe( } local_state._probe_columns.resize( local_state._shared_state->child_exprs_lists[_cur_child_id + 1].size()); + local_state._shared_state->probe_finished_children_dependency[_cur_child_id + 1] + ->set_ready(); } else { - local_state._dependency->set_ready_for_read(); + local_state._shared_state->source_dep->set_ready(); } - local_state._shared_state->set_probe_finished_children(_cur_child_id); } template diff --git a/be/src/pipeline/exec/set_probe_sink_operator.h b/be/src/pipeline/exec/set_probe_sink_operator.h index bb5eccdbdc..cd1dbd6267 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.h +++ b/be/src/pipeline/exec/set_probe_sink_operator.h @@ -67,14 +67,31 @@ private: int _child_id; }; +class SetProbeSinkDependency final : public Dependency { +public: + using SharedState = SetSharedState; + SetProbeSinkDependency(int id, int node_id) + : Dependency(id, node_id, "SetProbeSinkDependency", true) {} + ~SetProbeSinkDependency() override = default; + + void set_cur_child_id(int id) { + _child_idx = id; + ((SetSharedState*)_shared_state.get())->probe_finished_children_dependency[id] = this; + block(); + } + +private: + int _child_idx {0}; +}; + template class SetProbeSinkOperatorX; template -class SetProbeSinkLocalState final : public PipelineXSinkLocalState { +class SetProbeSinkLocalState final : public PipelineXSinkLocalState { public: ENABLE_FACTORY_CREATOR(SetProbeSinkLocalState); - using Base = PipelineXSinkLocalState; + using Base = PipelineXSinkLocalState; using Parent = SetProbeSinkOperatorX; SetProbeSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index 4c92c37e48..31e8185720 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -90,9 +90,10 @@ Status SetSinkOperatorX::sink(RuntimeState* state, vectorized::Blo }, *local_state._shared_state->hash_table_variants); } - local_state._shared_state->set_probe_finished_children(_cur_child_id); + local_state._shared_state->probe_finished_children_dependency[_cur_child_id + 1] + ->set_ready(); if (_child_quantity == 1) { - local_state._dependency->set_ready_for_read(); + local_state._shared_state->source_dep->set_ready(); } } } @@ -159,7 +160,7 @@ Status SetSinkOperatorX::_extract_build_column( template Status SetSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); _build_timer = ADD_TIMER(_profile, "BuildTime"); diff --git a/be/src/pipeline/exec/set_sink_operator.h b/be/src/pipeline/exec/set_sink_operator.h index a1b9d8b707..e318c354c8 100644 --- a/be/src/pipeline/exec/set_sink_operator.h +++ b/be/src/pipeline/exec/set_sink_operator.h @@ -60,14 +60,25 @@ private: vectorized::VSetOperationNode* _set_node; }; +class SetSinkDependency final : public Dependency { +public: + using SharedState = SetSharedState; + SetSinkDependency(int id, int node_id) : Dependency(id, node_id, "SetSinkDependency", true) {} + ~SetSinkDependency() override = default; + + void set_cur_child_id(int id) { + ((SetSharedState*)_shared_state.get())->probe_finished_children_dependency[id] = this; + } +}; + template class SetSinkOperatorX; template -class SetSinkLocalState final : public PipelineXSinkLocalState { +class SetSinkLocalState final : public PipelineXSinkLocalState { public: ENABLE_FACTORY_CREATOR(SetSinkLocalState); - using Base = PipelineXSinkLocalState; + using Base = PipelineXSinkLocalState; using Parent = SetSinkOperatorX; SetSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : Base(parent, state) {} diff --git a/be/src/pipeline/exec/set_source_operator.cpp b/be/src/pipeline/exec/set_source_operator.cpp index 6e5e014036..8baadf7e53 100644 --- a/be/src/pipeline/exec/set_source_operator.cpp +++ b/be/src/pipeline/exec/set_source_operator.cpp @@ -51,11 +51,11 @@ template class SetSourceOperator; template Status SetSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { - std::shared_ptr ss = nullptr; - auto& deps = info.dependencys; - ss.reset(new typename SetDependency::SharedState(deps.size())); + std::shared_ptr ss = nullptr; + auto& deps = info.upstream_dependencies; + ss.reset(new typename SetSourceDependency::SharedState(deps.size())); for (auto& dep : deps) { - ((SetDependency*)dep.get())->set_shared_state(ss); + ((SetSourceDependency*)dep.get())->set_shared_state(ss); } RETURN_IF_ERROR(Base::init(state, info)); return Status::OK(); @@ -65,7 +65,7 @@ template Status SetSourceLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(PipelineXLocalState::open(state)); + RETURN_IF_ERROR(PipelineXLocalState::open(state)); auto& child_exprs_lists = _shared_state->child_exprs_lists; auto output_data_types = vectorized::VectorizedUtils::get_data_types( diff --git a/be/src/pipeline/exec/set_source_operator.h b/be/src/pipeline/exec/set_source_operator.h index de81bbe4ec..cc7275444c 100644 --- a/be/src/pipeline/exec/set_source_operator.h +++ b/be/src/pipeline/exec/set_source_operator.h @@ -53,14 +53,21 @@ public: Status open(RuntimeState* /*state*/) override { return Status::OK(); } }; +class SetSourceDependency final : public Dependency { +public: + using SharedState = SetSharedState; + SetSourceDependency(int id, int node_id) : Dependency(id, node_id, "SetSourceDependency") {} + ~SetSourceDependency() override = default; +}; + template class SetSourceOperatorX; template -class SetSourceLocalState final : public PipelineXLocalState { +class SetSourceLocalState final : public PipelineXLocalState { public: ENABLE_FACTORY_CREATOR(SetSourceLocalState); - using Base = PipelineXLocalState; + using Base = PipelineXLocalState; using Parent = SetSourceOperatorX; SetSourceLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent) {}; Status init(RuntimeState* state, LocalStateInfo& infos) override; diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index 5471be54e5..13a204e197 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -29,7 +29,7 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(SortSinkOperator, StreamingOperator) Status SortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); @@ -165,7 +165,7 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in if (source_state == SourceState::FINISHED) { RETURN_IF_ERROR(local_state._shared_state->sorter->prepare_for_read()); - local_state._dependency->set_ready_for_read(); + local_state._dependency->set_ready_to_read(); } return Status::OK(); } diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index ccb19c5456..67305aad69 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -45,16 +45,23 @@ public: bool can_write() override { return true; } }; +class SortSinkDependency final : public Dependency { +public: + using SharedState = SortSharedState; + SortSinkDependency(int id, int node_id) : Dependency(id, node_id, "SortSinkDependency", true) {} + ~SortSinkDependency() override = default; +}; + enum class SortAlgorithm { HEAP_SORT, TOPN_SORT, FULL_SORT }; class SortSinkOperatorX; -class SortSinkLocalState : public PipelineXSinkLocalState { +class SortSinkLocalState : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(SortSinkLocalState); public: SortSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state) {} + : PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp index e13e518f76..24e49d4674 100644 --- a/be/src/pipeline/exec/sort_source_operator.cpp +++ b/be/src/pipeline/exec/sort_source_operator.cpp @@ -26,7 +26,7 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(SortSourceOperator, SourceOperator) SortLocalState::SortLocalState(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState(state, parent) {} + : PipelineXLocalState(state, parent) {} SortSourceOperatorX::SortSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, const DescriptorTbl& descs) diff --git a/be/src/pipeline/exec/sort_source_operator.h b/be/src/pipeline/exec/sort_source_operator.h index 5d2614fce1..3b615a58be 100644 --- a/be/src/pipeline/exec/sort_source_operator.h +++ b/be/src/pipeline/exec/sort_source_operator.h @@ -45,8 +45,15 @@ public: Status open(RuntimeState*) override { return Status::OK(); } }; +class SortSourceDependency final : public Dependency { +public: + using SharedState = SortSharedState; + SortSourceDependency(int id, int node_id) : Dependency(id, node_id, "SortSourceDependency") {} + ~SortSourceDependency() override = default; +}; + class SortSourceOperatorX; -class SortLocalState final : public PipelineXLocalState { +class SortLocalState final : public PipelineXLocalState { public: ENABLE_FACTORY_CREATOR(SortLocalState); SortLocalState(RuntimeState* state, OperatorXBase* parent); diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp index 3052222e24..17eb1b2db1 100644 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp @@ -143,7 +143,6 @@ StreamingAggSinkLocalState::StreamingAggSinkLocalState(DataSinkOperatorXBase* pa Status StreamingAggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); - _shared_state->data_queue.reset(new DataQueue(1, _dependency)); _queue_byte_size_counter = ADD_COUNTER(profile(), "MaxSizeInBlockQueue", TUnit::BYTES); _queue_size_counter = ADD_COUNTER(profile(), "MaxSizeOfBlockQueue", TUnit::UNIT); _streaming_agg_timer = ADD_TIMER(profile(), "StreamingAggTime"); @@ -156,7 +155,7 @@ Status StreamingAggSinkLocalState::do_pre_agg(vectorized::Block* input_block, // pre stream agg need use _num_row_return to decide whether to do pre stream agg _num_rows_returned += output_block->rows(); - _dependency->_make_nullable_output_key(output_block); + _make_nullable_output_key(output_block); // COUNTER_SET(_rows_returned_counter, _num_rows_returned); _executor.update_memusage(); return Status::OK(); @@ -271,7 +270,7 @@ Status StreamingAggSinkLocalState::_pre_agg_with_serialized_key( // non-nullable column(id in `_make_nullable_keys`) // will be converted to nullable. bool mem_reuse = - _dependency->make_nullable_keys().empty() && out_block->mem_reuse(); + _shared_state->make_nullable_keys.empty() && out_block->mem_reuse(); std::vector data_types; vectorized::MutableColumns value_columns; @@ -331,7 +330,7 @@ Status StreamingAggSinkLocalState::_pre_agg_with_serialized_key( for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add( - in_block, _dependency->offsets_of_aggregate_states()[i], _places.data(), + in_block, _shared_state->offsets_of_aggregate_states[i], _places.data(), _agg_arena_pool, _should_expand_hash_table)); } } diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h index 031515ba94..54898864bb 100644 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h @@ -75,10 +75,10 @@ private: class StreamingAggSinkOperatorX; class StreamingAggSinkLocalState final - : public AggSinkLocalState { + : public AggSinkLocalState { public: using Parent = StreamingAggSinkOperatorX; - using Base = AggSinkLocalState; + using Base = AggSinkLocalState; ENABLE_FACTORY_CREATOR(StreamingAggSinkLocalState); StreamingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); ~StreamingAggSinkLocalState() override = default; @@ -93,6 +93,16 @@ private: Status _pre_agg_with_serialized_key(doris::vectorized::Block* in_block, doris::vectorized::Block* out_block); bool _should_expand_preagg_hash_tables(); + void _make_nullable_output_key(vectorized::Block* block) { + if (block->rows() != 0) { + auto& shared_state = *Base ::_shared_state; + for (auto cid : shared_state.make_nullable_keys) { + block->get_by_position(cid).column = + make_nullable(block->get_by_position(cid).column); + block->get_by_position(cid).type = make_nullable(block->get_by_position(cid).type); + } + } + } RuntimeProfile::Counter* _queue_byte_size_counter; RuntimeProfile::Counter* _queue_size_counter; diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp index 353f997ce7..e68be656fe 100644 --- a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp @@ -76,7 +76,7 @@ OperatorPtr StreamingAggSourceOperatorBuilder::build_operator() { StreamingAggSourceOperatorX::StreamingAggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, const DescriptorTbl& descs) - : Base(pool, tnode, operator_id, descs) {} + : Base(pool, tnode, operator_id, descs, true) {} Status StreamingAggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { @@ -84,7 +84,7 @@ Status StreamingAggSourceOperatorX::get_block(RuntimeState* state, vectorized::B SCOPED_TIMER(local_state.exec_time_counter()); if (!local_state._shared_state->data_queue->data_exhausted()) { std::unique_ptr agg_block; - DCHECK(local_state._dependency->read_blocked_by() == nullptr); + DCHECK(local_state._dependency->is_blocked_by() == nullptr); RETURN_IF_ERROR(local_state._shared_state->data_queue->get_block_from_queue(&agg_block)); if (local_state._shared_state->data_queue->data_exhausted()) { diff --git a/be/src/pipeline/exec/union_sink_operator.h b/be/src/pipeline/exec/union_sink_operator.h index f4891ce1c5..e135566417 100644 --- a/be/src/pipeline/exec/union_sink_operator.h +++ b/be/src/pipeline/exec/union_sink_operator.h @@ -66,15 +66,24 @@ private: std::unique_ptr _output_block; }; +class UnionSinkDependency final : public Dependency { +public: + using SharedState = UnionSharedState; + UnionSinkDependency(int id, int node_id) + : Dependency(id, node_id, "UnionSinkDependency", true) {} + ~UnionSinkDependency() override = default; + void block() override {} +}; + class UnionSinkOperatorX; -class UnionSinkLocalState final : public PipelineXSinkLocalState { +class UnionSinkLocalState final : public PipelineXSinkLocalState { public: ENABLE_FACTORY_CREATOR(UnionSinkLocalState); UnionSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : Base(parent, state), _child_row_idx(0) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; friend class UnionSinkOperatorX; - using Base = PipelineXSinkLocalState; + using Base = PipelineXSinkLocalState; using Parent = UnionSinkOperatorX; private: diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index d1515faa1b..fc1dd124a0 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -23,6 +23,7 @@ #include "common/status.h" #include "pipeline/exec/data_queue.h" #include "pipeline/exec/operator.h" +#include "pipeline/exec/union_sink_operator.h" #include "pipeline/pipeline_x/dependency.h" #include "runtime/descriptors.h" #include "vec/core/block.h" @@ -104,22 +105,22 @@ Status UnionSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { int child_count = p.get_child_count(); auto ss = create_shared_state(); if (child_count != 0) { - auto& deps = info.dependencys; + auto& deps = info.upstream_dependencies; for (auto& dep : deps) { - ((UnionDependency*)dep.get())->set_shared_state(ss); + ((UnionSinkDependency*)dep.get())->set_shared_state(ss); } } else { - auto& deps = info.dependencys; + auto& deps = info.upstream_dependencies; DCHECK(child_count == 0); DCHECK(deps.size() == 1); DCHECK(deps.front() == nullptr); //child_count == 0 , we need to creat a UnionDependency deps.front() = - std::make_shared(_parent->operator_id(), _parent->node_id()); - ((UnionDependency*)deps.front().get())->set_shared_state(ss); + std::make_shared(_parent->operator_id(), _parent->node_id()); + ((UnionSourceDependency*)deps.front().get())->set_shared_state(ss); } RETURN_IF_ERROR(Base::init(state, info)); - ss->data_queue.set_dependency(_dependency); + ss->data_queue.set_dependency(_dependency, info.upstream_dependencies.front().get()); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); // Const exprs materialized by this node. These exprs don't refer to any children. @@ -146,7 +147,7 @@ Status UnionSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { std::shared_ptr UnionSourceLocalState::create_shared_state() { auto& p = _parent->cast(); std::shared_ptr data_queue = - std::make_shared(p._child_size, _dependency); + std::make_shared(p._child_size); return data_queue; } diff --git a/be/src/pipeline/exec/union_source_operator.h b/be/src/pipeline/exec/union_source_operator.h index 0cf2d3d403..0c846b828f 100644 --- a/be/src/pipeline/exec/union_source_operator.h +++ b/be/src/pipeline/exec/union_source_operator.h @@ -69,11 +69,31 @@ private: bool _need_read_for_const_expr; }; +class UnionSourceDependency final : public Dependency { +public: + using SharedState = UnionSharedState; + UnionSourceDependency(int id, int node_id) : Dependency(id, node_id, "UnionSourceDependency") {} + ~UnionSourceDependency() override = default; + + [[nodiscard]] Dependency* is_blocked_by(PipelineXTask* task) override { + if (((UnionSharedState*)_shared_state.get())->child_count() == 0) { + return nullptr; + } + if (((UnionSharedState*)_shared_state.get())->data_queue.is_all_finish() || + ((UnionSharedState*)_shared_state.get())->data_queue.remaining_has_data()) { + return nullptr; + } + return this; + } + bool push_to_blocking_queue() override { return true; } + void block() override {} +}; + class UnionSourceOperatorX; -class UnionSourceLocalState final : public PipelineXLocalState { +class UnionSourceLocalState final : public PipelineXLocalState { public: ENABLE_FACTORY_CREATOR(UnionSourceLocalState); - using Base = PipelineXLocalState; + using Base = PipelineXLocalState; using Parent = UnionSourceOperatorX; UnionSourceLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent) {}; diff --git a/be/src/pipeline/pipeline_x/dependency.cpp b/be/src/pipeline/pipeline_x/dependency.cpp index f21a48dc5f..c149da54fd 100644 --- a/be/src/pipeline/pipeline_x/dependency.cpp +++ b/be/src/pipeline/pipeline_x/dependency.cpp @@ -34,42 +34,18 @@ void Dependency::add_block_task(PipelineXTask* task) { _blocked_task.push_back(task); } -void WriteDependency::add_write_block_task(PipelineXTask* task) { - DCHECK(_write_blocked_task.empty() || - _write_blocked_task[_write_blocked_task.size() - 1] != task) - << "Duplicate task: " << task->debug_string(); - _write_blocked_task.push_back(task); -} - -void FinishDependency::add_block_task(PipelineXTask* task) { - DCHECK(_finish_blocked_task.empty() || - _finish_blocked_task[_finish_blocked_task.size() - 1] != task) - << "Duplicate task: " << task->debug_string(); - _finish_blocked_task.push_back(task); -} - -void RuntimeFilterDependency::add_block_task(PipelineXTask* task) { - DCHECK(_filter_blocked_task.empty() || - _filter_blocked_task[_filter_blocked_task.size() - 1] != task) - << "Duplicate task: " << task->debug_string(); - DCHECK(_blocked_by_rf) << "It is not allowed: task: " << task->debug_string() - << " \n dependency: " << debug_string() - << " \n state: " << get_state_name(task->get_state()); - _filter_blocked_task.push_back(task); -} - -void Dependency::set_ready_for_read() { - if (_ready_for_read) { +void Dependency::set_ready() { + if (_ready) { return; } - _read_dependency_watcher.stop(); + _watcher.stop(); std::vector local_block_task {}; { std::unique_lock lc(_task_lock); - if (_ready_for_read) { + if (_ready) { return; } - _ready_for_read = true; + _ready = true; local_block_task.swap(_blocked_task); } for (auto* task : local_block_task) { @@ -77,78 +53,22 @@ void Dependency::set_ready_for_read() { } } -void SetDependency::set_ready_for_read() { - if (_child_idx == 0) { - WriteDependency::set_ready_for_read(); - } else { - _set_state->probe_finished_children_dependency[0]->set_ready_for_read(); - } -} - -void WriteDependency::set_ready_for_write() { - if (_ready_for_write) { - return; - } - _write_dependency_watcher.stop(); - - std::vector local_block_task {}; - { - std::unique_lock lc(_write_task_lock); - if (_ready_for_write) { - return; - } - _ready_for_write = true; - local_block_task.swap(_write_blocked_task); - } - for (auto* task : local_block_task) { - task->try_wake_up(this); - } -} - -void FinishDependency::set_ready_to_finish() { - if (_ready_to_finish) { - return; - } - _finish_dependency_watcher.stop(); - - std::vector local_block_task {}; - { - std::unique_lock lc(_task_lock); - if (_ready_to_finish) { - return; - } - _ready_to_finish = true; - local_block_task.swap(_finish_blocked_task); - } - for (auto* task : local_block_task) { - task->try_wake_up(this); - } -} - -Dependency* Dependency::read_blocked_by(PipelineXTask* task) { - if (config::enable_fuzzy_mode && !_ready_for_read && - _should_log(_read_dependency_watcher.elapsed_time())) { +Dependency* Dependency::is_blocked_by(PipelineXTask* task) { + if (config::enable_fuzzy_mode && !_ready && _should_log(_watcher.elapsed_time())) { LOG(WARNING) << "========Dependency may be blocked by some reasons: " << name() << " " << _node_id << " block tasks: " << _blocked_task.size() - << " write block tasks: " - << (is_write_dependency() - ? ((WriteDependency*)this)->_write_blocked_task.size() - : 0) - << " write done: " - << (is_write_dependency() ? ((WriteDependency*)this)->_ready_for_write.load() - : true) << "task: " << (task ? task->fragment_context()->debug_string() : ""); } std::unique_lock lc(_task_lock); - auto ready_for_read = _ready_for_read.load(); - if (!ready_for_read && !push_to_blocking_queue() && task) { + auto ready = _ready.load(); + if (!ready && !push_to_blocking_queue() && task) { add_block_task(task); } - return ready_for_read ? nullptr : this; + return ready ? nullptr : this; } -RuntimeFilterDependency* RuntimeFilterDependency::filter_blocked_by(PipelineXTask* task) { +Dependency* RuntimeFilterDependency::is_blocked_by(PipelineXTask* task) { if (!_blocked_by_rf) { return nullptr; } @@ -162,67 +82,11 @@ RuntimeFilterDependency* RuntimeFilterDependency::filter_blocked_by(PipelineXTas return nullptr; } -FinishDependency* FinishDependency::finish_blocked_by(PipelineXTask* task) { - std::unique_lock lc(_task_lock); - if (!_ready_to_finish && task) { - add_block_task(task); - } - return _ready_to_finish ? nullptr : this; -} - -WriteDependency* WriteDependency::write_blocked_by(PipelineXTask* task) { - std::unique_lock lc(_write_task_lock); - const auto ready_for_write = _ready_for_write.load(); - if (!ready_for_write && task) { - add_write_block_task(task); - } - return ready_for_write ? nullptr : this; -} - -template Status HashJoinDependency::extract_join_column( - vectorized::Block&, - COW::mutable_ptr>&, - std::vector>&, - std::vector> const&); - -template Status HashJoinDependency::extract_join_column( - vectorized::Block&, - COW::mutable_ptr>&, - std::vector>&, - std::vector> const&); - std::string Dependency::debug_string(int indentation_level) { fmt::memory_buffer debug_string_buffer; - fmt::format_to(debug_string_buffer, "{}{}: id={}, block task = {}, _ready_for_read={}", + fmt::format_to(debug_string_buffer, "{}{}: id={}, block task = {}, ready={}", std::string(indentation_level * 2, ' '), _name, _node_id, _blocked_task.size(), - _ready_for_read); - return fmt::to_string(debug_string_buffer); -} - -std::string WriteDependency::debug_string(int indentation_level) { - fmt::memory_buffer debug_string_buffer; - fmt::format_to(debug_string_buffer, - "{}{}: id={}, read block task = {},write block " - "task = {}, _ready_for_write = {}, _ready_for_read = {}", - std::string(indentation_level * 2, ' '), _name, _node_id, _blocked_task.size(), - _write_blocked_task.size(), _ready_for_write, _ready_for_read); - return fmt::to_string(debug_string_buffer); -} - -std::string FinishDependency::debug_string(int indentation_level) { - fmt::memory_buffer debug_string_buffer; - fmt::format_to(debug_string_buffer, "{}{}: id={}, block task = {}, _ready_to_finish = {}", - std::string(indentation_level * 2, ' '), _name, _node_id, - _finish_blocked_task.size(), _ready_to_finish); - return fmt::to_string(debug_string_buffer); -} - -std::string RuntimeFilterDependency::debug_string(int indentation_level) { - fmt::memory_buffer debug_string_buffer; - fmt::format_to( - debug_string_buffer, "{}{}: id={}, block task = {}, _blocked_by_rf = {}, _filters = {}", - std::string(indentation_level * 2, ' '), _name, _node_id, _filter_blocked_task.size(), - _blocked_by_rf ? _blocked_by_rf->load() : false, _filters); + _ready); return fmt::to_string(debug_string_buffer); } @@ -237,273 +101,6 @@ std::string AndDependency::debug_string(int indentation_level) { return fmt::to_string(debug_string_buffer); } -Status AggDependency::reset_hash_table() { - return std::visit( - [&](auto&& agg_method) { - auto& hash_table = *agg_method.hash_table; - using HashTableType = std::decay_t; - - agg_method.reset(); - - hash_table.for_each_mapped([&](auto& mapped) { - if (mapped) { - static_cast(destroy_agg_status(mapped)); - mapped = nullptr; - } - }); - - _agg_state.aggregate_data_container.reset(new vectorized::AggregateDataContainer( - sizeof(typename HashTableType::key_type), - ((_total_size_of_aggregate_states + _align_aggregate_states - 1) / - _align_aggregate_states) * - _align_aggregate_states)); - hash_table = HashTableType(); - _agg_state.agg_arena_pool.reset(new vectorized::Arena); - return Status::OK(); - }, - _agg_state.agg_data->method_variant); -} - -Status AggDependency::destroy_agg_status(vectorized::AggregateDataPtr data) { - for (int i = 0; i < _agg_state.aggregate_evaluators.size(); ++i) { - _agg_state.aggregate_evaluators[i]->function()->destroy(data + - _offsets_of_aggregate_states[i]); - } - return Status::OK(); -} - -Status AggDependency::create_agg_status(vectorized::AggregateDataPtr data) { - for (int i = 0; i < _agg_state.aggregate_evaluators.size(); ++i) { - try { - _agg_state.aggregate_evaluators[i]->create(data + _offsets_of_aggregate_states[i]); - } catch (...) { - for (int j = 0; j < i; ++j) { - _agg_state.aggregate_evaluators[j]->destroy(data + _offsets_of_aggregate_states[j]); - } - throw; - } - } - return Status::OK(); -} - -Status AggDependency::merge_spilt_data() { - CHECK(!_agg_state.spill_context.stream_ids.empty()); - - for (auto& reader : _agg_state.spill_context.readers) { - CHECK_LT(_agg_state.spill_context.read_cursor, reader->block_count()); - reader->seek(_agg_state.spill_context.read_cursor); - vectorized::Block block; - bool eos = false; - RETURN_IF_ERROR(reader->read(&block, &eos)); - - // TODO - // if (!block.empty()) { - // auto st = _merge_with_serialized_key_helper( - // &block); - // RETURN_IF_ERROR(st); - // } - } - _agg_state.spill_context.read_cursor++; - return Status::OK(); -} - -void AggDependency::release_tracker() { - mem_tracker()->release(_mem_usage_record.used_in_state + _mem_usage_record.used_in_arena); -} - -vectorized::BlockRowPos AnalyticDependency::get_partition_by_end() { - if (_analytic_state.current_row_position < - _analytic_state.partition_by_end.pos) { //still have data, return partition_by_end directly - return _analytic_state.partition_by_end; - } - - if (_analytic_state.partition_by_eq_expr_ctxs.empty() || - (_analytic_state.input_total_rows == 0)) { //no partition_by, the all block is end - return _analytic_state.all_block_end; - } - - vectorized::BlockRowPos cal_end = _analytic_state.all_block_end; - for (size_t i = 0; i < _analytic_state.partition_by_eq_expr_ctxs.size(); - ++i) { //have partition_by, binary search the partiton end - cal_end = compare_row_to_find_end(_analytic_state.partition_by_column_idxs[i], - _analytic_state.partition_by_end, cal_end); - } - cal_end.pos = - _analytic_state.input_block_first_row_positions[cal_end.block_num] + cal_end.row_num; - return cal_end; -} - -//_partition_by_columns,_order_by_columns save in blocks, so if need to calculate the boundary, may find in which blocks firstly -vectorized::BlockRowPos AnalyticDependency::compare_row_to_find_end(int idx, - vectorized::BlockRowPos start, - vectorized::BlockRowPos end, - bool need_check_first) { - int64_t start_init_row_num = start.row_num; - vectorized::ColumnPtr start_column = - _analytic_state.input_blocks[start.block_num].get_by_position(idx).column; - vectorized::ColumnPtr start_next_block_column = start_column; - - DCHECK_LE(start.block_num, end.block_num); - DCHECK_LE(start.block_num, _analytic_state.input_blocks.size() - 1); - int64_t start_block_num = start.block_num; - int64_t end_block_num = end.block_num; - int64_t mid_blcok_num = end.block_num; - // To fix this problem: https://github.com/apache/doris/issues/15951 - // in this case, the partition by column is last row of block, so it's pointed to a new block at row = 0, range is: [left, right) - // From the perspective of order by column, the two values are exactly equal. - // so the range will be get wrong because it's compare_at == 0 with next block at row = 0 - if (need_check_first && end.block_num > 0 && end.row_num == 0) { - end.block_num--; - end_block_num--; - end.row_num = _analytic_state.input_blocks[end_block_num].rows(); - } - //binary search find in which block - while (start_block_num < end_block_num) { - mid_blcok_num = (start_block_num + end_block_num + 1) >> 1; - start_next_block_column = - _analytic_state.input_blocks[mid_blcok_num].get_by_position(idx).column; - //Compares (*this)[n] and rhs[m], this: start[init_row] rhs: mid[0] - if (start_column->compare_at(start_init_row_num, 0, *start_next_block_column, 1) == 0) { - start_block_num = mid_blcok_num; - } else { - end_block_num = mid_blcok_num - 1; - } - } - - // have check the start.block_num: start_column[start_init_row_num] with mid_blcok_num start_next_block_column[0] - // now next block must not be result, so need check with end_block_num: start_next_block_column[last_row] - if (end_block_num == mid_blcok_num - 1) { - start_next_block_column = - _analytic_state.input_blocks[end_block_num].get_by_position(idx).column; - int64_t block_size = _analytic_state.input_blocks[end_block_num].rows(); - if ((start_column->compare_at(start_init_row_num, block_size - 1, *start_next_block_column, - 1) == 0)) { - start.block_num = end_block_num + 1; - start.row_num = 0; - return start; - } - } - - //check whether need get column again, maybe same as first init - // if the start_block_num have move to forword, so need update start block num and compare it from row_num=0 - if (start_block_num != start.block_num) { - start_init_row_num = 0; - start.block_num = start_block_num; - start_column = _analytic_state.input_blocks[start.block_num].get_by_position(idx).column; - } - //binary search, set start and end pos - int64_t start_pos = start_init_row_num; - int64_t end_pos = _analytic_state.input_blocks[start.block_num].rows(); - //if end_block_num haven't moved, only start_block_num go to the end block - //so could use the end.row_num for binary search - if (start.block_num == end.block_num) { - end_pos = end.row_num; - } - while (start_pos < end_pos) { - int64_t mid_pos = (start_pos + end_pos) >> 1; - if (start_column->compare_at(start_init_row_num, mid_pos, *start_column, 1)) { - end_pos = mid_pos; - } else { - start_pos = mid_pos + 1; - } - } - start.row_num = start_pos; //update row num, return the find end - return start; -} - -bool AnalyticDependency::whether_need_next_partition(vectorized::BlockRowPos& found_partition_end) { - if (_analytic_state.input_eos || - (_analytic_state.current_row_position < - _analytic_state.partition_by_end.pos)) { //now still have partition data - return false; - } - if ((_analytic_state.partition_by_eq_expr_ctxs.empty() && !_analytic_state.input_eos) || - (found_partition_end.pos == 0)) { //no partition, get until fetch to EOS - return true; - } - if (!_analytic_state.partition_by_eq_expr_ctxs.empty() && - found_partition_end.pos == _analytic_state.all_block_end.pos && - !_analytic_state.input_eos) { //current partition data calculate done - return true; - } - return false; -} - -Status HashJoinDependency::do_evaluate(vectorized::Block& block, - vectorized::VExprContextSPtrs& exprs, - RuntimeProfile::Counter& expr_call_timer, - std::vector& res_col_ids) { - for (size_t i = 0; i < exprs.size(); ++i) { - int result_col_id = -1; - // execute build column - { - SCOPED_TIMER(&expr_call_timer); - RETURN_IF_ERROR(exprs[i]->execute(&block, &result_col_id)); - } - - // TODO: opt the column is const - block.get_by_position(result_col_id).column = - block.get_by_position(result_col_id).column->convert_to_full_column_if_const(); - res_col_ids[i] = result_col_id; - } - return Status::OK(); -} - -std::vector HashJoinDependency::convert_block_to_null(vectorized::Block& block) { - std::vector results; - for (int i = 0; i < block.columns(); ++i) { - if (auto& column_type = block.safe_get_by_position(i); !column_type.type->is_nullable()) { - DCHECK(!column_type.column->is_nullable()); - column_type.column = make_nullable(column_type.column); - column_type.type = make_nullable(column_type.type); - results.emplace_back(i); - } - } - return results; -} - -void SetSharedState::set_probe_finished_children(int child_id) { - if (child_id + 1 < probe_finished_children_dependency.size()) { - probe_finished_children_dependency[child_id + 1]->set_ready_for_write(); - } -} - -template -Status HashJoinDependency::extract_join_column(vectorized::Block& block, - vectorized::ColumnUInt8::MutablePtr& null_map, - vectorized::ColumnRawPtrs& raw_ptrs, - const std::vector& res_col_ids) { - for (size_t i = 0; i < _join_state.build_exprs_size; ++i) { - if (_join_state.is_null_safe_eq_join[i]) { - raw_ptrs[i] = block.get_by_position(res_col_ids[i]).column.get(); - } else { - auto column = block.get_by_position(res_col_ids[i]).column.get(); - if (auto* nullable = check_and_get_column(*column)) { - auto& col_nested = nullable->get_nested_column(); - auto& col_nullmap = nullable->get_null_map_data(); - - if constexpr (!BuildSide) { - DCHECK(null_map != nullptr); - vectorized::VectorizedUtils::update_null_map(null_map->get_data(), col_nullmap); - } - if (_join_state.store_null_in_hash_table[i]) { - raw_ptrs[i] = nullable; - } else { - if constexpr (BuildSide) { - DCHECK(null_map != nullptr); - vectorized::VectorizedUtils::update_null_map(null_map->get_data(), - col_nullmap); - } - raw_ptrs[i] = &col_nested; - } - } else { - raw_ptrs[i] = column; - } - } - } - return Status::OK(); -} - bool RuntimeFilterTimer::has_ready() { std::unique_lock lc(_lock); return _runtime_filter->is_ready(); @@ -613,7 +210,7 @@ void RuntimeFilterDependency::sub_filters() { { std::unique_lock lc(_task_lock); *_blocked_by_rf = false; - local_block_task.swap(_filter_blocked_task); + local_block_task.swap(_blocked_task); } for (auto* task : local_block_task) { task->try_wake_up(this); diff --git a/be/src/pipeline/pipeline_x/dependency.h b/be/src/pipeline/pipeline_x/dependency.h index 937aa01bd8..662e789d65 100644 --- a/be/src/pipeline/pipeline_x/dependency.h +++ b/be/src/pipeline/pipeline_x/dependency.h @@ -41,53 +41,89 @@ #include "vec/exec/vpartition_sort_node.h" namespace doris::pipeline { + class Dependency; +class AnalyticSourceDependency; +class AnalyticSinkDependency; class PipelineXTask; +struct BasicSharedState; using DependencySPtr = std::shared_ptr; +using DependencyMap = std::map>; static constexpr auto SLOW_DEPENDENCY_THRESHOLD = 60 * 1000L * 1000L * 1000L; static constexpr auto TIME_UNIT_DEPENDENCY_LOG = 30 * 1000L * 1000L * 1000L; static_assert(TIME_UNIT_DEPENDENCY_LOG < SLOW_DEPENDENCY_THRESHOLD); +struct BasicSharedState { + Dependency* source_dep; + Dependency* sink_dep; +}; + class Dependency : public std::enable_shared_from_this { public: Dependency(int id, int node_id, std::string name) - : _id(id), _node_id(node_id), _name(std::move(name)), _ready_for_read(false) {} + : _id(id), + _node_id(node_id), + _name(std::move(name)), + _is_write_dependency(false), + _ready(false) {} + Dependency(int id, int node_id, std::string name, bool ready) + : _id(id), + _node_id(node_id), + _name(std::move(name)), + _is_write_dependency(true), + _ready(ready) {} virtual ~Dependency() = default; [[nodiscard]] int id() const { return _id; } [[nodiscard]] virtual std::string name() const { return _name; } - virtual void* shared_state() { return nullptr; } + void set_parent(std::weak_ptr parent) { _parent = parent; } + void add_child(std::shared_ptr child) { _children.push_back(child); } + std::shared_ptr shared_state() { return _shared_state; } + void set_shared_state(std::shared_ptr shared_state) { + _shared_state = shared_state; + } virtual std::string debug_string(int indentation_level = 0); - virtual bool is_write_dependency() { return false; } virtual bool push_to_blocking_queue() { return false; } // Start the watcher. We use it to count how long this dependency block the current pipeline task. - void start_read_watcher() { + void start_watcher() { for (auto& child : _children) { - child->start_read_watcher(); + child->start_watcher(); } - _read_dependency_watcher.start(); - } - - [[nodiscard]] virtual int64_t read_watcher_elapse_time() { - return _read_dependency_watcher.elapsed_time(); + _watcher.start(); } + [[nodiscard]] int64_t watcher_elapse_time() { return _watcher.elapsed_time(); } // Which dependency current pipeline task is blocked by. `nullptr` if this dependency is ready. - [[nodiscard]] virtual Dependency* read_blocked_by(PipelineXTask* task = nullptr); - + [[nodiscard]] virtual Dependency* is_blocked_by(PipelineXTask* task = nullptr); // Notify downstream pipeline tasks this dependency is ready. - virtual void set_ready_for_read(); + virtual void set_ready(); + void set_ready_to_read() { + DCHECK(_is_write_dependency) << debug_string(); + DCHECK(_shared_state->source_dep != nullptr) << debug_string(); + _shared_state->source_dep->set_ready(); + } + void set_eos() { + if (_eos) { + return; + } + _eos = true; + set_ready(); + if (_is_write_dependency && _shared_state->source_dep != nullptr) { + _shared_state->source_dep->set_eos(); + } + } + bool eos() const { return _eos.load(); } // Notify downstream pipeline tasks this dependency is blocked. - virtual void block_reading() { _ready_for_read = false; } - - void set_parent(std::weak_ptr parent) { _parent = parent; } - - virtual void add_child(std::shared_ptr child) { _children.push_back(child); } - - virtual void add_block_task(PipelineXTask* task); + virtual void block() { + if (_eos) { + return; + } + _ready = false; + } + void add_block_task(PipelineXTask* task); protected: bool _should_log(uint64_t cur_time) { @@ -101,87 +137,31 @@ protected: return true; } - int _id; + const int _id; const int _node_id; - std::string _name; - std::atomic _ready_for_read; - MonotonicStopWatch _read_dependency_watcher; + const std::string _name; + const bool _is_write_dependency; + std::shared_ptr _shared_state {nullptr}; + std::atomic _ready; + MonotonicStopWatch _watcher; std::weak_ptr _parent; - std::list> _children; uint64_t _last_log_time = 0; std::mutex _task_lock; std::vector _blocked_task; + std::atomic _eos {false}; }; -class WriteDependency : public Dependency { +struct FakeSharedState : public BasicSharedState {}; + +struct FakeDependency final : public Dependency { public: - WriteDependency(int id, int node_id, std::string name) : Dependency(id, node_id, name) {} - ~WriteDependency() override = default; + using SharedState = FakeSharedState; + FakeDependency(int id, int node_id) : Dependency(id, node_id, "FakeDependency") {} - bool is_write_dependency() override { return true; } - void start_write_watcher() { - for (auto& child : _children) { - CHECK(child->is_write_dependency()); - ((WriteDependency*)child.get())->start_write_watcher(); - } - _write_dependency_watcher.start(); - } - - [[nodiscard]] virtual int64_t write_watcher_elapse_time() { - return _write_dependency_watcher.elapsed_time(); - } - - [[nodiscard]] virtual WriteDependency* write_blocked_by(PipelineXTask* task); - - virtual void set_ready_for_write(); - - virtual void block_writing() { _ready_for_write = false; } - - std::string debug_string(int indentation_level = 0) override; - void add_write_block_task(PipelineXTask* task); - -protected: - friend class Dependency; - std::atomic _ready_for_write {true}; - std::mutex _write_task_lock; - MonotonicStopWatch _write_dependency_watcher; - -private: - std::vector _write_blocked_task; -}; - -class FinishDependency final : public Dependency { -public: - FinishDependency(int id, int node_id, std::string name) : Dependency(id, node_id, name) {} - ~FinishDependency() override = default; - - void should_finish_after_check() { _ready_to_finish = false; } - void start_finish_watcher() { - for (auto& child : _children) { - ((FinishDependency*)child.get())->start_finish_watcher(); - } - _finish_dependency_watcher.start(); - } - - [[nodiscard]] int64_t finish_watcher_elapse_time() { - return _finish_dependency_watcher.elapsed_time(); - } - - [[nodiscard]] FinishDependency* finish_blocked_by(PipelineXTask* task); - - void set_ready_to_finish(); - - std::string debug_string(int indentation_level = 0) override; - - void add_block_task(PipelineXTask* task) override; - -private: - std::atomic _ready_to_finish {true}; - MonotonicStopWatch _finish_dependency_watcher; - std::vector _finish_blocked_task; + [[nodiscard]] Dependency* is_blocked_by(PipelineXTask* task) override { return nullptr; } }; class RuntimeFilterDependency; @@ -222,33 +202,28 @@ class RuntimeFilterDependency final : public Dependency { public: RuntimeFilterDependency(int id, int node_id, std::string name) : Dependency(id, node_id, name) {} - RuntimeFilterDependency* filter_blocked_by(PipelineXTask* task); + Dependency* is_blocked_by(PipelineXTask* task); void add_filters(IRuntimeFilter* runtime_filter); void sub_filters(); void set_blocked_by_rf(std::shared_ptr blocked_by_rf) { _blocked_by_rf = blocked_by_rf; } - std::string debug_string(int indentation_level = 0) override; - - void add_block_task(PipelineXTask* task) override; protected: std::atomic_int _filters; std::shared_ptr _blocked_by_rf; - -private: - std::vector _filter_blocked_task; }; -class AndDependency final : public WriteDependency { +class AndDependency final : public Dependency { public: + using SharedState = FakeSharedState; ENABLE_FACTORY_CREATOR(AndDependency); - AndDependency(int id, int node_id) : WriteDependency(id, node_id, "AndDependency") {} + AndDependency(int id, int node_id) : Dependency(id, node_id, "AndDependency") {} [[nodiscard]] std::string name() const override { fmt::memory_buffer debug_string_buffer; - fmt::format_to(debug_string_buffer, "{}[", _name); - for (auto& child : _children) { + fmt::format_to(debug_string_buffer, "{}[", Dependency::_name); + for (auto& child : Dependency::_children) { fmt::format_to(debug_string_buffer, "{}, ", child->name()); } fmt::format_to(debug_string_buffer, "]"); @@ -257,19 +232,9 @@ public: std::string debug_string(int indentation_level = 0) override; - [[nodiscard]] Dependency* read_blocked_by(PipelineXTask* task) override { - for (auto& child : _children) { - if (auto* dep = child->read_blocked_by(task)) { - return dep; - } - } - return nullptr; - } - - [[nodiscard]] WriteDependency* write_blocked_by(PipelineXTask* task) override { - for (auto& child : _children) { - CHECK(child->is_write_dependency()); - if (auto* dep = ((WriteDependency*)child.get())->write_blocked_by(task)) { + [[nodiscard]] Dependency* is_blocked_by(PipelineXTask* task) override { + for (auto& child : Dependency::_children) { + if (auto* dep = child->is_blocked_by(task)) { return dep; } } @@ -277,20 +242,7 @@ public: } }; -struct FakeSharedState {}; -struct FakeDependency final : public WriteDependency { -public: - FakeDependency(int id, int node_id) : WriteDependency(id, node_id, "FakeDependency") {} - using SharedState = FakeSharedState; - [[nodiscard]] Dependency* read_blocked_by(PipelineXTask* task) override { return nullptr; } - [[nodiscard]] WriteDependency* write_blocked_by(PipelineXTask* task) override { - return nullptr; - } - [[nodiscard]] int64_t read_watcher_elapse_time() override { return 0; } - [[nodiscard]] int64_t write_watcher_elapse_time() override { return 0; } -}; - -struct AggSharedState { +struct AggSharedState : public BasicSharedState { public: AggSharedState() { agg_data = std::make_unique(); @@ -313,186 +265,43 @@ public: std::vector values; std::unique_ptr agg_profile_arena; std::unique_ptr data_queue = nullptr; -}; - -class AggDependency final : public WriteDependency { -public: - using SharedState = AggSharedState; - AggDependency(int id, int node_id) : WriteDependency(id, node_id, "AggDependency") { - _mem_tracker = std::make_unique("AggregateOperator:"); - } - ~AggDependency() override = default; - - void block_reading() override { - if (_is_streaming_agg_state()) { - if (_agg_state.data_queue->_cur_blocks_nums_in_queue[0] == 0 && - !_agg_state.data_queue->_is_finished[0]) { - Dependency::block_reading(); - } - } else { - Dependency::block_reading(); - } - } - - void block_writing() override { - if (_is_streaming_agg_state()) { - if (!_agg_state.data_queue->has_enough_space_to_push()) { - WriteDependency::block_writing(); - } - } else { - WriteDependency::block_writing(); - } - } - - void set_ready_for_write() override { - if (_is_streaming_agg_state()) { - if (_agg_state.data_queue->has_enough_space_to_push()) { - WriteDependency::set_ready_for_write(); - } - } else { - WriteDependency::set_ready_for_write(); - } - } - - void* shared_state() override { return (void*)&_agg_state; }; - - Status reset_hash_table(); - - Status merge_spilt_data(); - - void set_total_size_of_aggregate_states(size_t total_size_of_aggregate_states) { - _total_size_of_aggregate_states = total_size_of_aggregate_states; - } - void set_align_aggregate_states(size_t align_aggregate_states) { - _align_aggregate_states = align_aggregate_states; - } - - void set_offsets_of_aggregate_states(vectorized::Sizes& offsets_of_aggregate_states) { - _offsets_of_aggregate_states = offsets_of_aggregate_states; - } - - Status destroy_agg_status(vectorized::AggregateDataPtr data); - Status create_agg_status(vectorized::AggregateDataPtr data); - - const vectorized::Sizes& offsets_of_aggregate_states() { return _offsets_of_aggregate_states; } - - void set_make_nullable_keys(std::vector& make_nullable_keys) { - _make_nullable_keys = make_nullable_keys; - } - void _make_nullable_output_key(vectorized::Block* block) { - if (block->rows() != 0) { - for (auto cid : _make_nullable_keys) { - block->get_by_position(cid).column = - make_nullable(block->get_by_position(cid).column); - block->get_by_position(cid).type = make_nullable(block->get_by_position(cid).type); - } - } - } - const std::vector& make_nullable_keys() { return _make_nullable_keys; } - void release_tracker(); + /// The total size of the row from the aggregate functions. + size_t total_size_of_aggregate_states = 0; + size_t align_aggregate_states = 1; + /// The offset to the n-th aggregate function in a row of aggregate functions. + vectorized::Sizes offsets_of_aggregate_states; + std::vector make_nullable_keys; struct MemoryRecord { MemoryRecord() : used_in_arena(0), used_in_state(0) {} int64_t used_in_arena; int64_t used_in_state; }; - MemoryRecord& mem_usage_record() { return _mem_usage_record; } - - MemTracker* mem_tracker() { return _mem_tracker.get(); } - -protected: - /// The total size of the row from the aggregate functions. - size_t _total_size_of_aggregate_states = 0; - size_t _align_aggregate_states = 1; - /// The offset to the n-th aggregate function in a row of aggregate functions. - vectorized::Sizes _offsets_of_aggregate_states; - std::vector _make_nullable_keys; - - MemoryRecord _mem_usage_record; - std::unique_ptr _mem_tracker; - -private: - bool _is_streaming_agg_state() { return _agg_state.data_queue != nullptr; } - AggSharedState _agg_state; + MemoryRecord mem_usage_record; + std::unique_ptr mem_tracker = std::make_unique("AggregateOperator"); }; -struct SortSharedState { +struct SortSharedState : public BasicSharedState { public: std::unique_ptr sorter; }; -class SortDependency final : public WriteDependency { +struct UnionSharedState : public BasicSharedState { public: - using SharedState = SortSharedState; - SortDependency(int id, int node_id) : WriteDependency(id, node_id, "SortDependency") {} - ~SortDependency() override = default; - void* shared_state() override { return (void*)&_sort_state; }; - -private: - SortSharedState _sort_state; -}; - -struct UnionSharedState { -public: - UnionSharedState(int child_count = 1, WriteDependency* dependency = nullptr) - : data_queue(child_count, dependency), _child_count(child_count) {}; + UnionSharedState(int child_count = 1) : data_queue(child_count), _child_count(child_count) {}; int child_count() const { return _child_count; } DataQueue data_queue; const int _child_count; }; -class UnionDependency final : public WriteDependency { -public: - using SharedState = UnionSharedState; - UnionDependency(int id, int node_id) : WriteDependency(id, node_id, "UnionDependency") {} - ~UnionDependency() override = default; - - void* shared_state() override { return (void*)_union_state.get(); } - void set_shared_state(std::shared_ptr union_state) { - _union_state = union_state; - } - - [[nodiscard]] Dependency* read_blocked_by(PipelineXTask* task) override { - if (_union_state->child_count() == 0) { - return nullptr; - } - if (_union_state->data_queue.is_all_finish() || - _union_state->data_queue.remaining_has_data()) { - return nullptr; - } - return this; - } - bool push_to_blocking_queue() override { return true; } - void block_reading() override {} - void block_writing() override {} - -private: - std::shared_ptr _union_state; -}; - -struct MultiCastSharedState { +struct MultiCastSharedState : public BasicSharedState { public: MultiCastSharedState(const RowDescriptor& row_desc, ObjectPool* pool, int cast_sender_count) : multi_cast_data_streamer(row_desc, pool, cast_sender_count, true) {} pipeline::MultiCastDataStreamer multi_cast_data_streamer; }; -class MultiCastDependency final : public WriteDependency { -public: - using SharedState = MultiCastSharedState; - MultiCastDependency(int id, int node_id) - : WriteDependency(id, node_id, "MultiCastDependency") {} - ~MultiCastDependency() override = default; - void* shared_state() override { return (void*)_multi_cast_state.get(); }; - void set_shared_state(std::shared_ptr multi_cast_state) { - _multi_cast_state = multi_cast_state; - } - -private: - std::shared_ptr _multi_cast_state; -}; - -struct AnalyticSharedState { +struct AnalyticSharedState : public BasicSharedState { public: AnalyticSharedState() = default; @@ -514,38 +323,7 @@ public: std::vector ordey_by_column_idxs; }; -class AnalyticDependency final : public WriteDependency { -public: - using SharedState = AnalyticSharedState; - AnalyticDependency(int id, int node_id) : WriteDependency(id, node_id, "AnalyticDependency") {} - ~AnalyticDependency() override = default; - - void* shared_state() override { return (void*)&_analytic_state; }; - - vectorized::BlockRowPos get_partition_by_end(); - - bool refresh_need_more_input() { - auto need_more_input = whether_need_next_partition(_analytic_state.found_partition_end); - if (need_more_input) { - block_reading(); - set_ready_for_write(); - } else { - block_writing(); - set_ready_for_read(); - } - return need_more_input; - } - - bool whether_need_next_partition(vectorized::BlockRowPos& found_partition_end); - vectorized::BlockRowPos compare_row_to_find_end(int idx, vectorized::BlockRowPos start, - vectorized::BlockRowPos end, - bool need_check_first = false); - -private: - AnalyticSharedState _analytic_state; -}; - -struct JoinSharedState { +struct JoinSharedState : public BasicSharedState { // For some join case, we can apply a short circuit strategy // 1. _has_null_in_build_side = true // 2. build side rows is empty, Join op is: inner join/right outer join/left semi/right semi/right anti @@ -572,29 +350,6 @@ struct HashJoinSharedState : public JoinSharedState { bool probe_ignore_null = false; }; -class HashJoinDependency final : public WriteDependency { -public: - using SharedState = HashJoinSharedState; - HashJoinDependency(int id, int node_id) : WriteDependency(id, node_id, "HashJoinDependency") {} - ~HashJoinDependency() override = default; - - void* shared_state() override { return (void*)&_join_state; } - - Status do_evaluate(vectorized::Block& block, vectorized::VExprContextSPtrs& exprs, - RuntimeProfile::Counter& expr_call_timer, std::vector& res_col_ids); - - std::vector convert_block_to_null(vectorized::Block& block); - - template - Status extract_join_column(vectorized::Block& block, - vectorized::ColumnUInt8::MutablePtr& null_map, - vectorized::ColumnRawPtrs& raw_ptrs, - const std::vector& res_col_ids); - -private: - HashJoinSharedState _join_state; -}; - struct NestedLoopJoinSharedState : public JoinSharedState { // if true, left child has no more rows to process bool left_side_eos = false; @@ -604,20 +359,7 @@ struct NestedLoopJoinSharedState : public JoinSharedState { vectorized::Blocks build_blocks; }; -class NestedLoopJoinDependency final : public WriteDependency { -public: - using SharedState = NestedLoopJoinSharedState; - NestedLoopJoinDependency(int id, int node_id) - : WriteDependency(id, node_id, "NestedLoopJoinDependency") {} - ~NestedLoopJoinDependency() override = default; - - void* shared_state() override { return (void*)&_join_state; } - -private: - NestedLoopJoinSharedState _join_state; -}; - -struct PartitionSortNodeSharedState { +struct PartitionSortNodeSharedState : public BasicSharedState { public: std::queue blocks_buffer; std::mutex buffer_mutex; @@ -625,51 +367,16 @@ public: std::unique_ptr previous_row = nullptr; }; -class PartitionSortDependency final : public WriteDependency { -public: - using SharedState = PartitionSortNodeSharedState; - PartitionSortDependency(int id, int node_id) - : WriteDependency(id, node_id, "PartitionSortDependency"), _eos(false) {} - ~PartitionSortDependency() override = default; - void* shared_state() override { return (void*)&_partition_sort_state; }; - void set_ready_for_write() override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Should not reach here!"); - } - void block_writing() override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Should not reach here!"); - } - - void block_reading() override { - if (_eos) { - return; - } - Dependency::block_reading(); - } - - void set_eos() { - if (_eos) { - return; - } - _eos = true; - WriteDependency::set_ready_for_read(); - } - -private: - PartitionSortNodeSharedState _partition_sort_state; - std::atomic _eos; -}; - -class AsyncWriterDependency final : public WriteDependency { +class AsyncWriterDependency final : public Dependency { public: + using SharedState = FakeSharedState; ENABLE_FACTORY_CREATOR(AsyncWriterDependency); AsyncWriterDependency(int id, int node_id) - : WriteDependency(id, node_id, "AsyncWriterDependency") {} + : Dependency(id, node_id, "AsyncWriterDependency", true) {} ~AsyncWriterDependency() override = default; }; -class SetDependency; - -struct SetSharedState { +struct SetSharedState : public BasicSharedState { public: SetSharedState(int num_deps) { probe_finished_children_dependency.resize(num_deps, nullptr); } /// default init @@ -696,15 +403,13 @@ public: /// init in build side int child_quantity; vectorized::VExprContextSPtrs build_child_exprs; - std::vector probe_finished_children_dependency; + std::vector probe_finished_children_dependency; /// init in probe side std::vector probe_child_exprs_lists; std::atomic ready_for_read = false; - void set_probe_finished_children(int child_id); - /// called in setup_local_state void hash_table_init() { if (child_exprs_lists[0].size() == 1 && (!build_not_ignore_null[0])) { @@ -757,33 +462,9 @@ public: } }; -class SetDependency final : public WriteDependency { -public: - using SharedState = SetSharedState; - SetDependency(int id, int node_id) : WriteDependency(id, node_id, "SetDependency") {} - ~SetDependency() override = default; - void* shared_state() override { return (void*)_set_state.get(); } - - void set_shared_state(std::shared_ptr set_state) { _set_state = set_state; } - - void set_ready_for_read() override; - - void set_cur_child_id(int id) { - _child_idx = id; - _set_state->probe_finished_children_dependency[id] = this; - if (id != 0) { - block_writing(); - } - } - -private: - std::shared_ptr _set_state; - int _child_idx {0}; -}; - using PartitionedBlock = std::pair, std::tuple>, size_t, size_t>>; -struct LocalExchangeSharedState { +struct LocalExchangeSharedState : public BasicSharedState { public: ENABLE_FACTORY_CREATOR(LocalExchangeSharedState); std::vector> data_queue; @@ -799,35 +480,18 @@ public: void _set_ready_for_read() { for (auto* dep : source_dependencies) { DCHECK(dep); - dep->set_ready_for_read(); + dep->set_ready(); } } void set_dep_by_channel_id(Dependency* dep, int channel_id) { source_dependencies[channel_id] = dep; - dep->block_reading(); + dep->block(); } void set_ready_for_read(int channel_id) { auto* dep = source_dependencies[channel_id]; DCHECK(dep); - dep->set_ready_for_read(); + dep->set_ready(); } }; -struct LocalExchangeDependency final : public WriteDependency { -public: - using SharedState = LocalExchangeSharedState; - LocalExchangeDependency(int id, int node_id) - : WriteDependency(id, node_id, "LocalExchangeDependency"), - _local_exchange_shared_state(nullptr) {} - ~LocalExchangeDependency() override = default; - void* shared_state() override { return _local_exchange_shared_state.get(); } - void set_shared_state(std::shared_ptr state) { - DCHECK(_local_exchange_shared_state == nullptr); - _local_exchange_shared_state = state; - } - -private: - std::shared_ptr _local_exchange_shared_state; -}; - } // namespace doris::pipeline 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 56d3d77460..c6c28fbe8d 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 @@ -22,10 +22,19 @@ namespace doris::pipeline { -class LocalExchangeSinkOperatorX; -class LocalExchangeSinkLocalState final : public PipelineXSinkLocalState { +struct LocalExchangeSinkDependency final : public Dependency { public: - using Base = PipelineXSinkLocalState; + using SharedState = LocalExchangeSharedState; + LocalExchangeSinkDependency(int id, int node_id) + : Dependency(id, node_id, "LocalExchangeSinkDependency", true) {} + ~LocalExchangeSinkDependency() override = default; +}; + +class LocalExchangeSinkOperatorX; +class LocalExchangeSinkLocalState final + : public PipelineXSinkLocalState { +public: + using Base = PipelineXSinkLocalState; ENABLE_FACTORY_CREATOR(LocalExchangeSinkLocalState); LocalExchangeSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.cpp b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.cpp index a1bff19cb2..81c9cc2144 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.cpp +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.cpp @@ -24,7 +24,7 @@ Status LocalExchangeSourceLocalState::init(RuntimeState* state, LocalStateInfo& SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); _dependency->set_shared_state(info.local_exchange_state); - _shared_state = (LocalExchangeSharedState*)_dependency->shared_state(); + _shared_state = (LocalExchangeSharedState*)_dependency->shared_state().get(); DCHECK(_shared_state != nullptr); _channel_id = info.task_idx; _shared_state->set_dep_by_channel_id(_dependency, _channel_id); diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h index b992f45469..dfc89a86c9 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h @@ -22,10 +22,19 @@ namespace doris::pipeline { -class LocalExchangeSourceOperatorX; -class LocalExchangeSourceLocalState final : public PipelineXLocalState { +struct LocalExchangeSourceDependency final : public Dependency { public: - using Base = PipelineXLocalState; + using SharedState = LocalExchangeSharedState; + LocalExchangeSourceDependency(int id, int node_id) + : Dependency(id, node_id, "LocalExchangeSourceDependency") {} + ~LocalExchangeSourceDependency() override = default; +}; + +class LocalExchangeSourceOperatorX; +class LocalExchangeSourceLocalState final + : public PipelineXLocalState { +public: + using Base = PipelineXLocalState; ENABLE_FACTORY_CREATOR(LocalExchangeSourceLocalState); LocalExchangeSourceLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent) {} diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index dc5696fab9..4e3fec7abb 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -282,19 +282,40 @@ Status DataSinkOperatorX::setup_local_state(RuntimeState* state, return Status::OK(); } +template +constexpr bool NeedToCreate = true; +template <> +inline constexpr bool NeedToCreate = false; +template <> +inline constexpr bool NeedToCreate = false; +template <> +inline constexpr bool NeedToCreate = false; +template <> +inline constexpr bool NeedToCreate = false; + template void DataSinkOperatorX::get_dependency(vector& dependency) { - using DependencyType = typename LocalStateType::Dependency; - if constexpr (!std::is_same_v) { + std::shared_ptr ss = nullptr; + if constexpr (NeedToCreate) { + ss.reset(new typename LocalStateType::DependencyType::SharedState()); + } + if constexpr (!std::is_same_v) { auto& dests = dests_id(); for (auto& dest_id : dests) { - dependency.push_back(std::make_shared(dest_id, _node_id)); + dependency.push_back( + std::make_shared(dest_id, _node_id)); + dependency.back()->set_shared_state(ss); } } else { dependency.push_back(nullptr); } } +template +DependencySPtr OperatorX::get_dependency() { + return std::make_shared(_operator_id, _node_id); +} + template Status OperatorX::setup_local_state(RuntimeState* state, LocalStateInfo& info) { auto local_state = LocalStateType::create_unique(state, this); @@ -307,8 +328,8 @@ PipelineXSinkLocalStateBase::PipelineXSinkLocalStateBase(DataSinkOperatorXBase* RuntimeState* state) : _parent(parent), _state(state), - _finish_dependency(new FinishDependency(parent->operator_id(), parent->node_id(), - parent->get_name() + "_FINISH_DEPENDENCY")) {} + _finish_dependency(new Dependency(parent->operator_id(), parent->node_id(), + parent->get_name() + "_FINISH_DEPENDENCY", true)) {} PipelineXLocalStateBase::PipelineXLocalStateBase(RuntimeState* state, OperatorXBase* parent) : _num_rows_returned(0), @@ -316,8 +337,8 @@ PipelineXLocalStateBase::PipelineXLocalStateBase(RuntimeState* state, OperatorXB _peak_memory_usage_counter(nullptr), _parent(parent), _state(state), - _finish_dependency(new FinishDependency(parent->operator_id(), parent->node_id(), - parent->get_name() + "_FINISH_DEPENDENCY")) { + _finish_dependency(new Dependency(parent->operator_id(), parent->node_id(), + parent->get_name() + "_FINISH_DEPENDENCY", true)) { _filter_dependency = std::make_shared( parent->operator_id(), parent->node_id(), parent->get_name() + "_FILTER_DEPENDENCY"); } @@ -331,18 +352,15 @@ Status PipelineXLocalState::init(RuntimeState* state, LocalState info.parent_profile->add_child(_runtime_profile.get(), true, nullptr); _wait_for_finish_dependency_timer = ADD_TIMER(_runtime_profile, "WaitForPendingFinishDependency"); + _dependency = (DependencyType*)info.dependency.get(); if constexpr (!std::is_same_v) { - auto& deps = info.dependencys; - _dependency = (DependencyType*)deps.front().get(); - if (_dependency) { - _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state(); - _wait_for_dependency_timer = ADD_TIMER( - _runtime_profile, "WaitForDependency[" + _dependency->name() + "]Time"); - } - } else { - auto& deps = info.dependencys; - deps.front() = std::make_shared(0, 0); - _dependency = (DependencyType*)deps.front().get(); + auto& deps = info.upstream_dependencies; + _dependency->set_shared_state(deps.front()->shared_state()); + _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state().get(); + _wait_for_dependency_timer = + ADD_TIMER(_runtime_profile, "WaitForDependency[" + _dependency->name() + "]Time"); + _shared_state->source_dep = _dependency; + _shared_state->sink_dep = deps.front().get(); } _conjuncts.resize(_parent->_conjuncts.size()); @@ -374,10 +392,9 @@ Status PipelineXLocalState::close(RuntimeState* state) { return Status::OK(); } if constexpr (!std::is_same_v) { - COUNTER_SET(_wait_for_dependency_timer, _dependency->read_watcher_elapse_time()); + COUNTER_SET(_wait_for_dependency_timer, _dependency->watcher_elapse_time()); } - COUNTER_SET(_wait_for_finish_dependency_timer, - _finish_dependency->finish_watcher_elapse_time()); + COUNTER_SET(_wait_for_finish_dependency_timer, _finish_dependency->watcher_elapse_time()); if (_rows_returned_counter != nullptr) { COUNTER_SET(_rows_returned_counter, _num_rows_returned); } @@ -397,7 +414,8 @@ Status PipelineXSinkLocalState::init(RuntimeState* state, auto& deps = info.dependencys; _dependency = (DependencyType*)deps.front().get(); if (_dependency) { - _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state(); + _shared_state = + (typename DependencyType::SharedState*)_dependency->shared_state().get(); _wait_for_dependency_timer = ADD_TIMER(_profile, "WaitForDependency[" + _dependency->name() + "]Time"); } @@ -421,10 +439,9 @@ Status PipelineXSinkLocalState::close(RuntimeState* state, Statu return Status::OK(); } if constexpr (!std::is_same_v) { - COUNTER_SET(_wait_for_dependency_timer, _dependency->write_watcher_elapse_time()); + COUNTER_SET(_wait_for_dependency_timer, _dependency->watcher_elapse_time()); } - COUNTER_SET(_wait_for_finish_dependency_timer, - _finish_dependency->finish_watcher_elapse_time()); + COUNTER_SET(_wait_for_finish_dependency_timer, _finish_dependency->watcher_elapse_time()); _closed = true; return Status::OK(); } @@ -488,7 +505,7 @@ Status AsyncWriterSink::init(RuntimeState* state, LocalSinkState _wait_for_dependency_timer = ADD_TIMER(_profile, "WaitForDependency[" + _async_writer_dependency->name() + "]Time"); - _finish_dependency->should_finish_after_check(); + _finish_dependency->block(); return Status::OK(); } @@ -506,7 +523,7 @@ Status AsyncWriterSink::sink(RuntimeState* state, vectorized::Bl } template -WriteDependency* AsyncWriterSink::write_blocked_by(PipelineXTask* task) { +Dependency* AsyncWriterSink::write_blocked_by(PipelineXTask* task) { return _writer->write_blocked_by(task); } @@ -515,7 +532,7 @@ Status AsyncWriterSink::close(RuntimeState* state, Status exec_s if (_closed) { return Status::OK(); } - COUNTER_SET(_wait_for_dependency_timer, _async_writer_dependency->write_watcher_elapse_time()); + COUNTER_SET(_wait_for_dependency_timer, _async_writer_dependency->watcher_elapse_time()); // if the init failed, the _writer may be nullptr. so here need check if (_writer) { if (_writer->need_normal_close()) { @@ -598,29 +615,30 @@ template class StatefulOperatorX; template class StatefulOperatorX; template class StatefulOperatorX; -template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; -template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; -template class PipelineXLocalState; -template class PipelineXLocalState; -template class PipelineXLocalState; -template class PipelineXLocalState; -template class PipelineXLocalState; +template class PipelineXLocalState; +template class PipelineXLocalState; +template class PipelineXLocalState; +template class PipelineXLocalState; +template class PipelineXLocalState; template class PipelineXLocalState; -template class PipelineXLocalState; -template class PipelineXLocalState; -template class PipelineXLocalState; -template class PipelineXLocalState; -template class PipelineXLocalState; +template class PipelineXLocalState; +template class PipelineXLocalState; +template class PipelineXLocalState; +template class PipelineXLocalState; +template class PipelineXLocalState; template class AsyncWriterSink; template class AsyncWriterSink; diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index 4e8f030d50..5afa080e6d 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -35,9 +35,11 @@ namespace doris::pipeline { struct LocalStateInfo { RuntimeProfile* parent_profile; const std::vector scan_ranges; - std::vector& dependencys; + std::vector& upstream_dependencies; std::shared_ptr local_exchange_state; int task_idx; + + DependencySPtr dependency; }; // This struct is used only for initializing local sink state. @@ -99,7 +101,7 @@ public: virtual Dependency* dependency() { return nullptr; } - FinishDependency* finishdependency() { return _finish_dependency.get(); } + Dependency* finishdependency() { return _finish_dependency.get(); } RuntimeFilterDependency* filterdependency() { return _filter_dependency.get(); } protected: @@ -132,7 +134,7 @@ protected: vectorized::VExprContextSPtrs _projections; bool _closed = false; vectorized::Block _origin_block; - std::shared_ptr _finish_dependency; + std::shared_ptr _finish_dependency; std::shared_ptr _filter_dependency; }; @@ -174,6 +176,7 @@ public: throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, _op_name); } [[nodiscard]] std::string get_name() const override { return _op_name; } + virtual DependencySPtr get_dependency() = 0; Status prepare(RuntimeState* state) override; @@ -303,11 +306,14 @@ public: [[nodiscard]] LocalState& get_local_state(RuntimeState* state) const { return state->get_local_state(operator_id())->template cast(); } + + DependencySPtr get_dependency() override; }; -template +template class PipelineXLocalState : public PipelineXLocalStateBase { public: + using DependencyType = DependencyArg; PipelineXLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalStateBase(state, parent) {} ~PipelineXLocalState() override = default; @@ -370,9 +376,9 @@ public: RuntimeProfile::Counter* rows_input_counter() { return _rows_input_counter; } RuntimeProfile::Counter* exec_time_counter() { return _exec_timer; } - virtual WriteDependency* dependency() { return nullptr; } + virtual Dependency* dependency() { return nullptr; } - FinishDependency* finishdependency() { return _finish_dependency.get(); } + Dependency* finishdependency() { return _finish_dependency.get(); } protected: DataSinkOperatorXBase* _parent; @@ -397,7 +403,7 @@ protected: RuntimeProfile::Counter* _wait_for_dependency_timer; RuntimeProfile::Counter* _wait_for_finish_dependency_timer; RuntimeProfile::Counter* _exec_timer; - std::shared_ptr _finish_dependency; + std::shared_ptr _finish_dependency; }; class DataSinkOperatorXBase : public OperatorBase { @@ -553,10 +559,10 @@ public: } }; -template +template class PipelineXSinkLocalState : public PipelineXSinkLocalStateBase { public: - using Dependency = DependencyType; + using DependencyType = DependencyArg; PipelineXSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : PipelineXSinkLocalStateBase(parent, state) {} ~PipelineXSinkLocalState() override = default; @@ -573,11 +579,11 @@ public: virtual std::string id_name() { return " (id=" + std::to_string(_parent->node_id()) + ")"; } - WriteDependency* dependency() override { return _dependency; } + Dependency* dependency() override { return _dependency; } protected: DependencyType* _dependency = nullptr; - typename DependencyType::SharedState* _shared_state; + typename DependencyType::SharedState* _shared_state = nullptr; }; /** @@ -637,8 +643,8 @@ public: Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state); - WriteDependency* write_blocked_by(PipelineXTask* task); - WriteDependency* dependency() override { return _async_writer_dependency.get(); } + Dependency* write_blocked_by(PipelineXTask* task); + Dependency* dependency() override { return _async_writer_dependency.get(); } Status close(RuntimeState* state, Status exec_status) override; Status try_close(RuntimeState* state, Status exec_status) override; diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp index ade718c149..5b3524c69a 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp @@ -58,6 +58,9 @@ PipelineXTask::PipelineXTask(PipelinePtr& pipeline, uint32_t task_id, RuntimeSta _task_idx(task_idx) { _pipeline_task_watcher.start(); _sink->get_dependency(_downstream_dependency); + for (auto& op : _operators) { + _source_dependency.insert({op->operator_id(), op->get_dependency()}); + } } Status PipelineXTask::prepare(RuntimeState* state, const TPipelineInstanceParams& local_params, @@ -86,7 +89,11 @@ Status PipelineXTask::prepare(RuntimeState* state, const TPipelineInstanceParams op_idx == _operators.size() - 1 ? _parent_profile : state->get_local_state(_operators[op_idx + 1]->operator_id())->profile(), - scan_ranges, deps, _local_exchange_state, _task_idx}; + scan_ranges, + deps, + _local_exchange_state, + _task_idx, + _source_dependency[_operators[op_idx]->operator_id()]}; RETURN_IF_ERROR(_operators[op_idx]->setup_local_state(state, info)); } @@ -183,7 +190,7 @@ Status PipelineXTask::_open() { for (size_t i = 0; i < 2; i++) { auto st = local_state->open(_state); if (st.is()) { - _blocked_dep = _filter_dependency->filter_blocked_by(this); + _blocked_dep = _filter_dependency->is_blocked_by(this); if (_blocked_dep) { set_state(PipelineTaskState::BLOCKED_FOR_RF); set_use_blocking_queue(); diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.h b/be/src/pipeline/pipeline_x/pipeline_x_task.h index f920bf219b..e0d0e58e65 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.h @@ -145,10 +145,10 @@ public: private: Dependency* _write_blocked_dependency() { - _blocked_dep = _write_dependencies->write_blocked_by(this); + _blocked_dep = _write_dependencies->is_blocked_by(this); if (_blocked_dep != nullptr) { set_use_blocking_queue(); - static_cast(_blocked_dep)->start_write_watcher(); + static_cast(_blocked_dep)->start_watcher(); return _blocked_dep; } return nullptr; @@ -156,10 +156,10 @@ private: Dependency* _finish_blocked_dependency() { for (auto* fin_dep : _finish_dependencies) { - _blocked_dep = fin_dep->finish_blocked_by(this); + _blocked_dep = fin_dep->is_blocked_by(this); if (_blocked_dep != nullptr) { set_use_blocking_queue(); - static_cast(_blocked_dep)->start_finish_watcher(); + _blocked_dep->start_watcher(); return _blocked_dep; } } @@ -168,10 +168,10 @@ private: Dependency* _read_blocked_dependency() { for (auto* op_dep : _read_dependencies) { - _blocked_dep = op_dep->read_blocked_by(this); + _blocked_dep = op_dep->is_blocked_by(this); if (_blocked_dep != nullptr) { set_use_blocking_queue(); - _blocked_dep->start_read_watcher(); + _blocked_dep->start_watcher(); return _blocked_dep; } } @@ -183,7 +183,6 @@ private: void set_close_pipeline_time() override {} void _init_profile() override; void _fresh_profile_counter() override; - using DependencyMap = std::map>; Status _open() override; OperatorXs _operators; // left is _source, right is _root @@ -192,12 +191,12 @@ private: DataSinkOperatorXPtr _sink; std::vector _read_dependencies; - WriteDependency* _write_dependencies; - std::vector _finish_dependencies; + Dependency* _write_dependencies; + std::vector _finish_dependencies; RuntimeFilterDependency* _filter_dependency; DependencyMap _upstream_dependency; - + std::map _source_dependency; std::vector _downstream_dependency; std::shared_ptr _local_exchange_state; int _task_idx; diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp index 867f34a012..91cb032c76 100644 --- a/be/src/runtime/buffer_control_block.cpp +++ b/be/src/runtime/buffer_control_block.cpp @@ -305,10 +305,10 @@ void PipBufferControlBlock::set_dependency( void PipBufferControlBlock::_update_dependency() { if (_result_sink_dependency && (_batch_queue_empty || _buffer_rows < _buffer_limit || _is_cancelled)) { - _result_sink_dependency->set_ready_for_write(); + _result_sink_dependency->set_ready(); } else if (_result_sink_dependency && (!_batch_queue_empty && _buffer_rows < _buffer_limit && !_is_cancelled)) { - _result_sink_dependency->block_writing(); + _result_sink_dependency->block(); } } diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h index fe00a9489a..111e6ea2ab 100644 --- a/be/src/vec/exec/scan/pip_scanner_context.h +++ b/be/src/vec/exec/scan/pip_scanner_context.h @@ -77,7 +77,7 @@ public: _blocks_queues[id].pop_front(); if (_blocks_queues[id].empty() && _dependency) { - _dependency->block_reading(); + _dependency->block(); } } _current_used_bytes -= (*block)->allocated_bytes(); @@ -150,7 +150,7 @@ public: _blocks_queues[queue].emplace_back(std::move(blocks[j])); } if (_dependency) { - _dependency->set_ready_for_read(); + _dependency->set_ready(); } } _next_queue_to_feed = queue + 1 < queue_size ? queue + 1 : 0; @@ -202,7 +202,7 @@ public: _colocate_mutable_blocks[i]->clear(); } if (_dependency) { - _dependency->set_ready_for_read(); + _dependency->set_ready(); } } } @@ -256,7 +256,7 @@ private: _blocks_queues[loc].emplace_back(std::move(_colocate_blocks[loc])); } if (_dependency) { - _dependency->set_ready_for_read(); + _dependency->set_ready(); } _colocate_blocks[loc] = get_free_block(); _colocate_mutable_blocks[loc]->set_muatable_columns( diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index b2c481871a..0d9b0351dd 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -323,7 +323,7 @@ void ScannerContext::set_ready_to_finish() { // `_should_stop == true` means this task has already ended and wait for pending finish now. if (_finish_dependency && _should_stop && _num_running_scanners == 0 && _num_scheduling_ctx == 0) { - _finish_dependency->set_ready_to_finish(); + _finish_dependency->set_ready(); } } diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index a0702960ac..70df1c2c8b 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -44,7 +44,7 @@ class TupleDescriptor; namespace pipeline { class ScanLocalStateBase; class ScanDependency; -class FinishDependency; +class Dependency; } // namespace pipeline namespace taskgroup { @@ -106,7 +106,7 @@ public: } void set_dependency(std::shared_ptr dependency, - std::shared_ptr finish_dependency) { + std::shared_ptr finish_dependency) { _dependency = dependency; _finish_dependency = finish_dependency; } @@ -284,7 +284,7 @@ protected: RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; std::shared_ptr _dependency = nullptr; - std::shared_ptr _finish_dependency = nullptr; + std::shared_ptr _finish_dependency = nullptr; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/runtime/shared_hash_table_controller.cpp b/be/src/vec/runtime/shared_hash_table_controller.cpp index 524af272c1..0dc50f2085 100644 --- a/be/src/vec/runtime/shared_hash_table_controller.cpp +++ b/be/src/vec/runtime/shared_hash_table_controller.cpp @@ -72,7 +72,7 @@ void SharedHashTableController::signal(int my_node_id, Status status) { _shared_contexts.erase(it); } for (auto& dep : _dependencies[my_node_id]) { - dep->set_ready_for_write(); + dep->set_ready(); } _cv.notify_all(); } @@ -85,7 +85,7 @@ void SharedHashTableController::signal(int my_node_id) { _shared_contexts.erase(it); } for (auto& dep : _dependencies[my_node_id]) { - dep->set_ready_for_write(); + dep->set_ready(); } _cv.notify_all(); } diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index 891161f071..94bc7a7bde 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -101,10 +101,10 @@ Status VDataStreamRecvr::SenderQueue::_inner_get_batch_without_lock(Block* block _block_queue.pop_front(); if (_block_queue.size() == 0 && _dependency) { if (!_is_cancelled && _num_remaining_senders > 0) { - _dependency->block_reading(); + _dependency->block(); } if (_local_channel_dependency) { - _local_channel_dependency->set_ready_for_write(); + _local_channel_dependency->set_ready(); } } @@ -178,7 +178,7 @@ Status VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_num if (!empty) { _block_queue.emplace_back(std::move(block), block_byte_size); if (_dependency) { - _dependency->set_ready_for_read(); + _dependency->set_ready(); } } // if done is nullptr, this function can't delay this response @@ -234,7 +234,7 @@ void VDataStreamRecvr::SenderQueue::add_block(Block* block, bool use_move) { if (!empty) { _block_queue.emplace_back(std::move(nblock), block_mem_size); if (_dependency) { - _dependency->set_ready_for_read(); + _dependency->set_ready(); } _data_arrival_cv.notify_one(); } @@ -273,7 +273,7 @@ void VDataStreamRecvr::SenderQueue::decrement_senders(int be_number) { << " #senders=" << _num_remaining_senders; if (_num_remaining_senders == 0) { if (_dependency) { - _dependency->set_always_done(); + _dependency->set_eos(); } _data_arrival_cv.notify_one(); } @@ -288,7 +288,7 @@ void VDataStreamRecvr::SenderQueue::cancel(Status cancel_status) { _is_cancelled = true; _cancel_status = cancel_status; if (_dependency) { - _dependency->set_always_done(); + _dependency->set_eos(); } VLOG_QUERY << "cancelled stream: _fragment_instance_id=" << print_id(_recvr->fragment_instance_id()) @@ -318,7 +318,7 @@ void VDataStreamRecvr::SenderQueue::close() { std::lock_guard l(_lock); _is_cancelled = true; if (_dependency) { - _dependency->set_always_done(); + _dependency->set_eos(); } for (auto closure_pair : _pending_closures) { @@ -511,7 +511,7 @@ void VDataStreamRecvr::close() { } _is_closed = true; for (auto& it : _sender_to_local_channel_dependency) { - it->set_ready_for_write(); + it->set_ready(); } for (int i = 0; i < _sender_queues.size(); ++i) { _sender_queues[i]->close(); @@ -559,7 +559,7 @@ void VDataStreamRecvr::PipSenderQueue::add_block(Block* block, bool use_move) { } _block_queue.emplace_back(std::move(nblock), block_mem_size); if (_dependency) { - _dependency->set_ready_for_read(); + _dependency->set_ready(); } COUNTER_UPDATE(_recvr->_local_bytes_received_counter, block_mem_size); _recvr->update_blocks_memory_usage(block_mem_size); diff --git a/be/src/vec/sink/writer/async_result_writer.cpp b/be/src/vec/sink/writer/async_result_writer.cpp index 4ae7c3d364..d1fc46dc8d 100644 --- a/be/src/vec/sink/writer/async_result_writer.cpp +++ b/be/src/vec/sink/writer/async_result_writer.cpp @@ -38,7 +38,7 @@ AsyncResultWriter::AsyncResultWriter(const doris::vectorized::VExprContextSPtrs& _finish_dependency(nullptr) {} void AsyncResultWriter::set_dependency(pipeline::AsyncWriterDependency* dep, - pipeline::FinishDependency* finish_dep) { + pipeline::Dependency* finish_dep) { _dependency = dep; _finish_dependency = finish_dep; } @@ -60,12 +60,12 @@ Status AsyncResultWriter::sink(Block* block, bool eos) { std::lock_guard l(_m); _eos = eos; if (_dependency && _is_finished()) { - _dependency->set_ready_for_write(); + _dependency->set_ready(); } if (rows) { _data_queue.emplace_back(std::move(add_block)); if (_dependency && !_data_queue_is_available() && !_is_finished()) { - _dependency->block_writing(); + _dependency->block(); } } else if (_eos && _data_queue.empty()) { status = Status::EndOfFile("Run out of sink data"); @@ -81,7 +81,7 @@ std::unique_ptr AsyncResultWriter::_get_block_from_queue() { auto block = std::move(_data_queue.front()); _data_queue.pop_front(); if (_dependency && _data_queue_is_available()) { - _dependency->set_ready_for_write(); + _dependency->set_ready(); } return block; } @@ -124,7 +124,7 @@ void AsyncResultWriter::process_block(RuntimeState* state, RuntimeProfile* profi std::unique_lock l(_m); _writer_status = status; if (_dependency && _is_finished()) { - _dependency->set_ready_for_write(); + _dependency->set_ready(); } break; } @@ -141,7 +141,7 @@ void AsyncResultWriter::process_block(RuntimeState* state, RuntimeProfile* profi } _writer_thread_closed = true; if (_finish_dependency) { - _finish_dependency->set_ready_to_finish(); + _finish_dependency->set_ready(); } } @@ -161,7 +161,7 @@ void AsyncResultWriter::force_close(Status s) { std::lock_guard l(_m); _writer_status = s; if (_dependency && _is_finished()) { - _dependency->set_ready_for_write(); + _dependency->set_ready(); } _cv.notify_one(); } @@ -180,10 +180,10 @@ std::unique_ptr AsyncResultWriter::_get_free_block(doris::vectorized::Blo return b; } -pipeline::WriteDependency* AsyncResultWriter::write_blocked_by(pipeline::PipelineXTask* task) { +pipeline::Dependency* AsyncResultWriter::write_blocked_by(pipeline::PipelineXTask* task) { std::lock_guard l(_m); DCHECK(_dependency != nullptr); - return _dependency->write_blocked_by(task); + return _dependency->is_blocked_by(task); } } // namespace vectorized diff --git a/be/src/vec/sink/writer/async_result_writer.h b/be/src/vec/sink/writer/async_result_writer.h index 75cc6529ba..a50b9296c7 100644 --- a/be/src/vec/sink/writer/async_result_writer.h +++ b/be/src/vec/sink/writer/async_result_writer.h @@ -34,8 +34,7 @@ class TExpr; namespace pipeline { class AsyncWriterDependency; -class WriteDependency; -class FinishDependency; +class Dependency; class PipelineXTask; } // namespace pipeline @@ -58,8 +57,7 @@ class AsyncResultWriter : public ResultWriter { public: AsyncResultWriter(const VExprContextSPtrs& output_expr_ctxs); - void set_dependency(pipeline::AsyncWriterDependency* dep, - pipeline::FinishDependency* finish_dep); + void set_dependency(pipeline::AsyncWriterDependency* dep, pipeline::Dependency* finish_dep); void force_close(Status s); @@ -80,7 +78,7 @@ public: return _data_queue_is_available() || _is_finished(); } - pipeline::WriteDependency* write_blocked_by(pipeline::PipelineXTask* task); + pipeline::Dependency* write_blocked_by(pipeline::PipelineXTask* task); [[nodiscard]] bool is_pending_finish() const { return !_writer_thread_closed; } @@ -120,7 +118,7 @@ private: // Used by pipelineX pipeline::AsyncWriterDependency* _dependency; - pipeline::FinishDependency* _finish_dependency; + pipeline::Dependency* _finish_dependency; moodycamel::ConcurrentQueue> _free_blocks; };