diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index 0c94b3edf0..b30bb9b815 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -46,9 +46,10 @@ OPERATOR_CODE_GENERATOR(AggSinkOperator, StreamingOperator) /// using the planner's estimated input cardinality and the assumption that input /// is in a random order. This means that we assume that the reduction factor will /// increase over time. -template -AggSinkLocalState::AggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : PipelineXSinkLocalState(parent, state), +template +AggSinkLocalState::AggSinkLocalState(DataSinkOperatorXBase* parent, + RuntimeState* state) + : Base(parent, state), _hash_table_compute_timer(nullptr), _hash_table_input_counter(nullptr), _build_timer(nullptr), @@ -60,66 +61,68 @@ AggSinkLocalState::AggSinkLocalState(DataSinkOperatorXBase* parent, Run _deserialize_data_timer(nullptr), _max_row_size_counter(nullptr) {} -template -Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); - _agg_data = _shared_state->agg_data.get(); - _agg_arena_pool = _shared_state->agg_arena_pool.get(); - auto& p = _parent->cast(); - _dependency->set_align_aggregate_states(p._align_aggregate_states); - _dependency->set_total_size_of_aggregate_states(p._total_size_of_aggregate_states); - _dependency->set_offsets_of_aggregate_states(p._offsets_of_aggregate_states); - _dependency->set_make_nullable_keys(p._make_nullable_keys); - _shared_state->init_spill_partition_helper(p._spill_partition_count_bits); +template +Status AggSinkLocalState::init(RuntimeState* state, + LocalSinkStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(Base::profile()->total_time_counter()); + _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->init_spill_partition_helper(p._spill_partition_count_bits); for (auto& evaluator : p._aggregate_evaluators) { - _shared_state->aggregate_evaluators.push_back(evaluator->clone(state, p._pool)); - _shared_state->aggregate_evaluators.back()->set_timer(_exec_timer, _merge_timer, - _expr_timer); + Base::_shared_state->aggregate_evaluators.push_back(evaluator->clone(state, p._pool)); + Base::_shared_state->aggregate_evaluators.back()->set_timer(_exec_timer, _merge_timer, + _expr_timer); } - _shared_state->probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); - for (size_t i = 0; i < _shared_state->probe_expr_ctxs.size(); i++) { - RETURN_IF_ERROR(p._probe_expr_ctxs[i]->clone(state, _shared_state->probe_expr_ctxs[i])); + Base::_shared_state->probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); + for (size_t i = 0; i < Base::_shared_state->probe_expr_ctxs.size(); i++) { + RETURN_IF_ERROR( + p._probe_expr_ctxs[i]->clone(state, Base::_shared_state->probe_expr_ctxs[i])); } - _memory_usage_counter = ADD_LABEL_COUNTER(profile(), "MemoryUsage"); + _memory_usage_counter = ADD_LABEL_COUNTER(Base::profile(), "MemoryUsage"); _hash_table_memory_usage = - ADD_CHILD_COUNTER(profile(), "HashTable", TUnit::BYTES, "MemoryUsage"); - _serialize_key_arena_memory_usage = - profile()->AddHighWaterMarkCounter("SerializeKeyArena", TUnit::BYTES, "MemoryUsage"); + ADD_CHILD_COUNTER(Base::profile(), "HashTable", TUnit::BYTES, "MemoryUsage"); + _serialize_key_arena_memory_usage = Base::profile()->AddHighWaterMarkCounter( + "SerializeKeyArena", TUnit::BYTES, "MemoryUsage"); - _build_timer = ADD_TIMER(profile(), "BuildTime"); - _build_table_convert_timer = ADD_TIMER(profile(), "BuildConvertToPartitionedTime"); - _serialize_key_timer = ADD_TIMER(profile(), "SerializeKeyTime"); - _exec_timer = ADD_TIMER(profile(), "ExecTime"); - _merge_timer = ADD_TIMER(profile(), "MergeTime"); - _expr_timer = ADD_TIMER(profile(), "ExprTime"); - _serialize_data_timer = ADD_TIMER(profile(), "SerializeDataTime"); - _deserialize_data_timer = ADD_TIMER(profile(), "DeserializeAndMergeTime"); - _hash_table_compute_timer = ADD_TIMER(profile(), "HashTableComputeTime"); - _hash_table_emplace_timer = ADD_TIMER(profile(), "HashTableEmplaceTime"); - _hash_table_input_counter = ADD_COUNTER(profile(), "HashTableInputCount", TUnit::UNIT); - _max_row_size_counter = ADD_COUNTER(profile(), "MaxRowSizeInBytes", TUnit::UNIT); + _build_timer = ADD_TIMER(Base::profile(), "BuildTime"); + _build_table_convert_timer = ADD_TIMER(Base::profile(), "BuildConvertToPartitionedTime"); + _serialize_key_timer = ADD_TIMER(Base::profile(), "SerializeKeyTime"); + _exec_timer = ADD_TIMER(Base::profile(), "ExecTime"); + _merge_timer = ADD_TIMER(Base::profile(), "MergeTime"); + _expr_timer = ADD_TIMER(Base::profile(), "ExprTime"); + _serialize_data_timer = ADD_TIMER(Base::profile(), "SerializeDataTime"); + _deserialize_data_timer = ADD_TIMER(Base::profile(), "DeserializeAndMergeTime"); + _hash_table_compute_timer = ADD_TIMER(Base::profile(), "HashTableComputeTime"); + _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); + _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); + _max_row_size_counter = ADD_COUNTER(Base::profile(), "MaxRowSizeInBytes", TUnit::UNIT); COUNTER_SET(_max_row_size_counter, (int64_t)0); - _shared_state->agg_profile_arena = std::make_unique(); + Base::_shared_state->agg_profile_arena = std::make_unique(); - if (_shared_state->probe_expr_ctxs.empty()) { + if (Base::_shared_state->probe_expr_ctxs.empty()) { _agg_data->init(vectorized::AggregatedDataVariants::Type::without_key); _agg_data->without_key = reinterpret_cast( - _shared_state->agg_profile_arena->alloc(p._total_size_of_aggregate_states)); + Base::_shared_state->agg_profile_arena->alloc(p._total_size_of_aggregate_states)); if (p._is_merge) { - _executor.execute = std::bind(&AggSinkLocalState::_merge_without_key, - this, std::placeholders::_1); + _executor.execute = + std::bind(&Derived::_merge_without_key, this, std::placeholders::_1); } else { - _executor.execute = std::bind(&AggSinkLocalState::_execute_without_key, - this, std::placeholders::_1); + _executor.execute = + std::bind(&Derived::_execute_without_key, this, std::placeholders::_1); } - _executor.update_memusage = - std::bind(&AggSinkLocalState::_update_memusage_without_key, this); + _executor.update_memusage = std::bind(&Derived::_update_memusage_without_key, this); } else { - _init_hash_method(_shared_state->probe_expr_ctxs); + _init_hash_method(Base::_shared_state->probe_expr_ctxs); std::visit( [&](auto&& agg_method) { @@ -127,7 +130,7 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& using KeyType = typename HashTableType::key_type; /// some aggregate functions (like AVG for decimal) have align issues. - _shared_state->aggregate_data_container.reset( + Base::_shared_state->aggregate_data_container.reset( new vectorized::AggregateDataContainer( sizeof(KeyType), ((p._total_size_of_aggregate_states + p._align_aggregate_states - 1) / @@ -136,17 +139,15 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& }, _agg_data->method_variant); if (p._is_merge) { - _executor.execute = - std::bind(&AggSinkLocalState::_merge_with_serialized_key, this, - std::placeholders::_1); + _executor.execute = std::bind(&Derived::_merge_with_serialized_key, this, + std::placeholders::_1); } else { - _executor.execute = - std::bind(&AggSinkLocalState::_execute_with_serialized_key, - this, std::placeholders::_1); + _executor.execute = std::bind(&Derived::_execute_with_serialized_key, this, + std::placeholders::_1); } - _executor.update_memusage = std::bind( - &AggSinkLocalState::_update_memusage_with_serialized_key, this); + _executor.update_memusage = + std::bind(&Derived::_update_memusage_with_serialized_key, this); _should_limit_output = p._limit != -1 && // has limit (!p._have_conjuncts) && // no having conjunct @@ -156,36 +157,38 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& return Status::OK(); } -template -Status AggSinkLocalState::open(RuntimeState* state) { - RETURN_IF_ERROR(PipelineXSinkLocalState::open(state)); - _agg_data = _shared_state->agg_data.get(); +template +Status AggSinkLocalState::open(RuntimeState* state) { + SCOPED_TIMER(Base::profile()->total_time_counter()); + RETURN_IF_ERROR(Base::open(state)); + _agg_data = Base::_shared_state->agg_data.get(); // move _create_agg_status to open not in during prepare, // because during prepare and open thread is not the same one, // this could cause unable to get JVM - if (_shared_state->probe_expr_ctxs.empty()) { + 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(_dependency->create_agg_status(_agg_data->without_key)); + RETURN_IF_CATCH_EXCEPTION(Base::_dependency->create_agg_status(_agg_data->without_key)); } return Status::OK(); } -template -Status AggSinkLocalState::_execute_without_key(vectorized::Block* block) { +template +Status AggSinkLocalState::_execute_without_key(vectorized::Block* block) { DCHECK(_agg_data->without_key != nullptr); SCOPED_TIMER(_build_timer); - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_single_add( + for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { + RETURN_IF_ERROR(Base::_shared_state->aggregate_evaluators[i]->execute_single_add( block, - _agg_data->without_key + - _parent->cast()._offsets_of_aggregate_states[i], + _agg_data->without_key + Base::_parent->template cast() + ._offsets_of_aggregate_states[i], _agg_arena_pool)); } return Status::OK(); } -template -Status AggSinkLocalState::_merge_with_serialized_key(vectorized::Block* block) { +template +Status AggSinkLocalState::_merge_with_serialized_key( + vectorized::Block* block) { if (_reach_limit) { return _merge_with_serialized_key_helper(block); } else { @@ -193,8 +196,8 @@ Status AggSinkLocalState::_merge_with_serialized_key(vectorized::Block* } } -template -size_t AggSinkLocalState::_memory_usage() const { +template +size_t AggSinkLocalState::_memory_usage() const { size_t usage = 0; std::visit( [&](auto&& agg_method) { @@ -211,42 +214,46 @@ size_t AggSinkLocalState::_memory_usage() const { usage += _agg_arena_pool->size(); } - if (_shared_state->aggregate_data_container) { - usage += _shared_state->aggregate_data_container->memory_usage(); + if (Base::_shared_state->aggregate_data_container) { + usage += Base::_shared_state->aggregate_data_container->memory_usage(); } return usage; } -template -void AggSinkLocalState::_update_memusage_with_serialized_key() { +template +void AggSinkLocalState::_update_memusage_with_serialized_key() { std::visit( [&](auto&& agg_method) -> void { auto& data = agg_method.data; - auto arena_memory_usage = _agg_arena_pool->size() + - _shared_state->aggregate_data_container->memory_usage() - - _dependency->mem_usage_record().used_in_arena; - _dependency->mem_tracker()->consume(arena_memory_usage); - _dependency->mem_tracker()->consume(data.get_buffer_size_in_bytes() - - _dependency->mem_usage_record().used_in_state); + 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( + data.get_buffer_size_in_bytes() - + Base::_dependency->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() - - _dependency->mem_usage_record().used_in_state); - _dependency->mem_usage_record().used_in_state = data.get_buffer_size_in_bytes(); - _dependency->mem_usage_record().used_in_arena = + Base::_dependency->mem_usage_record().used_in_state); + Base::_dependency->mem_usage_record().used_in_state = + data.get_buffer_size_in_bytes(); + Base::_dependency->mem_usage_record().used_in_arena = _agg_arena_pool->size() + - _shared_state->aggregate_data_container->memory_usage(); + Base::_shared_state->aggregate_data_container->memory_usage(); }, _agg_data->method_variant); } -template +template template -Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* block) { +Status AggSinkLocalState::_merge_with_serialized_key_helper( + vectorized::Block* block) { SCOPED_TIMER(_merge_timer); - size_t key_size = _shared_state->probe_expr_ctxs.size(); + size_t key_size = Base::_shared_state->probe_expr_ctxs.size(); vectorized::ColumnRawPtrs key_columns(key_size); for (size_t i = 0; i < key_size; ++i) { @@ -254,7 +261,8 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized: key_columns[i] = block->get_by_position(i).column.get(); } else { int result_column_id = -1; - RETURN_IF_ERROR(_shared_state->probe_expr_ctxs[i]->execute(block, &result_column_id)); + RETURN_IF_ERROR( + Base::_shared_state->probe_expr_ctxs[i]->execute(block, &result_column_id)); block->replace_by_position_if_const(result_column_id); key_columns[i] = block->get_by_position(result_column_id).column.get(); } @@ -268,49 +276,52 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized: if constexpr (limit) { _find_in_hash_table(_places.data(), key_columns, rows); - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - if (_shared_state->aggregate_evaluators[i]->is_merge()) { - int col_id = _get_slot_column_id(_shared_state->aggregate_evaluators[i]); + for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { + if (Base::_shared_state->aggregate_evaluators[i]->is_merge()) { + int col_id = _get_slot_column_id(Base::_shared_state->aggregate_evaluators[i]); auto column = block->get_by_position(col_id).column; if (column->is_nullable()) { column = ((vectorized::ColumnNullable*)column.get())->get_nested_column_ptr(); } size_t buffer_size = - _shared_state->aggregate_evaluators[i]->function()->size_of_data() * rows; + Base::_shared_state->aggregate_evaluators[i]->function()->size_of_data() * + rows; if (_deserialize_buffer.size() < buffer_size) { _deserialize_buffer.resize(buffer_size); } { SCOPED_TIMER(_deserialize_data_timer); - _shared_state->aggregate_evaluators[i] + Base::_shared_state->aggregate_evaluators[i] ->function() ->deserialize_and_merge_vec_selected( _places.data(), - _parent->cast() + Base::_parent->template cast() ._offsets_of_aggregate_states[i], _deserialize_buffer.data(), (vectorized::ColumnString*)(column.get()), _agg_arena_pool, rows); } } else { - RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add_selected( - block, - _parent->cast()._offsets_of_aggregate_states[i], - _places.data(), _agg_arena_pool)); + RETURN_IF_ERROR( + Base::_shared_state->aggregate_evaluators[i]->execute_batch_add_selected( + block, + Base::_parent->template cast() + ._offsets_of_aggregate_states[i], + _places.data(), _agg_arena_pool)); } } } else { _emplace_into_hash_table(_places.data(), key_columns, rows); - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - if (_shared_state->aggregate_evaluators[i]->is_merge() || for_spill) { + for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { + if (Base::_shared_state->aggregate_evaluators[i]->is_merge() || for_spill) { int col_id; if constexpr (for_spill) { - col_id = _shared_state->probe_expr_ctxs.size() + i; + col_id = Base::_shared_state->probe_expr_ctxs.size() + i; } else { - col_id = _get_slot_column_id(_shared_state->aggregate_evaluators[i]); + col_id = _get_slot_column_id(Base::_shared_state->aggregate_evaluators[i]); } auto column = block->get_by_position(col_id).column; if (column->is_nullable()) { @@ -318,31 +329,36 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized: } size_t buffer_size = - _shared_state->aggregate_evaluators[i]->function()->size_of_data() * rows; + Base::_shared_state->aggregate_evaluators[i]->function()->size_of_data() * + rows; if (_deserialize_buffer.size() < buffer_size) { _deserialize_buffer.resize(buffer_size); } { SCOPED_TIMER(_deserialize_data_timer); - _shared_state->aggregate_evaluators[i]->function()->deserialize_and_merge_vec( - _places.data(), - _parent->cast() - ._offsets_of_aggregate_states[i], - _deserialize_buffer.data(), (vectorized::ColumnString*)(column.get()), - _agg_arena_pool, rows); + Base::_shared_state->aggregate_evaluators[i] + ->function() + ->deserialize_and_merge_vec( + _places.data(), + Base::_parent->template cast() + ._offsets_of_aggregate_states[i], + _deserialize_buffer.data(), + (vectorized::ColumnString*)(column.get()), _agg_arena_pool, + rows); } } else { - RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add( + RETURN_IF_ERROR(Base::_shared_state->aggregate_evaluators[i]->execute_batch_add( block, - _parent->cast()._offsets_of_aggregate_states[i], + Base::_parent->template cast() + ._offsets_of_aggregate_states[i], _places.data(), _agg_arena_pool)); } } if (_should_limit_output) { - _reach_limit = - _get_hash_table_size() >= _parent->cast()._limit; + _reach_limit = _get_hash_table_size() >= + Base::_parent->template cast()._limit; } } @@ -352,8 +368,9 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized: // 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. -template -int AggSinkLocalState::_get_slot_column_id(const vectorized::AggFnEvaluator* evaluator) { +template +int AggSinkLocalState::_get_slot_column_id( + const vectorized::AggFnEvaluator* evaluator) { auto ctxs = evaluator->input_exprs_ctxs(); CHECK(ctxs.size() == 1 && ctxs[0]->root()->is_slot_ref()) << "input_exprs_ctxs is invalid, input_exprs_ctx[0]=" @@ -361,45 +378,50 @@ int AggSinkLocalState::_get_slot_column_id(const vectorized::AggFnEvalu return ((vectorized::VSlotRef*)ctxs[0]->root().get())->column_id(); } -template -Status AggSinkLocalState::_merge_without_key(vectorized::Block* block) { +template +Status AggSinkLocalState::_merge_without_key(vectorized::Block* block) { SCOPED_TIMER(_merge_timer); DCHECK(_agg_data->without_key != nullptr); - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - if (_shared_state->aggregate_evaluators[i]->is_merge()) { - int col_id = _get_slot_column_id(_shared_state->aggregate_evaluators[i]); + for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { + if (Base::_shared_state->aggregate_evaluators[i]->is_merge()) { + int col_id = _get_slot_column_id(Base::_shared_state->aggregate_evaluators[i]); auto column = block->get_by_position(col_id).column; if (column->is_nullable()) { column = ((vectorized::ColumnNullable*)column.get())->get_nested_column_ptr(); } SCOPED_TIMER(_deserialize_data_timer); - _shared_state->aggregate_evaluators[i]->function()->deserialize_and_merge_from_column( - _agg_data->without_key + _parent->cast() - ._offsets_of_aggregate_states[i], - *column, _agg_arena_pool); + Base::_shared_state->aggregate_evaluators[i] + ->function() + ->deserialize_and_merge_from_column( + _agg_data->without_key + + Base::_parent->template cast() + ._offsets_of_aggregate_states[i], + *column, _agg_arena_pool); } else { - RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_single_add( + RETURN_IF_ERROR(Base::_shared_state->aggregate_evaluators[i]->execute_single_add( block, - _agg_data->without_key + _parent->cast() - ._offsets_of_aggregate_states[i], + _agg_data->without_key + + Base::_parent->template cast() + ._offsets_of_aggregate_states[i], _agg_arena_pool)); } } return Status::OK(); } -template -void AggSinkLocalState::_update_memusage_without_key() { +template +void AggSinkLocalState::_update_memusage_without_key() { auto arena_memory_usage = - _agg_arena_pool->size() - _dependency->mem_usage_record().used_in_arena; - _dependency->mem_tracker()->consume(arena_memory_usage); + _agg_arena_pool->size() - Base::_dependency->mem_usage_record().used_in_arena; + Base::_dependency->mem_tracker()->consume(arena_memory_usage); _serialize_key_arena_memory_usage->add(arena_memory_usage); - _dependency->mem_usage_record().used_in_arena = _agg_arena_pool->size(); + Base::_dependency->mem_usage_record().used_in_arena = _agg_arena_pool->size(); } -template -Status AggSinkLocalState::_execute_with_serialized_key(vectorized::Block* block) { +template +Status AggSinkLocalState::_execute_with_serialized_key( + vectorized::Block* block) { if (_reach_limit) { return _execute_with_serialized_key_helper(block); } else { @@ -407,19 +429,21 @@ Status AggSinkLocalState::_execute_with_serialized_key(vectorized::Bloc } } -template +template template -Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* block) { +Status AggSinkLocalState::_execute_with_serialized_key_helper( + vectorized::Block* block) { SCOPED_TIMER(_build_timer); - DCHECK(!_shared_state->probe_expr_ctxs.empty()); + DCHECK(!Base::_shared_state->probe_expr_ctxs.empty()); - size_t key_size = _shared_state->probe_expr_ctxs.size(); + size_t key_size = Base::_shared_state->probe_expr_ctxs.size(); vectorized::ColumnRawPtrs key_columns(key_size); { SCOPED_TIMER(_expr_timer); for (size_t i = 0; i < key_size; ++i) { int result_column_id = -1; - RETURN_IF_ERROR(_shared_state->probe_expr_ctxs[i]->execute(block, &result_column_id)); + RETURN_IF_ERROR( + Base::_shared_state->probe_expr_ctxs[i]->execute(block, &result_column_id)); block->get_by_position(result_column_id).column = block->get_by_position(result_column_id) .column->convert_to_full_column_if_const(); @@ -435,27 +459,31 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorize if constexpr (limit) { _find_in_hash_table(_places.data(), key_columns, rows); - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add_selected( - block, - _parent->cast()._offsets_of_aggregate_states[i], - _places.data(), _agg_arena_pool)); + for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { + RETURN_IF_ERROR( + Base::_shared_state->aggregate_evaluators[i]->execute_batch_add_selected( + block, + Base::_parent->template cast() + ._offsets_of_aggregate_states[i], + _places.data(), _agg_arena_pool)); } } else { _emplace_into_hash_table(_places.data(), key_columns, rows); - for (int i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - RETURN_IF_ERROR(_shared_state->aggregate_evaluators[i]->execute_batch_add( + for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { + RETURN_IF_ERROR(Base::_shared_state->aggregate_evaluators[i]->execute_batch_add( block, - _parent->cast()._offsets_of_aggregate_states[i], + Base::_parent->template cast() + ._offsets_of_aggregate_states[i], _places.data(), _agg_arena_pool)); } if (_should_limit_output) { - _reach_limit = - _get_hash_table_size() >= _parent->cast()._limit; - if (_reach_limit && _parent->cast()._can_short_circuit) { - _dependency->set_done(); + _reach_limit = _get_hash_table_size() >= + Base::_parent->template cast()._limit; + if (_reach_limit && + Base::_parent->template cast()._can_short_circuit) { + Base::_dependency->set_ready_for_read(); return Status::Error(""); } } @@ -464,23 +492,23 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorize return Status::OK(); } -template -size_t AggSinkLocalState::_get_hash_table_size() { +template +size_t AggSinkLocalState::_get_hash_table_size() { return std::visit([&](auto&& agg_method) { return agg_method.data.size(); }, _agg_data->method_variant); } -template -void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* places, - vectorized::ColumnRawPtrs& key_columns, - const size_t num_rows) { +template +void AggSinkLocalState::_emplace_into_hash_table( + vectorized::AggregateDataPtr* places, vectorized::ColumnRawPtrs& key_columns, + const size_t num_rows) { std::visit( [&](auto&& agg_method) -> void { SCOPED_TIMER(_hash_table_compute_timer); using HashMethodType = std::decay_t; using HashTableType = std::decay_t; using AggState = typename HashMethodType::State; - AggState state(key_columns, _shared_state->probe_key_sz, nullptr); + AggState state(key_columns, Base::_shared_state->probe_key_sz, nullptr); _pre_serialize_key_if_need(state, agg_method, key_columns, num_rows); @@ -491,23 +519,25 @@ void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateD StringRef string_ref = to_string_ref(key); vectorized::ArenaKeyHolder key_holder {string_ref, *_agg_arena_pool}; key_holder_persist_key(key_holder); - auto mapped = _shared_state->aggregate_data_container->append_data( + auto mapped = Base::_shared_state->aggregate_data_container->append_data( key_holder.key); - _dependency->create_agg_status(mapped); + Base::_dependency->create_agg_status(mapped); ctor(key, mapped); } else { - auto mapped = _shared_state->aggregate_data_container->append_data(key); - _dependency->create_agg_status(mapped); + auto mapped = + Base::_shared_state->aggregate_data_container->append_data(key); + Base::_dependency->create_agg_status(mapped); ctor(key, mapped); } }; auto creator_for_null_key = [&](auto& mapped) { mapped = _agg_arena_pool->aligned_alloc( - _parent->cast() + Base::_parent->template cast() ._total_size_of_aggregate_states, - _parent->cast()._align_aggregate_states); - _dependency->create_agg_status(mapped); + Base::_parent->template cast() + ._align_aggregate_states); + Base::_dependency->create_agg_status(mapped); }; if constexpr (HashTableTraits::is_phmap) { @@ -557,16 +587,16 @@ void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateD _agg_data->method_variant); } -template -void AggSinkLocalState::_find_in_hash_table(vectorized::AggregateDataPtr* places, - vectorized::ColumnRawPtrs& key_columns, - size_t num_rows) { +template +void AggSinkLocalState::_find_in_hash_table( + vectorized::AggregateDataPtr* places, vectorized::ColumnRawPtrs& key_columns, + size_t num_rows) { std::visit( [&](auto&& agg_method) -> void { using HashMethodType = std::decay_t; using HashTableType = std::decay_t; using AggState = typename HashMethodType::State; - AggState state(key_columns, _shared_state->probe_key_sz, nullptr); + AggState state(key_columns, Base::_shared_state->probe_key_sz, nullptr); _pre_serialize_key_if_need(state, agg_method, key_columns, num_rows); @@ -611,8 +641,8 @@ void AggSinkLocalState::_find_in_hash_table(vectorized::AggregateDataPt _agg_data->method_variant); } -template -void AggSinkLocalState::_init_hash_method( +template +void AggSinkLocalState::_init_hash_method( const vectorized::VExprContextSPtrs& probe_exprs) { DCHECK(probe_exprs.size() >= 1); @@ -667,18 +697,21 @@ void AggSinkLocalState::_init_hash_method( t = Type::serialized; } - _agg_data->init(get_hash_key_type_with_phase( - t, !_parent->cast()._is_first_phase), - is_nullable); + _agg_data->init( + get_hash_key_type_with_phase( + t, + !Base::_parent->template cast()._is_first_phase), + is_nullable); } else { bool use_fixed_key = true; bool has_null = false; size_t key_byte_size = 0; - size_t bitmap_size = vectorized::get_bitmap_size(_shared_state->probe_expr_ctxs.size()); + size_t bitmap_size = + vectorized::get_bitmap_size(Base::_shared_state->probe_expr_ctxs.size()); - _shared_state->probe_key_sz.resize(_shared_state->probe_expr_ctxs.size()); - for (int i = 0; i < _shared_state->probe_expr_ctxs.size(); ++i) { - const auto& expr = _shared_state->probe_expr_ctxs[i]->root(); + Base::_shared_state->probe_key_sz.resize(Base::_shared_state->probe_expr_ctxs.size()); + for (int i = 0; i < Base::_shared_state->probe_expr_ctxs.size(); ++i) { + const auto& expr = Base::_shared_state->probe_expr_ctxs[i]->root(); const auto& data_type = expr->data_type(); if (!data_type->have_maximum_size_of_value()) { @@ -688,9 +721,9 @@ void AggSinkLocalState::_init_hash_method( auto is_null = data_type->is_nullable(); has_null |= is_null; - _shared_state->probe_key_sz[i] = + Base::_shared_state->probe_key_sz[i] = data_type->get_maximum_size_of_value_in_memory() - (is_null ? 1 : 0); - key_byte_size += _shared_state->probe_key_sz[i]; + key_byte_size += Base::_shared_state->probe_key_sz[i]; } if (!has_null) { @@ -712,7 +745,8 @@ void AggSinkLocalState::_init_hash_method( t = Type::int256_keys; } _agg_data->init(get_hash_key_type_with_phase( - t, !_parent->cast()._is_first_phase), + t, !Base::_parent->template cast() + ._is_first_phase), has_null); } else { _agg_data->init(Type::serialized); @@ -720,16 +754,18 @@ void AggSinkLocalState::_init_hash_method( } } -template -Status AggSinkLocalState::try_spill_disk(bool eos) { - if (_parent->cast()._external_agg_bytes_threshold == 0) { +template +Status AggSinkLocalState::try_spill_disk(bool eos) { + if (Base::_parent->template cast()._external_agg_bytes_threshold == + 0) { return Status::OK(); } return std::visit( [&](auto&& agg_method) -> Status { auto& hash_table = agg_method.data; - if (!eos && _memory_usage() < _parent->cast() - ._external_agg_bytes_threshold) { + if (!eos && + _memory_usage() < Base::_parent->template cast() + ._external_agg_bytes_threshold) { return Status::OK(); } @@ -738,7 +774,7 @@ Status AggSinkLocalState::try_spill_disk(bool eos) { } RETURN_IF_ERROR(_spill_hash_table(agg_method, hash_table)); - return _dependency->reset_hash_table(); + return Base::_dependency->reset_hash_table(); }, _agg_data->method_variant); } @@ -850,14 +886,6 @@ Status AggSinkOperatorX::prepare(RuntimeState* state) { } } - fmt::memory_buffer msg; - fmt::format_to(msg, - "(_is_merge: {}, _needs_finalize: {}, agg size: " - "{}, limit: {})", - _is_merge ? "true" : "false", _needs_finalize ? "true" : "false", - std::to_string(_aggregate_evaluators.size()), std::to_string(_limit)); - std::string title = fmt::format("Aggregation Sink {}", fmt::to_string(msg)); - DataSinkOperatorX::_profile = _pool->add(new RuntimeProfile(title)); return Status::OK(); } @@ -878,6 +906,8 @@ Status AggSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in_block, SourceState source_state) { auto& local_state = state->get_sink_local_state(id())->template cast(); + SCOPED_TIMER(local_state.profile()->total_time_counter()); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); local_state._shared_state->input_num_rows += in_block->rows(); if (in_block->rows() > 0) { RETURN_IF_ERROR(local_state._executor.execute(in_block)); @@ -889,14 +919,15 @@ Status AggSinkOperatorX::sink(doris::RuntimeState* state, local_state.try_spill_disk(true); RETURN_IF_ERROR(local_state._shared_state->spill_context.prepare_for_reading()); } - local_state._dependency->set_done(); + local_state._dependency->set_ready_for_read(); } return Status::OK(); } -template -Status AggSinkLocalState::close(RuntimeState* state) { - if (_closed) { +template +Status AggSinkLocalState::close(RuntimeState* state) { + SCOPED_TIMER(Base::profile()->total_time_counter()); + if (Base::_closed) { return Status::OK(); } _preagg_block.clear(); @@ -908,14 +939,14 @@ Status AggSinkLocalState::close(RuntimeState* state) { std::vector tmp_hash_values; _hash_values.swap(tmp_hash_values); - return PipelineXSinkLocalState::close(state); + return Base::close(state); } class StreamingAggSinkLocalState; template class AggSinkOperatorX; template class AggSinkOperatorX; -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 803899cd7a..0a484803cb 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -48,13 +48,14 @@ public: template class AggSinkOperatorX; -template -class AggSinkLocalState : public PipelineXSinkLocalState { +template +class AggSinkLocalState : public PipelineXSinkLocalState { public: + using Base = PipelineXSinkLocalState; virtual ~AggSinkLocalState() = default; virtual Status init(RuntimeState* state, LocalSinkStateInfo& info) override; - virtual Status open(RuntimeState* state) override; + Status open(RuntimeState* state) override; virtual Status close(RuntimeState* state) override; Status try_spill_disk(bool eos = false); @@ -102,8 +103,8 @@ protected: template Status _serialize_hash_table_to_block(HashTableCtxType& context, HashTableType& hash_table, vectorized::Block& block, std::vector& keys_) { - int key_size = _shared_state->probe_expr_ctxs.size(); - int agg_size = _shared_state->aggregate_evaluators.size(); + int key_size = Base::_shared_state->probe_expr_ctxs.size(); + int agg_size = Base::_shared_state->aggregate_evaluators.size(); vectorized::MutableColumns value_columns(agg_size); vectorized::DataTypes value_data_types(agg_size); @@ -111,31 +112,32 @@ protected: for (int i = 0; i < key_size; ++i) { key_columns.emplace_back( - _shared_state->probe_expr_ctxs[i]->root()->data_type()->create_column()); + Base::_shared_state->probe_expr_ctxs[i]->root()->data_type()->create_column()); } - for (size_t i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { + for (size_t i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { value_data_types[i] = - _shared_state->aggregate_evaluators[i]->function()->get_serialized_type(); - value_columns[i] = - _shared_state->aggregate_evaluators[i]->function()->create_serialize_column(); + Base::_shared_state->aggregate_evaluators[i]->function()->get_serialized_type(); + value_columns[i] = Base::_shared_state->aggregate_evaluators[i] + ->function() + ->create_serialize_column(); } context.init_once(); const auto size = hash_table.size(); std::vector keys(size); - if (_shared_state->values.size() < size) { - _shared_state->values.resize(size); + if (Base::_shared_state->values.size() < size) { + Base::_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; + Base::_shared_state->aggregate_data_container->init_once(); + auto& iter = Base::_shared_state->aggregate_data_container->iterator; { - while (iter != _shared_state->aggregate_data_container->end()) { - keys[num_rows] = iter.get_key(); - _shared_state->values[num_rows] = iter.get_aggregate_data(); + while (iter != Base::_shared_state->aggregate_data_container->end()) { + keys[num_rows] = iter.template get_key(); + Base::_shared_state->values[num_rows] = iter.get_aggregate_data(); ++iter; ++num_rows; } @@ -143,7 +145,7 @@ protected: { context.insert_keys_into_columns(keys, key_columns, num_rows, - _shared_state->probe_key_sz); + Base::_shared_state->probe_key_sz); } if (hash_table.has_null_key_data()) { @@ -154,27 +156,28 @@ protected: key_columns[0]->insert_data(nullptr, 0); // Here is no need to set `keys[num_rows]`, keep it as default value. - _shared_state->values[num_rows] = hash_table.get_null_key_data(); + Base::_shared_state->values[num_rows] = hash_table.get_null_key_data(); ++num_rows; } - for (size_t i = 0; i < _shared_state->aggregate_evaluators.size(); ++i) { - _shared_state->aggregate_evaluators[i]->function()->serialize_to_column( - _shared_state->values, _dependency->offsets_of_aggregate_states()[i], - value_columns[i], num_rows); + 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], + num_rows); } 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()); + Base::_shared_state->probe_expr_ctxs[i]->root()->data_type(), + Base::_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], - _shared_state->aggregate_evaluators[i]->function()->get_name()); + Base::_shared_state->aggregate_evaluators[i]->function()->get_name()); } block = columns_with_schema; @@ -190,42 +193,43 @@ protected: CHECK_EQ(block.rows(), hash_table.size()); CHECK_EQ(keys.size(), block.rows()); - if (!_shared_state->spill_context.has_data) { - _shared_state->spill_context.has_data = true; - _shared_state->spill_context.runtime_profile = - profile()->create_child("Spill", true, true); + if (!Base::_shared_state->spill_context.has_data) { + Base::_shared_state->spill_context.has_data = true; + Base::_shared_state->spill_context.runtime_profile = + Base::profile()->create_child("Spill", true, true); } vectorized::BlockSpillWriterUPtr writer; RETURN_IF_ERROR(ExecEnv::GetInstance()->block_spill_mgr()->get_writer( std::numeric_limits::max(), writer, - _shared_state->spill_context.runtime_profile)); + Base::_shared_state->spill_context.runtime_profile)); Defer defer {[&]() { // redundant call is ok writer->close(); }}; - _shared_state->spill_context.stream_ids.emplace_back(writer->get_id()); + Base::_shared_state->spill_context.stream_ids.emplace_back(writer->get_id()); std::vector partitioned_indices(block.rows()); - std::vector blocks_rows(_shared_state->spill_partition_helper->partition_count); + std::vector blocks_rows( + Base::_shared_state->spill_partition_helper->partition_count); // The last row may contain a null key. const size_t rows = hash_table.has_null_key_data() ? block.rows() - 1 : block.rows(); for (size_t i = 0; i < rows; ++i) { - const auto index = - _shared_state->spill_partition_helper->get_index(hash_table.hash(keys[i])); + const auto index = Base::_shared_state->spill_partition_helper->get_index( + hash_table.hash(keys[i])); partitioned_indices[i] = index; blocks_rows[index]++; } if (hash_table.has_null_key_data()) { // Here put the row with null key at the last partition. - const auto index = _shared_state->spill_partition_helper->partition_count - 1; + const auto index = Base::_shared_state->spill_partition_helper->partition_count - 1; partitioned_indices[rows] = index; blocks_rows[index]++; } - for (size_t i = 0; i < _shared_state->spill_partition_helper->partition_count; ++i) { + for (size_t i = 0; i < Base::_shared_state->spill_partition_helper->partition_count; ++i) { vectorized::Block block_to_write = block.clone_empty(); if (blocks_rows[i] == 0) { /// Here write one empty block to ensure there are enough blocks in the file, @@ -313,13 +317,14 @@ protected: executor _executor; }; -class BlockingAggSinkLocalState : public AggSinkLocalState { +class BlockingAggSinkLocalState + : public AggSinkLocalState { public: ENABLE_FACTORY_CREATOR(BlockingAggSinkLocalState); using Parent = AggSinkOperatorX; BlockingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : AggSinkLocalState(parent, state) {} + : AggSinkLocalState(parent, state) {} ~BlockingAggSinkLocalState() = default; }; @@ -346,7 +351,7 @@ public: using DataSinkOperatorX::id; protected: - template + template friend class AggSinkLocalState; friend class StreamingAggSinkLocalState; std::vector _aggregate_evaluators; diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index 054df663de..476683f8b8 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -20,6 +20,7 @@ #include #include "pipeline/exec/operator.h" +#include "pipeline/exec/streaming_aggregation_source_operator.h" #include "vec//utils/util.hpp" namespace doris { @@ -27,8 +28,9 @@ namespace pipeline { OPERATOR_CODE_GENERATOR(AggSourceOperator, SourceOperator) -AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) - : PipelineXLocalState(state, parent), +template +AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) + : Base(state, parent), _get_results_timer(nullptr), _serialize_result_timer(nullptr), _hash_table_iterate_timer(nullptr), @@ -36,122 +38,129 @@ AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) _serialize_data_timer(nullptr), _hash_table_size_counter(nullptr) {} -Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { - RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); - _agg_data = _shared_state->agg_data.get(); - _get_results_timer = ADD_TIMER(profile(), "GetResultsTime"); - _serialize_result_timer = ADD_TIMER(profile(), "SerializeResultTime"); - _hash_table_iterate_timer = ADD_TIMER(profile(), "HashTableIterateTime"); - _insert_keys_to_column_timer = ADD_TIMER(profile(), "InsertKeysToColumnTime"); - _serialize_data_timer = ADD_TIMER(profile(), "SerializeDataTime"); - _hash_table_size_counter = ADD_COUNTER(profile(), "HashTableSize", TUnit::UNIT); - auto& p = _parent->cast(); +template +Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(Base::profile()->total_time_counter()); + _agg_data = Base::_shared_state->agg_data.get(); + _get_results_timer = ADD_TIMER(Base::profile(), "GetResultsTime"); + _serialize_result_timer = ADD_TIMER(Base::profile(), "SerializeResultTime"); + _hash_table_iterate_timer = ADD_TIMER(Base::profile(), "HashTableIterateTime"); + _insert_keys_to_column_timer = ADD_TIMER(Base::profile(), "InsertKeysToColumnTime"); + _serialize_data_timer = ADD_TIMER(Base::profile(), "SerializeDataTime"); + _hash_table_size_counter = ADD_COUNTER(Base::profile(), "HashTableSize", TUnit::UNIT); + auto& p = Base::_parent->template cast(); if (p._without_key) { if (p._needs_finalize) { - _executor.get_result = std::bind(&AggLocalState::_get_without_key_result, this, - std::placeholders::_1, std::placeholders::_2, - std::placeholders::_3); + _executor.get_result = std::bind(&Derived::_get_without_key_result, + (Derived*)this, std::placeholders::_1, + std::placeholders::_2, std::placeholders::_3); } else { - _executor.get_result = std::bind(&AggLocalState::_serialize_without_key, this, - std::placeholders::_1, std::placeholders::_2, - std::placeholders::_3); + _executor.get_result = std::bind(&Derived::_serialize_without_key, + (Derived*)this, std::placeholders::_1, + std::placeholders::_2, std::placeholders::_3); } - _executor.close = std::bind(&AggLocalState::_close_without_key, this); + _executor.close = std::bind(&Derived::_close_without_key, (Derived*)this); } else { if (p._needs_finalize) { - _executor.get_result = std::bind( - &AggLocalState::_get_with_serialized_key_result, this, std::placeholders::_1, - std::placeholders::_2, std::placeholders::_3); + _executor.get_result = std::bind(&Derived::_get_with_serialized_key_result, + (Derived*)this, std::placeholders::_1, + std::placeholders::_2, std::placeholders::_3); } else { _executor.get_result = std::bind( - &AggLocalState::_serialize_with_serialized_key_result, this, + &Derived::_serialize_with_serialized_key_result, (Derived*)this, std::placeholders::_1, std::placeholders::_2, std::placeholders::_3); } - _executor.close = std::bind(&AggLocalState::_close_with_serialized_key, this); + _executor.close = std::bind(&Derived::_close_with_serialized_key, (Derived*)this); } _agg_data_created_without_key = p._without_key; return Status::OK(); } -void AggLocalState::_close_with_serialized_key() { +template +void AggLocalState::_close_with_serialized_key() { std::visit( [&](auto&& agg_method) -> void { auto& data = agg_method.data; data.for_each_mapped([&](auto& mapped) { if (mapped) { - _dependency->destroy_agg_status(mapped); + Base::_dependency->destroy_agg_status(mapped); mapped = nullptr; } }); if (data.has_null_key_data()) { - _dependency->destroy_agg_status(data.get_null_key_data()); + Base::_dependency->destroy_agg_status(data.get_null_key_data()); } }, _agg_data->method_variant); - _dependency->release_tracker(); + Base::_dependency->release_tracker(); } -void AggLocalState::_close_without_key() { +template +void AggLocalState::_close_without_key() { //because prepare maybe failed, and couldn't create agg data. //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) { - _dependency->destroy_agg_status(_agg_data->without_key); + Base::_dependency->destroy_agg_status(_agg_data->without_key); _agg_data_created_without_key = false; } - _dependency->release_tracker(); + Base::_dependency->release_tracker(); } -Status AggLocalState::_serialize_with_serialized_key_result(RuntimeState* state, - vectorized::Block* block, - SourceState& source_state) { - if (_shared_state->spill_context.has_data) { +template +Status AggLocalState::_serialize_with_serialized_key_result( + RuntimeState* state, vectorized::Block* block, SourceState& source_state) { + if (Base::_shared_state->spill_context.has_data) { return _serialize_with_serialized_key_result_with_spilt_data(state, block, source_state); } else { return _serialize_with_serialized_key_result_non_spill(state, block, source_state); } } -Status AggLocalState::_serialize_with_serialized_key_result_with_spilt_data( +template +Status +AggLocalState::_serialize_with_serialized_key_result_with_spilt_data( RuntimeState* state, vectorized::Block* block, SourceState& source_state) { - CHECK(!_shared_state->spill_context.stream_ids.empty()); - CHECK(_shared_state->spill_partition_helper != nullptr) + CHECK(!Base::_shared_state->spill_context.stream_ids.empty()); + CHECK(Base::_shared_state->spill_partition_helper != nullptr) << "_spill_partition_helper should not be null"; - _shared_state->aggregate_data_container->init_once(); - while (_shared_state->aggregate_data_container->iterator == - _shared_state->aggregate_data_container->end()) { - if (_shared_state->spill_context.read_cursor == - _shared_state->spill_partition_helper->partition_count) { + Base::_shared_state->aggregate_data_container->init_once(); + while (Base::_shared_state->aggregate_data_container->iterator == + Base::_shared_state->aggregate_data_container->end()) { + if (Base::_shared_state->spill_context.read_cursor == + Base::_shared_state->spill_partition_helper->partition_count) { break; } - RETURN_IF_ERROR(_dependency->reset_hash_table()); - RETURN_IF_ERROR(_dependency->merge_spilt_data()); - _shared_state->aggregate_data_container->init_once(); + RETURN_IF_ERROR(Base::_dependency->reset_hash_table()); + RETURN_IF_ERROR(Base::_dependency->merge_spilt_data()); + Base::_shared_state->aggregate_data_container->init_once(); } RETURN_IF_ERROR(_serialize_with_serialized_key_result_non_spill(state, block, source_state)); if (source_state == SourceState::FINISHED) { - source_state = _shared_state->spill_context.read_cursor == - _shared_state->spill_partition_helper->partition_count + source_state = Base::_shared_state->spill_context.read_cursor == + Base::_shared_state->spill_partition_helper->partition_count ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; } CHECK(!block->empty() || source_state == SourceState::FINISHED); return Status::OK(); } -Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeState* state, - vectorized::Block* block, - SourceState& source_state) { + +template +Status AggLocalState::_serialize_with_serialized_key_result_non_spill( + RuntimeState* state, vectorized::Block* block, SourceState& source_state) { SCOPED_TIMER(_serialize_result_timer); - int key_size = _shared_state->probe_expr_ctxs.size(); - int agg_size = _shared_state->aggregate_evaluators.size(); + int key_size = Base::_shared_state->probe_expr_ctxs.size(); + int agg_size = Base::_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 = Base::_dependency->make_nullable_keys().empty() && block->mem_reuse(); vectorized::MutableColumns key_columns; for (int i = 0; i < key_size; ++i) { @@ -159,7 +168,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()); + Base::_shared_state->probe_expr_ctxs[i]->root()->data_type()->create_column()); } } @@ -171,20 +180,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 (Base::_shared_state->values.size() < size + 1) { + Base::_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; + Base::_shared_state->aggregate_data_container->init_once(); + auto& iter = Base::_shared_state->aggregate_data_container->iterator; { SCOPED_TIMER(_hash_table_iterate_timer); - while (iter != _shared_state->aggregate_data_container->end() && + while (iter != Base::_shared_state->aggregate_data_container->end() && num_rows < state->batch_size()) { - keys[num_rows] = iter.get_key(); - _shared_state->values[num_rows] = iter.get_aggregate_data(); + keys[num_rows] = iter.template get_key(); + Base::_shared_state->values[num_rows] = iter.get_aggregate_data(); ++iter; ++num_rows; } @@ -193,10 +202,10 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta { SCOPED_TIMER(_insert_keys_to_column_timer); agg_method.insert_keys_into_columns(keys, key_columns, num_rows, - _shared_state->probe_key_sz); + Base::_shared_state->probe_key_sz); } - if (iter == _shared_state->aggregate_data_container->end()) { + if (iter == Base::_shared_state->aggregate_data_container->end()) { if (agg_method.data.has_null_key_data()) { // only one key of group by support wrap null key // here need additional processing logic on the null key / value @@ -204,7 +213,8 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta DCHECK(key_columns[0]->is_nullable()); if (agg_method.data.has_null_key_data()) { key_columns[0]->insert_data(nullptr, 0); - _shared_state->values[num_rows] = agg_method.data.get_null_key_data(); + Base::_shared_state->values[num_rows] = + agg_method.data.get_null_key_data(); ++num_rows; source_state = SourceState::FINISHED; } @@ -215,8 +225,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 < Base::_shared_state->aggregate_evaluators.size(); ++i) { + value_data_types[i] = Base::_shared_state->aggregate_evaluators[i] ->function() ->get_serialized_type(); if (mem_reuse) { @@ -224,14 +234,16 @@ 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] = Base::_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); + Base::_shared_state->aggregate_evaluators[i] + ->function() + ->serialize_to_column( + Base::_shared_state->values, + Base::_dependency->offsets_of_aggregate_states()[i], + value_columns[i], num_rows); } } }, @@ -242,8 +254,8 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta 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()); + Base::_shared_state->probe_expr_ctxs[i]->root()->data_type(), + Base::_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], ""); @@ -254,36 +266,38 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta return Status::OK(); } -Status AggLocalState::_get_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { - if (_shared_state->spill_context.has_data) { +template +Status AggLocalState::_get_with_serialized_key_result( + RuntimeState* state, vectorized::Block* block, SourceState& source_state) { + if (Base::_shared_state->spill_context.has_data) { return _get_result_with_spilt_data(state, block, source_state); } else { return _get_result_with_serialized_key_non_spill(state, block, source_state); } } -Status AggLocalState::_get_result_with_spilt_data(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { - CHECK(!_shared_state->spill_context.stream_ids.empty()); - CHECK(_shared_state->spill_partition_helper != nullptr) +template +Status AggLocalState::_get_result_with_spilt_data( + RuntimeState* state, vectorized::Block* block, SourceState& source_state) { + CHECK(!Base::_shared_state->spill_context.stream_ids.empty()); + CHECK(Base::_shared_state->spill_partition_helper != nullptr) << "_spill_partition_helper should not be null"; - _shared_state->aggregate_data_container->init_once(); - while (_shared_state->aggregate_data_container->iterator == - _shared_state->aggregate_data_container->end()) { - if (_shared_state->spill_context.read_cursor == - _shared_state->spill_partition_helper->partition_count) { + Base::_shared_state->aggregate_data_container->init_once(); + while (Base::_shared_state->aggregate_data_container->iterator == + Base::_shared_state->aggregate_data_container->end()) { + if (Base::_shared_state->spill_context.read_cursor == + Base::_shared_state->spill_partition_helper->partition_count) { break; } - RETURN_IF_ERROR(_dependency->reset_hash_table()); - RETURN_IF_ERROR(_dependency->merge_spilt_data()); - _shared_state->aggregate_data_container->init_once(); + RETURN_IF_ERROR(Base::_dependency->reset_hash_table()); + RETURN_IF_ERROR(Base::_dependency->merge_spilt_data()); + Base::_shared_state->aggregate_data_container->init_once(); } RETURN_IF_ERROR(_get_result_with_serialized_key_non_spill(state, block, source_state)); if (source_state == SourceState::FINISHED) { - source_state = _shared_state->spill_context.read_cursor == - _shared_state->spill_partition_helper->partition_count + source_state = Base::_shared_state->spill_context.read_cursor == + Base::_shared_state->spill_partition_helper->partition_count ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE; } @@ -291,15 +305,15 @@ Status AggLocalState::_get_result_with_spilt_data(RuntimeState* state, vectorize return Status::OK(); } -Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* state, - vectorized::Block* block, - SourceState& source_state) { +template +Status AggLocalState::_get_result_with_serialized_key_non_spill( + RuntimeState* state, vectorized::Block* block, SourceState& source_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 = Base::_dependency->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(); + Base::_parent->template cast()._row_descriptor); + int key_size = Base::_shared_state->probe_expr_ctxs.size(); vectorized::MutableColumns key_columns; for (int i = 0; i < key_size; ++i) { @@ -326,20 +340,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 (Base::_shared_state->values.size() < size) { + Base::_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; + Base::_shared_state->aggregate_data_container->init_once(); + auto& iter = Base::_shared_state->aggregate_data_container->iterator; { SCOPED_TIMER(_hash_table_iterate_timer); - while (iter != _shared_state->aggregate_data_container->end() && + while (iter != Base::_shared_state->aggregate_data_container->end() && num_rows < state->batch_size()) { - keys[num_rows] = iter.get_key(); - _shared_state->values[num_rows] = iter.get_aggregate_data(); + keys[num_rows] = iter.template get_key(); + Base::_shared_state->values[num_rows] = iter.get_aggregate_data(); ++iter; ++num_rows; } @@ -348,16 +362,17 @@ Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* st { SCOPED_TIMER(_insert_keys_to_column_timer); agg_method.insert_keys_into_columns(keys, key_columns, num_rows, - _shared_state->probe_key_sz); + Base::_shared_state->probe_key_sz); } - 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 < Base::_shared_state->aggregate_evaluators.size(); ++i) { + Base::_shared_state->aggregate_evaluators[i]->insert_result_info_vec( + Base::_shared_state->values, + Base::_dependency->offsets_of_aggregate_states()[i], value_columns[i].get(), num_rows); } - if (iter == _shared_state->aggregate_data_container->end()) { + if (iter == Base::_shared_state->aggregate_data_container->end()) { if (agg_method.data.has_null_key_data()) { // only one key of group by support wrap null key // here need additional processing logic on the null key / value @@ -366,9 +381,11 @@ Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* st if (key_columns[0]->size() < state->batch_size()) { key_columns[0]->insert_data(nullptr, 0); auto mapped = agg_method.data.get_null_key_data(); - 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 < Base::_shared_state->aggregate_evaluators.size(); + ++i) + Base::_shared_state->aggregate_evaluators[i]->insert_result_info( + mapped + + Base::_dependency->offsets_of_aggregate_states()[i], value_columns[i].get()); source_state = SourceState::FINISHED; } @@ -395,39 +412,42 @@ Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* st return Status::OK(); } -Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { +template +Status AggLocalState::_serialize_without_key(RuntimeState* state, + vectorized::Block* block, + SourceState& source_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 SCOPED_TIMER(_serialize_result_timer); - if (UNLIKELY(_shared_state->input_num_rows == 0)) { + if (UNLIKELY(Base::_shared_state->input_num_rows == 0)) { source_state = SourceState::FINISHED; return Status::OK(); } block->clear(); DCHECK(_agg_data->without_key != nullptr); - int agg_size = _shared_state->aggregate_evaluators.size(); + int agg_size = Base::_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 < Base::_shared_state->aggregate_evaluators.size(); ++i) { + data_types[i] = + Base::_shared_state->aggregate_evaluators[i]->function()->get_serialized_type(); value_columns[i] = - _shared_state->aggregate_evaluators[i]->function()->create_serialize_column(); + Base::_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 < Base::_shared_state->aggregate_evaluators.size(); ++i) { + Base::_shared_state->aggregate_evaluators[i]->function()->serialize_without_key_to_column( + _agg_data->without_key + Base::_dependency->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 < Base::_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)); } @@ -439,26 +459,29 @@ Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Bl return Status::OK(); } -Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) { +template +Status AggLocalState::_get_without_key_result(RuntimeState* state, + vectorized::Block* block, + SourceState& source_state) { DCHECK(_agg_data->without_key != nullptr); block->clear(); - auto& p = _parent->cast(); + auto& p = Base::_parent->template cast(); *block = vectorized::VectorizedUtils::create_empty_columnswithtypename(p._row_descriptor); - int agg_size = _shared_state->aggregate_evaluators.size(); + int agg_size = Base::_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 < Base::_shared_state->aggregate_evaluators.size(); ++i) { + data_types[i] = Base::_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 < Base::_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); + Base::_shared_state->aggregate_evaluators[i]->insert_result_info( + _agg_data->without_key + Base::_dependency->offsets_of_aggregate_states()[i], + column); } const auto& block_schema = block->get_columns_with_type_and_name(); @@ -479,7 +502,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, Base::_shared_state->input_num_rows == 0); columns[i] = ptr->assume_mutable(); } } @@ -492,37 +515,39 @@ Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::B AggSourceOperatorX::AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : OperatorX(pool, tnode, descs), + : Base(pool, tnode, descs), _needs_finalize(tnode.agg_node.need_finalize), _without_key(tnode.agg_node.grouping_exprs.empty()) {} Status AggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { - auto& local_state = state->get_local_state(id())->cast(); + auto& local_state = state->get_local_state(id())->cast(); + SCOPED_TIMER(local_state.profile()->total_time_counter()); RETURN_IF_ERROR(local_state._executor.get_result(state, block, source_state)); local_state.make_nullable_output_key(block); // dispose the having clause, should not be execute in prestreaming agg RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, block, block->columns())); - bool eos = false; - local_state.reached_limit(block, &eos); - source_state = eos ? SourceState::FINISHED : source_state; + local_state.reached_limit(block, source_state); return Status::OK(); } -void AggLocalState::make_nullable_output_key(vectorized::Block* block) { +template +void AggLocalState::make_nullable_output_key(vectorized::Block* block) { if (block->rows() != 0) { - for (auto cid : _dependency->make_nullable_keys()) { + for (auto cid : Base::_dependency->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); } } } -Status AggLocalState::close(RuntimeState* state) { - if (_closed) { +template +Status AggLocalState::close(RuntimeState* state) { + SCOPED_TIMER(Base::profile()->total_time_counter()); + if (Base::_closed) { return Status::OK(); } - for (auto* aggregate_evaluator : _shared_state->aggregate_evaluators) { + for (auto* aggregate_evaluator : Base::_shared_state->aggregate_evaluators) { aggregate_evaluator->close(state); } if (_executor.close) { @@ -538,19 +563,26 @@ Status AggLocalState::close(RuntimeState* state) { _agg_data->method_variant); } - _shared_state->agg_data = nullptr; - _shared_state->aggregate_data_container = nullptr; - _shared_state->agg_arena_pool = nullptr; - _shared_state->agg_profile_arena = nullptr; + Base::_shared_state->agg_data = nullptr; + Base::_shared_state->aggregate_data_container = nullptr; + Base::_shared_state->agg_arena_pool = nullptr; + Base::_shared_state->agg_profile_arena = nullptr; std::vector tmp_values; - _shared_state->values.swap(tmp_values); - return PipelineXLocalState::close(state); + Base::_shared_state->values.swap(tmp_values); + return Base::close(state); } -bool AggSourceOperatorX::can_read(RuntimeState* state) { - return state->get_local_state(id())->cast()._dependency->done(); +Dependency* AggSourceOperatorX::wait_for_dependency(RuntimeState* state) { + return state->get_local_state(Base::id()) + ->cast() + ._dependency->read_blocked_by(); } +class StreamingAggLocalState; + +template class AggLocalState; +template class AggLocalState; + } // namespace pipeline -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h index 80a104a410..3148f06792 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.h +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -49,8 +49,11 @@ public: }; class AggSourceOperatorX; -class AggLocalState final : public PipelineXLocalState { + +template +class AggLocalState : public PipelineXLocalState { public: + using Base = PipelineXLocalState; ENABLE_FACTORY_CREATOR(AggLocalState); AggLocalState(RuntimeState* state, OperatorXBase* parent); @@ -59,7 +62,7 @@ public: void make_nullable_output_key(vectorized::Block* block); -private: +protected: friend class AggSourceOperatorX; friend class StreamingAggSourceOperatorX; @@ -107,18 +110,30 @@ private: bool _agg_data_created_without_key = false; }; -class AggSourceOperatorX : public OperatorX { +class BlockingAggLocalState final : public AggLocalState { public: - AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - virtual ~AggSourceOperatorX() = default; - virtual bool can_read(RuntimeState* state) override; + ENABLE_FACTORY_CREATOR(BlockingAggLocalState); + using Parent = AggSourceOperatorX; - virtual Status get_block(RuntimeState* state, vectorized::Block* block, - SourceState& source_state) override; + BlockingAggLocalState(RuntimeState* state, OperatorXBase* parent) + : AggLocalState(state, parent) {} + ~BlockingAggLocalState() = default; +}; + +class AggSourceOperatorX final : public OperatorX { +public: + using Base = OperatorX; + AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + ~AggSourceOperatorX() = default; + Dependency* wait_for_dependency(RuntimeState* state) override; + + Status get_block(RuntimeState* state, vectorized::Block* block, + SourceState& source_state) override; bool is_source() const override { return true; } private: + template friend class AggLocalState; bool _needs_finalize; diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index 75cc6e336a..ed00599462 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -117,7 +117,6 @@ Status AnalyticSinkOperatorX::prepare(RuntimeState* state) { vectorized::VExpr::prepare(_order_by_eq_expr_ctxs, state, cmp_row_desc)); } } - _profile = state->obj_pool()->add(new RuntimeProfile("AnalyticSinkOperatorX")); return Status::OK(); } @@ -142,6 +141,7 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block local_state._shared_state->input_eos = source_state == SourceState::FINISHED; if (local_state._shared_state->input_eos && input_block->rows() == 0) { local_state._shared_state->need_more_input = false; + local_state._dependency->set_ready_for_read(); return Status::OK(); } @@ -197,9 +197,7 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block local_state._shared_state->found_partition_end = local_state._dependency->get_partition_by_end(); } - local_state._shared_state->need_more_input = - local_state._dependency->whether_need_next_partition( - local_state._shared_state->found_partition_end); + local_state._dependency->refresh_need_more_input(); return Status::OK(); } diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp index 6ae5125058..0f0e85c2ee 100644 --- a/be/src/pipeline/exec/analytic_source_operator.cpp +++ b/be/src/pipeline/exec/analytic_source_operator.cpp @@ -404,10 +404,7 @@ Status AnalyticSourceOperatorX::get_block(RuntimeState* state, vectorized::Block local_state._shared_state->found_partition_end = local_state._dependency->get_partition_by_end(); } - local_state._shared_state->need_more_input = - local_state._dependency->whether_need_next_partition( - local_state._shared_state->found_partition_end); - if (local_state._shared_state->need_more_input) { + if (local_state._dependency->refresh_need_more_input()) { return Status::OK(); } local_state._next_partition = @@ -427,12 +424,9 @@ Status AnalyticSourceOperatorX::get_block(RuntimeState* state, vectorized::Block return Status::OK(); } -bool AnalyticSourceOperatorX::can_read(RuntimeState* state) { +Dependency* AnalyticSourceOperatorX::wait_for_dependency(RuntimeState* state) { auto& local_state = state->get_local_state(id())->cast(); - if (local_state._shared_state->need_more_input) { - return false; - } - return true; + return local_state._dependency->read_blocked_by(); } Status AnalyticLocalState::close(RuntimeState* state) { diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 866177ff71..36497c1ffa 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -117,7 +117,7 @@ private: class AnalyticSourceOperatorX final : public OperatorX { public: AnalyticSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - bool can_read(RuntimeState* state) override; + Dependency* wait_for_dependency(RuntimeState* state) override; Status get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) override; diff --git a/be/src/pipeline/exec/data_queue.cpp b/be/src/pipeline/exec/data_queue.cpp index bdcc12df95..eff91bdd3f 100644 --- a/be/src/pipeline/exec/data_queue.cpp +++ b/be/src/pipeline/exec/data_queue.cpp @@ -24,12 +24,14 @@ #include #include "gutil/integral_types.h" +#include "pipeline/pipeline_x/dependency.h" #include "vec/core/block.h" namespace doris { namespace pipeline { -DataQueue::DataQueue(int child_count) +DataQueue::DataQueue(int child_count, StreamingAggDependency* agg_dependency, + UnionDependency* union_dependency) : _queue_blocks_lock(child_count), _queue_blocks(child_count), _free_blocks_lock(child_count), @@ -39,7 +41,9 @@ DataQueue::DataQueue(int child_count) _is_canceled(child_count), _cur_bytes_in_queue(child_count), _cur_blocks_nums_in_queue(child_count), - _flag_queue_idx(0) { + _flag_queue_idx(0), + _agg_dependency(agg_dependency), + _union_dependency(union_dependency) { for (int i = 0; i < child_count; ++i) { _queue_blocks_lock[i].reset(new std::mutex()); _free_blocks_lock[i].reset(new std::mutex()); @@ -115,6 +119,10 @@ 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 (_agg_dependency && _cur_blocks_nums_in_queue[_flag_queue_idx] == 0 && + !_is_finished[0]) { + _agg_dependency->block_reading(); + } } else { if (_is_finished[_flag_queue_idx]) { _data_exhausted = true; @@ -133,6 +141,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 (_agg_dependency) { + _agg_dependency->set_ready_for_read(); + } //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()); _max_size_of_queue = std::max(_max_size_of_queue, (int64)_queue_blocks[0].size()); @@ -141,12 +152,24 @@ void DataQueue::push_block(std::unique_ptr block, int child_i void DataQueue::set_finish(int child_idx) { _is_finished[child_idx] = true; + if (_agg_dependency) { + _agg_dependency->set_ready_for_read(); + } + if (_union_dependency && is_all_finish()) { + _union_dependency->set_ready_for_read(); + } } void DataQueue::set_canceled(int child_idx) { DCHECK(!_is_finished[child_idx]); _is_canceled[child_idx] = true; _is_finished[child_idx] = true; + if (_agg_dependency) { + _agg_dependency->set_ready_for_read(); + } + if (_union_dependency && is_all_finish()) { + _union_dependency->set_ready_for_read(); + } } bool DataQueue::is_finish(int child_idx) { diff --git a/be/src/pipeline/exec/data_queue.h b/be/src/pipeline/exec/data_queue.h index e7f2fdfada..e6105a3264 100644 --- a/be/src/pipeline/exec/data_queue.h +++ b/be/src/pipeline/exec/data_queue.h @@ -30,10 +30,14 @@ namespace doris { namespace pipeline { +class StreamingAggDependency; +class UnionDependency; + class DataQueue { public: //always one is enough, but in union node it's has more children - DataQueue(int child_count = 1); + DataQueue(int child_count = 1, StreamingAggDependency* agg_dependency = nullptr, + UnionDependency* union_dependency = nullptr); ~DataQueue() = default; Status get_block_from_queue(std::unique_ptr* block, @@ -83,6 +87,9 @@ private: int64_t _max_bytes_in_queue = 0; int64_t _max_size_of_queue = 0; static constexpr int64_t MAX_BYTE_OF_QUEUE = 1024l * 1024 * 1024 / 10; + + StreamingAggDependency* _agg_dependency = nullptr; + UnionDependency* _union_dependency = nullptr; }; } // namespace pipeline -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 229fdd304e..652cfdc5c4 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -121,11 +121,7 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf _memory_usage_counter = ADD_LABEL_COUNTER(_profile, "MemoryUsage"); _peak_memory_usage_counter = _profile->AddHighWaterMarkCounter("PeakMemoryUsage", TUnit::BYTES, "MemoryUsage"); - return Status::OK(); -} -Status ExchangeSinkLocalState::open(RuntimeState* state) { - RETURN_IF_ERROR(PipelineXSinkLocalState<>::open(state)); _broadcast_pb_blocks.resize(config::num_broadcast_buffer); _broadcast_pb_block_idx = 0; auto& p = _parent->cast(); @@ -178,7 +174,6 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { id, p._dest_node_id, _sender_id, _state->be_number(), state->get_query_ctx()); register_channels(_sink_buffer.get()); - return Status::OK(); } @@ -187,11 +182,10 @@ segment_v2::CompressionTypePB& ExchangeSinkLocalState::compression_type() { } ExchangeSinkOperatorX::ExchangeSinkOperatorX( - RuntimeState* state, ObjectPool* pool, const RowDescriptor& row_desc, - const TDataStreamSink& sink, const std::vector& destinations, + RuntimeState* state, const RowDescriptor& row_desc, const TDataStreamSink& sink, + const std::vector& destinations, bool send_query_statistics_with_every_batch) : DataSinkOperatorX(sink.dest_node_id), - _pool(pool), _row_desc(row_desc), _part_type(sink.output_partition.type), _dests(destinations), @@ -208,11 +202,10 @@ ExchangeSinkOperatorX::ExchangeSinkOperatorX( } ExchangeSinkOperatorX::ExchangeSinkOperatorX( - ObjectPool* pool, const RowDescriptor& row_desc, PlanNodeId dest_node_id, + const RowDescriptor& row_desc, PlanNodeId dest_node_id, const std::vector& destinations, bool send_query_statistics_with_every_batch) : DataSinkOperatorX(dest_node_id), - _pool(pool), _row_desc(row_desc), _part_type(TPartitionType::UNPARTITIONED), _dests(destinations), @@ -222,10 +215,9 @@ ExchangeSinkOperatorX::ExchangeSinkOperatorX( _name = "ExchangeSinkOperatorX"; } -ExchangeSinkOperatorX::ExchangeSinkOperatorX(ObjectPool* pool, const RowDescriptor& row_desc, +ExchangeSinkOperatorX::ExchangeSinkOperatorX(const RowDescriptor& row_desc, bool send_query_statistics_with_every_batch) : DataSinkOperatorX(0), - _pool(pool), _row_desc(row_desc), _send_query_statistics_with_every_batch(send_query_statistics_with_every_batch), _dest_node_id(0) { @@ -251,9 +243,6 @@ Status ExchangeSinkOperatorX::init(const TDataSink& tsink) { Status ExchangeSinkOperatorX::prepare(RuntimeState* state) { _state = state; - std::string title = fmt::format("VDataStreamSender (dst_id={})", _dest_node_id); - _profile = _pool->add(new RuntimeProfile(title)); - SCOPED_TIMER(_profile->total_time_counter()); _mem_tracker = std::make_unique("ExchangeSinkOperatorX:"); SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); @@ -283,7 +272,7 @@ void ExchangeSinkOperatorX::_handle_eof_channel(RuntimeState* state, ChannelPtrT Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) { auto& local_state = state->get_sink_local_state(id())->cast(); - SCOPED_TIMER(_profile->total_time_counter()); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); local_state._peak_memory_usage_counter->set(_mem_tracker->peak_consumption()); bool all_receiver_eof = true; for (auto channel : local_state.channels) { diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index caeb844175..134a9c9cc4 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -77,7 +77,6 @@ public: _serializer(this) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; - Status open(RuntimeState* state) override; Status close(RuntimeState* state) override; Status serialize_block(vectorized::Block* src, PBlock* dest, int num_receivers = 1); @@ -152,14 +151,14 @@ private: class ExchangeSinkOperatorX final : public DataSinkOperatorX { public: - ExchangeSinkOperatorX(RuntimeState* state, ObjectPool* pool, const RowDescriptor& row_desc, + ExchangeSinkOperatorX(RuntimeState* state, const RowDescriptor& row_desc, const TDataStreamSink& sink, const std::vector& destinations, bool send_query_statistics_with_every_batch); - ExchangeSinkOperatorX(ObjectPool* pool, const RowDescriptor& row_desc, PlanNodeId dest_node_id, + ExchangeSinkOperatorX(const RowDescriptor& row_desc, PlanNodeId dest_node_id, const std::vector& destinations, bool send_query_statistics_with_every_batch); - ExchangeSinkOperatorX(ObjectPool* pool, const RowDescriptor& row_desc, + ExchangeSinkOperatorX(const RowDescriptor& row_desc, bool send_query_statistics_with_every_batch); Status init(const TDataSink& tsink) override; @@ -198,7 +197,6 @@ private: bool eos); RuntimeState* _state = nullptr; - ObjectPool* _pool; const RowDescriptor& _row_desc; TPartitionType::type _part_type; diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp index 76cd2ece9d..5cbb97d247 100644 --- a/be/src/pipeline/exec/exchange_source_operator.cpp +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -45,15 +45,37 @@ ExchangeLocalState::ExchangeLocalState(RuntimeState* state, OperatorXBase* paren Status ExchangeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); + SCOPED_TIMER(profile()->total_time_counter()); auto& p = _parent->cast(); stream_recvr = state->exec_env()->vstream_mgr()->create_recvr( state, p.input_row_desc(), state->fragment_instance_id(), p.id(), p.num_senders(), profile(), p.is_merging(), p.sub_plan_query_statistics_recvr()); + source_dependency = AndDependency::create_shared(_parent->id()); + const auto& queues = stream_recvr->sender_queues(); + deps.resize(queues.size()); + metrics.resize(queues.size()); + for (size_t i = 0; i < queues.size(); i++) { + deps[i] = ExchangeDataDependency::create_shared(_parent->id(), queues[i]); + queues[i]->set_dependency(deps[i]); + source_dependency->add_child(deps[i]); + } + for (size_t i = 0; i < queues.size(); i++) { + static const std::string timer_name = + "WaitForDependency[" + source_dependency->name() + "]Time"; + _wait_for_dependency_timer = ADD_TIMER(_runtime_profile, timer_name); + metrics[i] = ADD_CHILD_TIMER(_runtime_profile, "WaitForData", timer_name); + } RETURN_IF_ERROR(_parent->cast()._vsort_exec_exprs.clone( state, vsort_exec_exprs)); return Status::OK(); } +Status ExchangeLocalState::open(RuntimeState* state) { + SCOPED_TIMER(profile()->total_time_counter()); + RETURN_IF_ERROR(PipelineXLocalState<>::open(state)); + return Status::OK(); +} + ExchangeSourceOperatorX::ExchangeSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, int num_senders) : OperatorX(pool, tnode, descs), @@ -141,8 +163,10 @@ Status ExchangeSourceOperatorX::get_block(RuntimeState* state, vectorized::Block return status; } -bool ExchangeSourceOperatorX::can_read(RuntimeState* state) { - return state->get_local_state(id())->cast().stream_recvr->ready_to_read(); +Dependency* ExchangeSourceOperatorX::wait_for_dependency(RuntimeState* state) { + return state->get_local_state(id()) + ->cast() + .source_dependency->read_blocked_by(); } bool ExchangeSourceOperatorX::is_pending_finish(RuntimeState* /*state*/) const { @@ -150,9 +174,14 @@ bool ExchangeSourceOperatorX::is_pending_finish(RuntimeState* /*state*/) const { } Status ExchangeLocalState::close(RuntimeState* state) { + SCOPED_TIMER(profile()->total_time_counter()); if (_closed) { return Status::OK(); } + 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()); + } 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 a95cef8140..3c5da8c68d 100644 --- a/be/src/pipeline/exec/exchange_source_operator.h +++ b/be/src/pipeline/exec/exchange_source_operator.h @@ -50,25 +50,64 @@ public: bool is_pending_finish() const override; }; +struct ExchangeDataDependency : public Dependency { +public: + ENABLE_FACTORY_CREATOR(ExchangeDataDependency); + ExchangeDataDependency(int id, vectorized::VDataStreamRecvr::SenderQueue* sender_queue) + : Dependency(id, "DataDependency"), _sender_queue(sender_queue), _always_done(false) {} + void* shared_state() override { return nullptr; } + [[nodiscard]] Dependency* read_blocked_by() override { + return _sender_queue->should_wait() ? this : nullptr; + } + + void set_always_done() { + _always_done = true; + if (_ready_for_read) { + return; + } + _read_dependency_watcher.stop(); + _ready_for_read = true; + } + + void set_ready_for_read() override { + if (_always_done || !_ready_for_read) { + return; + } + _ready_for_read = false; + // ScannerContext is set done outside this function now and only stop watcher here. + _read_dependency_watcher.start(); + } + +private: + vectorized::VDataStreamRecvr::SenderQueue* _sender_queue; + std::atomic _always_done; +}; + class ExchangeSourceOperatorX; class ExchangeLocalState : public PipelineXLocalState<> { ENABLE_FACTORY_CREATOR(ExchangeLocalState); ExchangeLocalState(RuntimeState* state, OperatorXBase* parent); Status init(RuntimeState* state, LocalStateInfo& info) override; + Status open(RuntimeState* state) override; Status close(RuntimeState* state) override; std::shared_ptr stream_recvr; doris::vectorized::VSortExecExprs vsort_exec_exprs; int64_t num_rows_skipped; bool is_ready; + + std::shared_ptr source_dependency; + std::vector> deps; + + std::vector metrics; }; class ExchangeSourceOperatorX final : public OperatorX { public: ExchangeSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs, int num_senders); - bool can_read(RuntimeState* state) override; + Dependency* wait_for_dependency(RuntimeState* state) override; bool is_pending_finish(RuntimeState* state) const override; Status init(const TPlanNode& tnode, RuntimeState* state) override; @@ -105,4 +144,4 @@ private: std::vector _nulls_first; }; -} // namespace doris::pipeline \ No newline at end of file +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 94ab51e067..4aa92cf0b3 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -585,7 +585,7 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* local_state.init_short_circuit_for_probe(); if (source_state == SourceState::FINISHED) { - local_state._dependency->set_done(); + local_state._dependency->set_ready_for_read(); } return Status::OK(); diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index cdf2d0e3ca..be888307be 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -436,6 +436,7 @@ Status HashJoinProbeOperatorX::prepare(RuntimeState* state) { RETURN_IF_ERROR(conjunct->prepare(state, *_intermediate_row_desc)); } RETURN_IF_ERROR(vectorized::VExpr::prepare(_probe_expr_ctxs, state, _child_x->row_desc())); + DCHECK(_build_side_child != nullptr); // right table data types _right_table_data_types = vectorized::VectorizedUtils::get_data_types(_build_side_child->row_desc()); @@ -453,9 +454,9 @@ Status HashJoinProbeOperatorX::open(RuntimeState* state) { return Status::OK(); } -bool HashJoinProbeOperatorX::can_read(RuntimeState* state) { +Dependency* HashJoinProbeOperatorX::wait_for_dependency(RuntimeState* state) { auto& local_state = state->get_local_state(id())->cast(); - return local_state._dependency->done(); + return local_state._dependency->read_blocked_by(); } } // namespace pipeline diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index cfca146617..bac12a004a 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -109,7 +109,7 @@ public: Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - bool can_read(RuntimeState* state) override; + Dependency* wait_for_dependency(RuntimeState* state) override; Status push(RuntimeState* state, vectorized::Block* input_block, SourceState source_state) const override; 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 5bd17ae3b1..103ae8a943 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp @@ -113,7 +113,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_done(); + local_state._dependency->set_ready_for_read(); } return Status::OK(); 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 43d684effb..050d5ae0b8 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp @@ -566,9 +566,9 @@ Status NestedLoopJoinProbeOperatorX::pull(RuntimeState* state, vectorized::Block return Status::OK(); } -bool NestedLoopJoinProbeOperatorX::can_read(RuntimeState* state) { +Dependency* NestedLoopJoinProbeOperatorX::wait_for_dependency(RuntimeState* state) { auto& local_state = state->get_local_state(id())->cast(); - return local_state._dependency->done(); + return local_state._dependency->read_blocked_by(); } } // namespace doris::pipeline 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 17ce4b4de8..e754ae585f 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.h @@ -208,7 +208,7 @@ public: Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override; Status open(RuntimeState* state) override; - bool can_read(RuntimeState* state) override; + Dependency* wait_for_dependency(RuntimeState* state) override; Status push(RuntimeState* state, vectorized::Block* input_block, SourceState source_state) const override; diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index 37df7e5ed6..49af73335f 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -133,7 +133,7 @@ Status OlapScanLocalState::_init_profile() { Status OlapScanLocalState::_process_conjuncts() { SCOPED_TIMER(_process_conjunct_timer); RETURN_IF_ERROR(ScanLocalState::_process_conjuncts()); - if (_eos) { + if (ScanLocalState::_eos_dependency->read_blocked_by() == nullptr) { return Status::OK(); } RETURN_IF_ERROR(_build_key_ranges_and_filters()); @@ -213,7 +213,7 @@ bool OlapScanLocalState::_storage_no_merge() { Status OlapScanLocalState::_init_scanners(std::list* scanners) { if (_scan_ranges.empty()) { - _eos = true; + ScanLocalState::_eos_dependency->set_ready_for_read(); return Status::OK(); } SCOPED_TIMER(_scanner_init_timer); @@ -402,7 +402,9 @@ Status OlapScanLocalState::_build_key_ranges_and_filters() { }, iter->second)); } - _eos |= eos; + if (eos) { + ScanLocalState::_eos_dependency->set_ready_for_read(); + } for (auto& iter : _colname_to_value_range) { std::vector filters; diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp index e382477d78..d2220febd4 100644 --- a/be/src/pipeline/exec/repeat_operator.cpp +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -68,8 +68,6 @@ Status RepeatOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { Status RepeatOperatorX::prepare(RuntimeState* state) { VLOG_CRITICAL << "VRepeatNode::prepare"; - SCOPED_TIMER(_runtime_profile->total_time_counter()); - RETURN_IF_ERROR(OperatorXBase::prepare(state)); _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id); if (_output_tuple_desc == nullptr) { @@ -85,7 +83,6 @@ Status RepeatOperatorX::prepare(RuntimeState* state) { Status RepeatOperatorX::open(RuntimeState* state) { VLOG_CRITICAL << "VRepeatNode::open"; - SCOPED_TIMER(_runtime_profile->total_time_counter()); RETURN_IF_ERROR(OperatorXBase::open(state)); RETURN_IF_ERROR(vectorized::VExpr::open(_expr_ctxs, state)); return Status::OK(); diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index c8c02a77fd..5cab5beded 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -99,8 +99,6 @@ Status ResultSinkOperatorX::prepare(RuntimeState* state) { auto fragment_instance_id = state->fragment_instance_id(); auto title = fmt::format("VDataBufferSender (dst_fragment_instance_id={:x}-{:x})", fragment_instance_id.hi, fragment_instance_id.lo); - // create profile - _profile = state->obj_pool()->add(new RuntimeProfile(title)); // prepare output_expr // From the thrift expressions create the real exprs. RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_t_output_expr, _output_vexpr_ctxs)); diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index 9fc033da19..cb4675bcfb 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -113,7 +113,16 @@ bool ScanLocalState::should_run_serial() const { template Status ScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); + SCOPED_TIMER(profile()->total_time_counter()); RETURN_IF_ERROR(RuntimeFilterConsumer::init(state)); + + _source_dependency = OrDependency::create_shared(PipelineXLocalState<>::_parent->id()); + + _open_dependency = OpenDependency::create_shared(PipelineXLocalState<>::_parent->id()); + _source_dependency->add_child(_open_dependency); + _eos_dependency = EosDependency::create_shared(PipelineXLocalState<>::_parent->id()); + _source_dependency->add_child(_eos_dependency); + auto& p = _parent->cast(); set_scan_ranges(info.scan_ranges); _common_expr_ctxs_push_down.resize(p._common_expr_ctxs_push_down.size()); @@ -141,24 +150,36 @@ Status ScanLocalState::init(RuntimeState* state, LocalStateInfo& info) _open_timer = ADD_TIMER(_runtime_profile, "OpenTime"); _alloc_resource_timer = ADD_TIMER(_runtime_profile, "AllocateResourceTime"); + + static const std::string timer_name = + "WaitForDependency[" + _source_dependency->name() + "]Time"; + _wait_for_dependency_timer = ADD_TIMER(_runtime_profile, timer_name); + _wait_for_data_timer = ADD_CHILD_TIMER(_runtime_profile, "WaitForData", timer_name); + _wait_for_scanner_done_timer = + ADD_CHILD_TIMER(_runtime_profile, "WaitForScannerDone", timer_name); + _wait_for_eos_timer = ADD_CHILD_TIMER(_runtime_profile, "WaitForEos", timer_name); return Status::OK(); } template Status ScanLocalState::open(RuntimeState* state) { - if (_opened) { + SCOPED_TIMER(profile()->total_time_counter()); + if (_open_dependency == nullptr) { return Status::OK(); } RETURN_IF_ERROR(_acquire_runtime_filter()); RETURN_IF_ERROR(_process_conjuncts()); - auto status = !_eos ? _prepare_scanners(state->query_parallel_instance_num()) : Status::OK(); + auto status = _eos_dependency->read_blocked_by() == nullptr + ? Status::OK() + : _prepare_scanners(state->query_parallel_instance_num()); if (_scanner_ctx) { - DCHECK(!_eos && _num_scanners->value() > 0); + DCHECK(_eos_dependency->read_blocked_by() != nullptr && _num_scanners->value() > 0); RETURN_IF_ERROR(_scanner_ctx->init()); RETURN_IF_ERROR(state->exec_env()->scanner_scheduler()->submit(_scanner_ctx.get())); } - _opened = true; + _source_dependency->remove_first_child(); + _open_dependency = nullptr; return status; } @@ -240,7 +261,7 @@ Status ScanLocalState::_normalize_conjuncts() { std::visit( [&](auto&& range) { if (range.is_empty_value_range()) { - _eos = true; + _eos_dependency->set_ready_for_read(); } }, it.second.second); @@ -538,7 +559,7 @@ Status ScanLocalState::_eval_const_conjuncts(vectorized::VExpr* vexpr, constant_val = const_cast(const_column->get_data_at(0).data); if (constant_val == nullptr || !*reinterpret_cast(constant_val)) { *pdt = vectorized::VScanNode::PushDownType::ACCEPTABLE; - _eos = true; + _eos_dependency->set_ready_for_read(); } } else if (const vectorized::ColumnVector* bool_column = check_and_get_column>( @@ -555,7 +576,7 @@ Status ScanLocalState::_eval_const_conjuncts(vectorized::VExpr* vexpr, constant_val = const_cast(bool_column->get_data_at(0).data); if (constant_val == nullptr || !*reinterpret_cast(constant_val)) { *pdt = vectorized::VScanNode::PushDownType::ACCEPTABLE; - _eos = true; + _eos_dependency->set_ready_for_read(); } } else { LOG(WARNING) << "Constant predicate in scan node should return a bool column with " @@ -752,7 +773,7 @@ Status ScanLocalState::_normalize_not_in_and_not_eq_predicate( HybridSetBase::IteratorBase* iter = state->hybrid_set->begin(); auto fn_name = std::string(""); if (!is_fixed_range && state->null_in_set) { - _eos = true; + _eos_dependency->set_ready_for_read(); } while (iter->has_next()) { // column not in (nullptr) is always true @@ -1144,7 +1165,7 @@ Status ScanLocalState::_prepare_scanners(const int query_parallel_insta std::list scanners; RETURN_IF_ERROR(_init_scanners(&scanners)); if (scanners.empty()) { - _eos = true; + _eos_dependency->set_ready_for_read(); } else { COUNTER_SET(_num_scanners, static_cast(scanners.size())); RETURN_IF_ERROR(_start_scanners(scanners, query_parallel_instance_num)); @@ -1159,6 +1180,12 @@ Status ScanLocalState::_start_scanners(const std::listscan_queue_mem_limit(), p._col_distribute_ids, 1); + _scanner_done_dependency = ScannerDoneDependency::create_shared(p.id(), _scanner_ctx.get()); + _source_dependency->add_child(_scanner_done_dependency); + _data_ready_dependency = DataReadyDependency::create_shared(p.id(), _scanner_ctx.get()); + _source_dependency->add_child(_data_ready_dependency); + + _scanner_ctx->set_dependency(_data_ready_dependency, _scanner_done_dependency); return Status::OK(); } @@ -1246,24 +1273,10 @@ ScanOperatorX::ScanOperatorX(ObjectPool* pool, const TPlanNode& } template -bool ScanOperatorX::can_read(RuntimeState* state) { - auto& local_state = state->get_local_state(id())->template cast(); - if (!local_state._opened) { - return true; - } else { - if (local_state._eos || local_state._scanner_ctx->done()) { - // _eos: need eos - // _scanner_ctx->done(): need finish - // _scanner_ctx->no_schedule(): should schedule _scanner_ctx - return true; - } else { - if (local_state._scanner_ctx->get_num_running_scanners() == 0 && - local_state._scanner_ctx->has_enough_space_in_blocks_queue()) { - local_state._scanner_ctx->reschedule_scanner_ctx(); - } - return local_state.ready_to_read(); // there are some blocks to process - } - } +Dependency* ScanOperatorX::wait_for_dependency(RuntimeState* state) { + return state->get_local_state(id()) + ->template cast() + ._source_dependency->read_blocked_by(); } template @@ -1325,12 +1338,23 @@ Status ScanOperatorX::try_close(RuntimeState* state) { template Status ScanLocalState::close(RuntimeState* state) { + SCOPED_TIMER(profile()->total_time_counter()); if (_closed) { return Status::OK(); } if (_scanner_ctx.get()) { _scanner_ctx->clear_and_join(reinterpret_cast(this), state); } + if (_data_ready_dependency) { + COUNTER_SET(_wait_for_data_timer, _data_ready_dependency->read_watcher_elapse_time()); + } + if (_eos_dependency) { + COUNTER_SET(_wait_for_eos_timer, _eos_dependency->read_watcher_elapse_time()); + } + if (_scanner_done_dependency) { + COUNTER_SET(_wait_for_scanner_done_timer, + _scanner_done_dependency->read_watcher_elapse_time()); + } return PipelineXLocalState<>::close(state); } @@ -1372,7 +1396,7 @@ Status ScanOperatorX::get_block(RuntimeState* state, vectorized: } } - if (local_state._eos) { + if (local_state._eos_dependency->read_blocked_by() == nullptr) { source_state = SourceState::FINISHED; return Status::OK(); } diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index a04a5ca60f..1c2aaac332 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -55,6 +55,60 @@ public: Status try_close(RuntimeState* state) override; }; +struct OpenDependency : public Dependency { +public: + ENABLE_FACTORY_CREATOR(OpenDependency); + OpenDependency(int id) : Dependency(id, "OpenDependency") {} + void* shared_state() override { return nullptr; } + [[nodiscard]] Dependency* read_blocked_by() override { return nullptr; } + [[nodiscard]] int64_t read_watcher_elapse_time() override { return 0; } +}; + +struct EosDependency : public Dependency { +public: + ENABLE_FACTORY_CREATOR(EosDependency); + EosDependency(int id) : Dependency(id, "EosDependency") {} + void* shared_state() override { return nullptr; } +}; + +struct ScannerDoneDependency : public Dependency { +public: + ENABLE_FACTORY_CREATOR(ScannerDoneDependency); + ScannerDoneDependency(int id, vectorized::ScannerContext* scanner_ctx) + : Dependency(id, "ScannerDoneDependency"), _scanner_ctx(scanner_ctx) {} + void* shared_state() override { return nullptr; } + [[nodiscard]] Dependency* read_blocked_by() override { + return _scanner_ctx->done() ? nullptr : this; + } + void set_ready_for_read() override { + // ScannerContext is set done outside this function now and only stop watcher here. + _read_dependency_watcher.stop(); + } + +private: + vectorized::ScannerContext* _scanner_ctx; +}; + +struct DataReadyDependency : public Dependency { +public: + ENABLE_FACTORY_CREATOR(DataReadyDependency); + DataReadyDependency(int id, vectorized::ScannerContext* scanner_ctx) + : Dependency(id, "DataReadyDependency"), _scanner_ctx(scanner_ctx) {} + + void* shared_state() override { return nullptr; } + + [[nodiscard]] Dependency* read_blocked_by() override { + if (_scanner_ctx->get_num_running_scanners() == 0 && + _scanner_ctx->has_enough_space_in_blocks_queue()) { + _scanner_ctx->reschedule_scanner_ctx(); + } + return _ready_for_read ? nullptr : this; + } + +private: + vectorized::ScannerContext* _scanner_ctx; +}; + class ScanLocalStateBase : public PipelineXLocalState<>, public vectorized::RuntimeFilterConsumer { public: ScanLocalStateBase(RuntimeState* state, OperatorXBase* parent) @@ -89,6 +143,12 @@ protected: virtual Status _init_profile() = 0; + std::shared_ptr _open_dependency; + std::shared_ptr _eos_dependency; + std::shared_ptr _source_dependency; + std::shared_ptr _scanner_done_dependency; + std::shared_ptr _data_ready_dependency; + std::shared_ptr _scanner_profile; RuntimeProfile::Counter* _scanner_sched_counter = nullptr; RuntimeProfile::Counter* _scanner_ctx_sched_counter = nullptr; @@ -117,6 +177,11 @@ protected: // Wall based aggregate read throughput [rows/sec] RuntimeProfile::Counter* _total_throughput_counter; RuntimeProfile::Counter* _num_scanners; + + RuntimeProfile::Counter* _wait_for_data_timer = nullptr; + RuntimeProfile::Counter* _wait_for_scanner_done_timer = nullptr; + // time of prefilter input block from scanner + RuntimeProfile::Counter* _wait_for_eos_timer = nullptr; }; template @@ -295,9 +360,6 @@ protected: std::shared_ptr _scanner_ctx; - // indicate this scan node has no more data to return - bool _eos = false; - vectorized::FilterPredicates _filter_predicates {}; // Save all function predicates which may be pushed down to data source. @@ -338,8 +400,6 @@ protected: RuntimeProfile::Counter* _acquire_runtime_filter_timer = nullptr; doris::Mutex _block_lock; - - std::atomic _opened = false; }; template @@ -349,7 +409,7 @@ public: Status try_close(RuntimeState* state) override; - bool can_read(RuntimeState* state) override; + Dependency* wait_for_dependency(RuntimeState* state) override; bool is_pending_finish(RuntimeState* state) const override; Status init(const TPlanNode& tnode, RuntimeState* state) override; @@ -403,4 +463,4 @@ protected: TPushAggOp::type _push_down_agg_type; }; -} // namespace doris::pipeline \ No newline at end of file +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index 688c0ba072..e96bff55ad 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -133,7 +133,6 @@ Status SortSinkOperatorX::prepare(RuntimeState* state) { } else { _algorithm = SortAlgorithm::FULL_SORT; } - _profile = state->obj_pool()->add(new RuntimeProfile("SortSinkOperatorX")); return _vsort_exec_exprs.prepare(state, _child_x->row_desc(), _row_descriptor); } @@ -169,7 +168,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_done(); + local_state._dependency->set_ready_for_read(); } return Status::OK(); } diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp index 11fbb76864..89929ea351 100644 --- a/be/src/pipeline/exec/sort_source_operator.cpp +++ b/be/src/pipeline/exec/sort_source_operator.cpp @@ -47,16 +47,14 @@ Status SortSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* bl local_state._shared_state->sorter->get_next(state, block, &eos)); local_state.reached_limit(block, &eos); if (eos) { - _runtime_profile->add_info_string( - "Spilled", local_state._shared_state->sorter->is_spilled() ? "true" : "false"); source_state = SourceState::FINISHED; } return Status::OK(); } -bool SortSourceOperatorX::can_read(RuntimeState* state) { +Dependency* SortSourceOperatorX::wait_for_dependency(RuntimeState* state) { auto& local_state = state->get_local_state(id())->cast(); - return local_state._dependency->done(); + return local_state._dependency->read_blocked_by(); } Status SortLocalState::close(RuntimeState* state) { diff --git a/be/src/pipeline/exec/sort_source_operator.h b/be/src/pipeline/exec/sort_source_operator.h index 9937343a16..b58a123e44 100644 --- a/be/src/pipeline/exec/sort_source_operator.h +++ b/be/src/pipeline/exec/sort_source_operator.h @@ -64,7 +64,7 @@ private: class SortSourceOperatorX final : public OperatorX { public: SortSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - bool can_read(RuntimeState* state) override; + Dependency* wait_for_dependency(RuntimeState* state) override; Status get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) override; diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp index 6f71c36c41..e6915dd4e6 100644 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp @@ -137,13 +137,13 @@ static constexpr int STREAMING_HT_MIN_REDUCTION_SIZE = StreamingAggSinkLocalState::StreamingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : AggSinkLocalState(parent, state), + : Base(parent, state), _queue_byte_size_counter(nullptr), _queue_size_counter(nullptr), _streaming_agg_timer(nullptr) {} Status StreamingAggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(AggSinkLocalState::init(state, info)); + RETURN_IF_ERROR(Base::init(state, info)); _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"); @@ -351,7 +351,7 @@ Status StreamingAggSinkLocalState::_pre_agg_with_serialized_key( StreamingAggSinkOperatorX::StreamingAggSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : AggSinkOperatorX(pool, tnode, descs) {} + : AggSinkOperatorX(pool, tnode, descs) {} bool StreamingAggSinkOperatorX::can_write(RuntimeState* state) { // sink and source in diff threads @@ -360,9 +360,16 @@ bool StreamingAggSinkOperatorX::can_write(RuntimeState* state) { ._shared_state->data_queue->has_enough_space_to_push(); } +Status StreamingAggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(AggSinkOperatorX::init(tnode, state)); + _name = "STREAMING_AGGREGATION_SINK_OPERATOR"; + return Status::OK(); +} + Status StreamingAggSinkOperatorX::sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) { auto& local_state = state->get_sink_local_state(id())->cast(); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); local_state._shared_state->input_num_rows += in_block->rows(); Status ret = Status::OK(); if (in_block && in_block->rows() > 0) { @@ -394,7 +401,7 @@ Status StreamingAggSinkLocalState::close(RuntimeState* state) { COUNTER_SET(_queue_byte_size_counter, _shared_state->data_queue->max_bytes_in_queue()); } _preagg_block.clear(); - return AggSinkLocalState::close(state); + return Base::close(state); } } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h index d3c5e1e141..5b7f3073ff 100644 --- a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h @@ -73,9 +73,11 @@ private: class StreamingAggSinkOperatorX; -class StreamingAggSinkLocalState final : public AggSinkLocalState { +class StreamingAggSinkLocalState final + : public AggSinkLocalState { public: using Parent = StreamingAggSinkOperatorX; + using Base = AggSinkLocalState; ENABLE_FACTORY_CREATOR(StreamingAggSinkLocalState); StreamingAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); @@ -106,6 +108,7 @@ private: class StreamingAggSinkOperatorX final : public AggSinkOperatorX { public: StreamingAggSinkOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); + Status init(const TPlanNode& tnode, RuntimeState* state) override; Status sink(RuntimeState* state, vectorized::Block* in_block, SourceState source_state) override; diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp index 6aac1130e1..7b1544232d 100644 --- a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp @@ -32,6 +32,7 @@ class ExecNode; class RuntimeState; namespace pipeline { + StreamingAggSourceOperator::StreamingAggSourceOperator(OperatorBuilderBase* templ, ExecNode* node, std::shared_ptr queue) : SourceOperator(templ, node), _data_queue(std::move(queue)) {} @@ -74,33 +75,45 @@ OperatorPtr StreamingAggSourceOperatorBuilder::build_operator() { StreamingAggSourceOperatorX::StreamingAggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) - : AggSourceOperatorX(pool, tnode, descs) {} + : Base(pool, tnode, descs), + _needs_finalize(tnode.agg_node.need_finalize), + _without_key(tnode.agg_node.grouping_exprs.empty()) {} Status StreamingAggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) { - auto& local_state = state->get_local_state(id())->cast(); + auto& local_state = state->get_local_state(id())->cast(); + SCOPED_TIMER(local_state.profile()->total_time_counter()); if (!local_state._shared_state->data_queue->data_exhausted()) { std::unique_ptr agg_block; + DCHECK(local_state._dependency->read_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()) { - RETURN_IF_ERROR(AggSourceOperatorX::get_block(state, block, source_state)); - } else { + if (!local_state._shared_state->data_queue->data_exhausted()) { block->swap(*agg_block); agg_block->clear_column_data(row_desc().num_materialized_slots()); local_state._shared_state->data_queue->push_free_block(std::move(agg_block)); + return Status::OK(); } - } else { - RETURN_IF_ERROR(AggSourceOperatorX::get_block(state, block, source_state)); } + RETURN_IF_ERROR(local_state._executor.get_result(state, block, source_state)); + local_state.make_nullable_output_key(block); + // dispose the having clause, should not be execute in prestreaming agg + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, block, block->columns())); + local_state.reached_limit(block, source_state); return Status::OK(); } -bool StreamingAggSourceOperatorX::can_read(RuntimeState* state) { +Status StreamingAggSourceOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(Base::init(tnode, state)); + _op_name = "STREAMING_AGGREGATION_OPERATOR"; + return Status::OK(); +} + +Dependency* StreamingAggSourceOperatorX::wait_for_dependency(RuntimeState* state) { return state->get_local_state(id()) - ->cast() - ._shared_state->data_queue->has_data_or_finished(); + ->cast() + ._dependency->read_blocked_by(); } } // namespace pipeline diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.h b/be/src/pipeline/exec/streaming_aggregation_source_operator.h index 17c4058e59..41f99164d3 100644 --- a/be/src/pipeline/exec/streaming_aggregation_source_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.h @@ -60,15 +60,44 @@ private: std::shared_ptr _data_queue; }; -class StreamingAggSourceOperatorX final : public AggSourceOperatorX { +class StreamingAggSourceOperatorX; + +class StreamingAggLocalState final + : public AggLocalState { public: + using Parent = StreamingAggSourceOperatorX; + ENABLE_FACTORY_CREATOR(StreamingAggLocalState); + StreamingAggLocalState(RuntimeState* state, OperatorXBase* parent) + : AggLocalState(state, parent) {} + ~StreamingAggLocalState() = default; +}; + +class StreamingAggSourceOperatorX final : public OperatorX { +public: + using Base = OperatorX; StreamingAggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs); - bool can_read(RuntimeState* state) override; + ~StreamingAggSourceOperatorX() = default; + + Status init(const TPlanNode& tnode, RuntimeState* state) override; + + Dependency* wait_for_dependency(RuntimeState* state) override; Status get_block(RuntimeState* state, vectorized::Block* block, SourceState& source_state) override; + + bool is_source() const override { return true; } + +private: + template + friend class AggLocalState; + + bool _needs_finalize; + bool _without_key; + // left / full join will change the key nullable make output/input solt + // nullable diff. so we need make nullable of it. + std::vector _make_nullable_keys; }; } // namespace pipeline -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index c1fdae4851..815972e5e0 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -101,7 +101,8 @@ Status UnionSourceOperator::get_block(RuntimeState* state, vectorized::Block* bl Status UnionSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); auto& p = _parent->cast(); - std::shared_ptr data_queue = std::make_shared(p._child_size); + std::shared_ptr data_queue = + std::make_shared(p._child_size, nullptr, _dependency); _shared_state->_data_queue.swap(data_queue); return Status::OK(); } diff --git a/be/src/pipeline/exec/union_source_operator.h b/be/src/pipeline/exec/union_source_operator.h index 580a7d4a15..77e88571c5 100644 --- a/be/src/pipeline/exec/union_source_operator.h +++ b/be/src/pipeline/exec/union_source_operator.h @@ -68,6 +68,7 @@ private: std::shared_ptr _data_queue; bool _need_read_for_const_expr; }; + class UnionSourceOperatorX; class UnionSourceLocalState final : public PipelineXLocalState { public: @@ -87,9 +88,9 @@ public: UnionSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) : Base(pool, tnode, descs), _child_size(tnode.num_children) {}; ~UnionSourceOperatorX() override = default; - bool can_read(RuntimeState* state) override { + Dependency* wait_for_dependency(RuntimeState* state) override { auto& local_state = state->get_local_state(id())->cast(); - return local_state._shared_state->_data_queue->is_all_finish(); + return local_state._dependency->read_blocked_by(); } Status get_block(RuntimeState* state, vectorized::Block* block, diff --git a/be/src/pipeline/pipeline.h b/be/src/pipeline/pipeline.h index 3c8c00098b..b29aaeb00b 100644 --- a/be/src/pipeline/pipeline.h +++ b/be/src/pipeline/pipeline.h @@ -170,4 +170,4 @@ private: bool _always_can_write = false; }; -} // namespace doris::pipeline \ No newline at end of file +} // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index c66f782723..9c2bfa6737 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -850,11 +850,18 @@ void PipelineFragmentContext::send_report(bool done) { } _report_status_cb( - {exec_status, + {false, + exec_status, + {}, _runtime_state->enable_profile() ? _runtime_state->runtime_profile() : nullptr, _runtime_state->enable_profile() ? _runtime_state->load_channel_profile() : nullptr, - done || !exec_status.ok(), _query_ctx->coord_addr, _query_id, _fragment_id, - _fragment_instance_id, _backend_num, _runtime_state.get(), + done || !exec_status.ok(), + _query_ctx->coord_addr, + _query_id, + _fragment_id, + _fragment_instance_id, + _backend_num, + _runtime_state.get(), std::bind(&PipelineFragmentContext::update_status, this, std::placeholders::_1), std::bind(&PipelineFragmentContext::cancel, this, std::placeholders::_1, std::placeholders::_2)}); diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index 6fa5bcac24..b8b8e89215 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -236,7 +236,7 @@ public: } } - void set_close_pipeline_time() { + virtual void set_close_pipeline_time() { if (!_is_close_pipeline) { _close_pipeline_time = _pipeline_task_watcher.elapsed_time(); _is_close_pipeline = true; @@ -254,8 +254,8 @@ protected: } virtual Status _open(); - void _init_profile(); - void _fresh_profile_counter(); + virtual void _init_profile(); + virtual void _fresh_profile_counter(); uint32_t _index; PipelinePtr _pipeline; diff --git a/be/src/pipeline/pipeline_x/dependency.cpp b/be/src/pipeline/pipeline_x/dependency.cpp index f9a93d32fe..4ccc42a376 100644 --- a/be/src/pipeline/pipeline_x/dependency.cpp +++ b/be/src/pipeline/pipeline_x/dependency.cpp @@ -33,10 +33,35 @@ template Status HashJoinDependency::extract_join_column( std::vector>&, std::vector> const&); -std::string Dependency::debug_string(int indentation_level) const { +std::string Dependency::debug_string(int indentation_level) { fmt::memory_buffer debug_string_buffer; fmt::format_to(debug_string_buffer, "{}{}: id={}, done={}", - std::string(indentation_level * 2, ' '), _name, _id, _done.load()); + std::string(indentation_level * 2, ' '), _name, _id, + read_blocked_by() == nullptr); + return fmt::to_string(debug_string_buffer); +} + +std::string AndDependency::debug_string(int indentation_level) { + fmt::memory_buffer debug_string_buffer; + fmt::format_to(debug_string_buffer, "{}{}: id={}, done={}, children=[", + std::string(indentation_level * 2, ' '), _name, _id, + read_blocked_by() == nullptr); + for (auto& child : _children) { + fmt::format_to(debug_string_buffer, "{}, \n", child->debug_string(indentation_level = 1)); + } + fmt::format_to(debug_string_buffer, "{}]", std::string(indentation_level * 2, ' ')); + return fmt::to_string(debug_string_buffer); +} + +std::string OrDependency::debug_string(int indentation_level) { + fmt::memory_buffer debug_string_buffer; + fmt::format_to(debug_string_buffer, "{}{}: id={}, done={}, children=[", + std::string(indentation_level * 2, ' '), _name, _id, + read_blocked_by() == nullptr); + for (auto& child : _children) { + fmt::format_to(debug_string_buffer, "{}, \n", child->debug_string(indentation_level = 1)); + } + fmt::format_to(debug_string_buffer, "{}]", std::string(indentation_level * 2, ' ')); return fmt::to_string(debug_string_buffer); } @@ -218,7 +243,7 @@ vectorized::BlockRowPos AnalyticDependency::compare_row_to_find_end(int idx, return start; } -bool AnalyticDependency::whether_need_next_partition(vectorized::BlockRowPos found_partition_end) { +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 diff --git a/be/src/pipeline/pipeline_x/dependency.h b/be/src/pipeline/pipeline_x/dependency.h index 41c9299f01..cb9c8a5185 100644 --- a/be/src/pipeline/pipeline_x/dependency.h +++ b/be/src/pipeline/pipeline_x/dependency.h @@ -29,31 +29,206 @@ namespace pipeline { class Dependency; using DependencySPtr = std::shared_ptr; -class Dependency { +class Dependency : public std::enable_shared_from_this { public: - Dependency(int id, std::string name) : _id(id), _name(name), _done(false) {} + Dependency(int id, std::string name) : _id(id), _name(name), _ready_for_read(false) {} virtual ~Dependency() = default; - [[nodiscard]] bool done() const { return _done; } - void set_done() { _done = true; } - - virtual void* shared_state() = 0; [[nodiscard]] int id() const { return _id; } + [[nodiscard]] virtual std::string name() const { return _name; } + virtual void* shared_state() = 0; + virtual std::string debug_string(int indentation_level = 0); + virtual bool is_write_dependency() { return false; } - virtual std::string debug_string(int indentation_level = 0) const; + // Start the watcher. We use it to count how long this dependency block the current pipeline task. + void start_read_watcher() { + for (auto& child : _children) { + child->start_read_watcher(); + } + _read_dependency_watcher.start(); + } -private: + [[nodiscard]] virtual int64_t read_watcher_elapse_time() { + return _read_dependency_watcher.elapsed_time(); + } + + // Which dependency current pipeline task is blocked by. `nullptr` if this dependency is ready. + [[nodiscard]] virtual Dependency* read_blocked_by() { return _ready_for_read ? nullptr : this; } + + // Notify downstream pipeline tasks this dependency is ready. + virtual void set_ready_for_read() { + if (_ready_for_read) { + return; + } + _read_dependency_watcher.stop(); + _ready_for_read = true; + } + + // 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; } + + void add_child(std::shared_ptr child) { + std::unique_lock l(_lock); + _children.push_back(child); + child->set_parent(weak_from_this()); + } + + void remove_first_child() { + std::unique_lock l(_lock); + _children.erase(_children.begin()); + } + +protected: int _id; std::string _name; - std::atomic _done; + std::atomic _ready_for_read; + MonotonicStopWatch _read_dependency_watcher; + + std::weak_ptr _parent; + + std::list> _children; + std::mutex _lock; +}; + +class WriteDependency : public Dependency { +public: + WriteDependency(int id, std::string name) : Dependency(id, name), _ready_for_write(false) {} + virtual ~WriteDependency() = default; + + 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 Dependency* write_blocked_by() { + return _ready_for_write ? nullptr : this; + } + + virtual void set_ready_for_write() { + if (_ready_for_write) { + return; + } + _write_dependency_watcher.stop(); + _ready_for_write = true; + } + + virtual void block_writing() { _ready_for_write = false; } + +protected: + std::atomic _ready_for_write; + MonotonicStopWatch _write_dependency_watcher; +}; + +class AndDependency : public WriteDependency { +public: + ENABLE_FACTORY_CREATOR(AndDependency); + AndDependency(int id) : WriteDependency(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, "{}, ", child->name()); + } + fmt::format_to(debug_string_buffer, "]"); + return fmt::to_string(debug_string_buffer); + } + + void* shared_state() override { return nullptr; } + + std::string debug_string(int indentation_level = 0) override; + + [[nodiscard]] Dependency* read_blocked_by() override { + std::unique_lock l(_lock); + for (auto& child : _children) { + if (auto* dep = child->read_blocked_by()) { + return dep; + } + } + return nullptr; + } + + [[nodiscard]] Dependency* write_blocked_by() override { + std::unique_lock l(_lock); + for (auto& child : _children) { + CHECK(child->is_write_dependency()); + if (auto* dep = ((WriteDependency*)child.get())->write_blocked_by()) { + return dep; + } + } + return nullptr; + } +}; + +class OrDependency : public WriteDependency { +public: + ENABLE_FACTORY_CREATOR(OrDependency); + OrDependency(int id) : WriteDependency(id, "OrDependency") {} + + [[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, "{}, ", child->name()); + } + fmt::format_to(debug_string_buffer, "]"); + return fmt::to_string(debug_string_buffer); + } + + void* shared_state() override { return nullptr; } + + std::string debug_string(int indentation_level = 0) override; + + [[nodiscard]] Dependency* read_blocked_by() override { + Dependency* res = nullptr; + std::unique_lock l(_lock); + for (auto& child : _children) { + auto* cur_res = child->read_blocked_by(); + if (cur_res == nullptr) { + return nullptr; + } else { + res = cur_res; + } + } + return res; + } + + [[nodiscard]] Dependency* write_blocked_by() override { + Dependency* res = nullptr; + std::unique_lock l(_lock); + for (auto& child : _children) { + CHECK(child->is_write_dependency()); + auto* cur_res = ((WriteDependency*)child.get())->write_blocked_by(); + if (cur_res == nullptr) { + return nullptr; + } else { + res = cur_res; + } + } + return res; + } }; struct FakeSharedState {}; -struct FakeDependency : public Dependency { +struct FakeDependency : public WriteDependency { public: - FakeDependency(int id) : Dependency(0, "FakeDependency") {} + FakeDependency(int id) : WriteDependency(0, "FakeDependency") {} using SharedState = FakeSharedState; void* shared_state() 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 { @@ -61,8 +236,8 @@ public: AggSharedState() { agg_data = std::make_unique(); agg_arena_pool = std::make_unique(); - data_queue = std::make_unique(1); } + virtual ~AggSharedState() = default; void init_spill_partition_helper(size_t spill_partition_count_bits) { spill_partition_helper = std::make_unique(spill_partition_count_bits); @@ -79,10 +254,9 @@ public: size_t input_num_rows = 0; std::vector values; std::unique_ptr agg_profile_arena; - std::unique_ptr data_queue; }; -class AggDependency final : public Dependency { +class AggDependency : public Dependency { public: using SharedState = AggSharedState; AggDependency(int id) : Dependency(id, "AggDependency") { @@ -128,8 +302,7 @@ public: MemTracker* mem_tracker() { return _mem_tracker.get(); } -private: - AggSharedState _agg_state; +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; @@ -139,6 +312,29 @@ private: MemoryRecord _mem_usage_record; std::unique_ptr _mem_tracker; + +private: + AggSharedState _agg_state; +}; + +struct StreamingAggSharedState final : public AggSharedState { +public: + StreamingAggSharedState() : AggSharedState() {} + ~StreamingAggSharedState() = default; + std::unique_ptr data_queue; +}; + +class StreamingAggDependency final : public AggDependency { +public: + using SharedState = StreamingAggSharedState; + StreamingAggDependency(int id) : AggDependency(id) { + _streaming_agg_state.data_queue = std::make_unique(1, this); + } + + void* shared_state() override { return (void*)&_streaming_agg_state; } + +private: + StreamingAggSharedState _streaming_agg_state; }; struct SortSharedState { @@ -172,6 +368,7 @@ public: private: UnionSharedState _union_state; }; + struct AnalyticSharedState { public: AnalyticSharedState() = default; @@ -205,7 +402,18 @@ public: vectorized::BlockRowPos get_partition_by_end(); - bool whether_need_next_partition(vectorized::BlockRowPos found_partition_end); + bool refresh_need_more_input() { + _analytic_state.need_more_input = + whether_need_next_partition(_analytic_state.found_partition_end); + if (_analytic_state.need_more_input) { + block_reading(); + } else { + set_ready_for_read(); + } + return _analytic_state.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); diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index 328dd43cb0..b35d15cb7f 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -89,7 +89,6 @@ Status OperatorXBase::init(const TPlanNode& tnode, RuntimeState* /*state*/) { std::string node_name = print_plan_node_type(tnode.node_type); auto substr = node_name.substr(0, node_name.find("_NODE")); _op_name = substr + "_OPERATOR"; - _init_runtime_profile(); if (tnode.__isset.vconjunct) { vectorized::VExprContextSPtr context; @@ -136,13 +135,6 @@ Status OperatorXBase::open(RuntimeState* state) { return Status::OK(); } -void OperatorXBase::_init_runtime_profile() { - std::stringstream ss; - ss << get_name() << " (id=" << _id << ")"; - _runtime_profile.reset(new RuntimeProfile(ss.str())); - _runtime_profile->set_metadata(_id); -} - Status OperatorXBase::close(RuntimeState* state) { if (_child_x && !is_source()) { RETURN_IF_ERROR(_child_x->close(state)); @@ -270,7 +262,11 @@ Status DataSinkOperatorX::setup_local_state(RuntimeState* state, template void DataSinkOperatorX::get_dependency(DependencySPtr& dependency) { - dependency.reset(new typename LocalStateType::Dependency(dest_id())); + if constexpr (!std::is_same_v) { + dependency.reset(new typename LocalStateType::Dependency(dest_id())); + } else { + dependency.reset((typename LocalStateType::Dependency*)nullptr); + } } template @@ -285,6 +281,8 @@ Status StreamingOperatorX::get_block(RuntimeState* state, vector SourceState& source_state) { RETURN_IF_ERROR(OperatorX::_child_x->get_next_after_projects(state, block, source_state)); + COUNTER_UPDATE(state->get_local_state(OperatorX::id())->rows_input_counter(), + (int64_t)block->rows()); return pull(state, block, source_state); } @@ -297,6 +295,7 @@ Status StatefulOperatorX::get_block(RuntimeState* state, vectori local_state._child_block->clear_column_data(); RETURN_IF_ERROR(OperatorX::_child_x->get_next_after_projects( state, local_state._child_block.get(), local_state._child_source_state)); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)local_state._child_block->rows()); source_state = local_state._child_source_state; if (local_state._child_block->rows() == 0 && local_state._child_source_state != SourceState::FINISHED) { @@ -338,7 +337,8 @@ DECLARE_OPERATOR_X(HashJoinProbeLocalState) DECLARE_OPERATOR_X(OlapScanLocalState) DECLARE_OPERATOR_X(AnalyticLocalState) DECLARE_OPERATOR_X(SortLocalState) -DECLARE_OPERATOR_X(AggLocalState) +DECLARE_OPERATOR_X(BlockingAggLocalState) +DECLARE_OPERATOR_X(StreamingAggLocalState) DECLARE_OPERATOR_X(ExchangeLocalState) DECLARE_OPERATOR_X(RepeatLocalState) DECLARE_OPERATOR_X(NestedLoopJoinProbeLocalState) @@ -368,6 +368,7 @@ template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; +template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index e8a129795b..d6adfa7458 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -23,12 +23,14 @@ namespace doris::pipeline { // This struct is used only for initializing local state. struct LocalStateInfo { + RuntimeProfile* parent_profile; const std::vector scan_ranges; Dependency* dependency; }; // This struct is used only for initializing local sink state. struct LocalSinkStateInfo { + RuntimeProfile* parent_profile; const int sender_id; Dependency* dependency; }; @@ -81,6 +83,8 @@ public: RuntimeProfile::Counter* rows_returned_rate() { return _rows_returned_rate; } RuntimeProfile::Counter* memory_used_counter() { return _memory_used_counter; } RuntimeProfile::Counter* projection_timer() { return _projection_timer; } + RuntimeProfile::Counter* wait_for_dependency_timer() { return _wait_for_dependency_timer; } + RuntimeProfile::Counter* rows_input_counter() { return _rows_input_counter; } OperatorXBase* parent() { return _parent; } RuntimeState* state() { return _state; } @@ -105,7 +109,9 @@ protected: std::unique_ptr _mem_tracker; RuntimeProfile::Counter* _rows_returned_counter; + RuntimeProfile::Counter* _rows_input_counter; RuntimeProfile::Counter* _rows_returned_rate; + RuntimeProfile::Counter* _wait_for_dependency_timer; // Account for peak memory used by this node RuntimeProfile::Counter* _memory_used_counter; RuntimeProfile::Counter* _projection_timer; @@ -143,7 +149,9 @@ public: return Status::OK(); } [[nodiscard]] RuntimeProfile* get_runtime_profile() const override { - return _runtime_profile.get(); + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Runtime Profile is not owned by operator"); + return nullptr; } [[noreturn]] virtual const std::vector& runtime_filter_descs() { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, _op_name); @@ -189,7 +197,7 @@ public: virtual Status close(RuntimeState* state) override; - virtual bool can_read(RuntimeState* state) { return true; } + virtual Dependency* wait_for_dependency(RuntimeState* state) { return nullptr; } virtual bool is_pending_finish(RuntimeState* state) const { return false; } @@ -262,10 +270,6 @@ protected: const TBackendResourceProfile _resource_profile; int64_t _limit; // -1: no limit - std::unique_ptr _runtime_profile; - -private: - void _init_runtime_profile(); std::string _op_name; }; @@ -288,12 +292,19 @@ public: virtual ~PipelineXLocalState() {} virtual Status init(RuntimeState* state, LocalStateInfo& info) override { - _dependency = (DependencyType*)info.dependency; - if (_dependency) { - _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state(); + _runtime_profile.reset(new RuntimeProfile(_parent->get_name() + + " (id=" + std::to_string(_parent->id()) + ")")); + _runtime_profile->set_metadata(_parent->id()); + info.parent_profile->add_child(_runtime_profile.get(), true, nullptr); + if constexpr (!std::is_same_v) { + _dependency = (DependencyType*)info.dependency; + if (_dependency) { + _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state(); + _wait_for_dependency_timer = ADD_TIMER( + _runtime_profile, "WaitForDependency[" + _dependency->name() + "]Time"); + } } - _runtime_profile.reset(new RuntimeProfile("LocalState " + _parent->get_name())); - _parent->get_runtime_profile()->add_child(_runtime_profile.get(), true, nullptr); + _conjuncts.resize(_parent->_conjuncts.size()); _projections.resize(_parent->_projections.size()); for (size_t i = 0; i < _conjuncts.size(); i++) { @@ -304,6 +315,7 @@ public: } DCHECK(_runtime_profile.get() != nullptr); _rows_returned_counter = ADD_COUNTER(_runtime_profile, "RowsReturned", TUnit::UNIT); + _rows_input_counter = ADD_COUNTER(_runtime_profile, "InputRows", TUnit::UNIT); _projection_timer = ADD_TIMER(_runtime_profile, "ProjectionTime"); _rows_returned_rate = profile()->add_derived_counter( doris::ExecNode::ROW_THROUGHPUT_COUNTER, TUnit::UNIT_PER_SECOND, @@ -322,6 +334,9 @@ public: if (_closed) { return Status::OK(); } + if (_dependency) { + COUNTER_SET(_wait_for_dependency_timer, _dependency->read_watcher_elapse_time()); + } if (_rows_returned_counter != nullptr) { COUNTER_SET(_rows_returned_counter, _num_rows_returned); } @@ -351,7 +366,7 @@ public: // Do initialization. This step can be executed multiple times, so we should make sure it is // idempotent (e.g. wait for runtime filters). - virtual Status open(RuntimeState* state) { return Status::OK(); } + virtual Status open(RuntimeState* state) = 0; virtual Status close(RuntimeState* state) = 0; virtual std::string debug_string(int indentation_level) const; @@ -378,6 +393,8 @@ public: QueryStatistics* query_statistics() { return _query_statistics.get(); } RuntimeProfile* faker_runtime_profile() const { return _faker_runtime_profile.get(); } + RuntimeProfile::Counter* rows_input_counter() { return _rows_input_counter; } + protected: DataSinkOperatorXBase* _parent; RuntimeState* _state; @@ -394,6 +411,8 @@ protected: //so we could add those counter/timer in faker profile, and those will not display in web profile. std::unique_ptr _faker_runtime_profile = std::make_unique("faker profile"); + + RuntimeProfile::Counter* _rows_input_counter; }; class DataSinkOperatorXBase : public OperatorBase { @@ -458,13 +477,17 @@ public: virtual std::string debug_string(RuntimeState* state, int indentation_level) const; - bool is_sink() const override { return true; } + [[nodiscard]] bool is_sink() const override { return true; } - bool is_source() const override { return false; } + [[nodiscard]] bool is_source() const override { return false; } virtual Status close(RuntimeState* state, Status exec_status) { return Status::OK(); } - [[nodiscard]] RuntimeProfile* get_runtime_profile() const override { return _profile; } + [[nodiscard]] RuntimeProfile* get_runtime_profile() const override { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "Runtime Profile is not owned by operator"); + return nullptr; + } [[nodiscard]] int id() const override { return _id; } @@ -485,8 +508,6 @@ protected: std::shared_ptr _query_statistics; OpentelemetrySpan _span {}; - - RuntimeProfile* _profile = nullptr; }; template @@ -511,16 +532,23 @@ public: ~PipelineXSinkLocalState() override = default; virtual Status init(RuntimeState* state, LocalSinkStateInfo& info) override { - _dependency = (DependencyType*)info.dependency; - if (_dependency) { - _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state(); + if constexpr (!std::is_same_v) { + _dependency = (DependencyType*)info.dependency; + if (_dependency) { + _shared_state = (typename DependencyType::SharedState*)_dependency->shared_state(); + } } // create profile - _profile = state->obj_pool()->add(new RuntimeProfile(_parent->get_name())); + _profile = state->obj_pool()->add(new RuntimeProfile( + _parent->get_name() + " (id=" + std::to_string(_parent->id()) + ")")); + _rows_input_counter = ADD_COUNTER(_profile, "InputRows", TUnit::UNIT); + info.parent_profile->add_child(_profile, true, nullptr); _mem_tracker = std::make_unique(_parent->get_name()); return Status::OK(); } + virtual Status open(RuntimeState* state) override { return Status::OK(); } + Status close(RuntimeState* state) override { if (_closed) { return Status::OK(); diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index 03af3aabdd..8c55d004a2 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -210,21 +210,12 @@ Status PipelineXFragmentContext::prepare(const doris::TPipelineFragmentParams& r // 4. Initialize global states in pipelines. for (PipelinePtr& pipeline : _pipelines) { - for (auto& op : pipeline->operator_xs()) { - if (op->get_child() && !op->is_source()) { - op->get_runtime_profile()->add_child(op->get_child()->get_runtime_profile(), true, - nullptr); - } - } pipeline->sink_x()->set_child(pipeline->operator_xs().back()); RETURN_IF_ERROR(pipeline->prepare(_runtime_state.get())); } // 5. Build pipeline tasks and initialize local state. RETURN_IF_ERROR(_build_pipeline_tasks(request)); - _runtime_state->runtime_profile()->add_child(_root_op->get_runtime_profile(), true, nullptr); - _runtime_state->runtime_profile()->add_child(_sink->get_runtime_profile(), true, nullptr); - _runtime_state->runtime_profile()->add_child(_runtime_profile.get(), true, nullptr); _prepared = true; return Status::OK(); @@ -244,7 +235,7 @@ Status PipelineXFragmentContext::_create_data_sink(ObjectPool* pool, const TData params.__isset.send_query_statistics_with_every_batch ? params.send_query_statistics_with_every_batch : false; - _sink.reset(new ExchangeSinkOperatorX(state, pool, row_desc, thrift_sink.stream_sink, + _sink.reset(new ExchangeSinkOperatorX(state, row_desc, thrift_sink.stream_sink, params.destinations, send_query_statistics_with_every_batch)); break; @@ -270,7 +261,6 @@ Status PipelineXFragmentContext::_build_pipeline_tasks( int target_size = request.local_params.size(); _runtime_states.resize(target_size); _tasks.resize(target_size); - std::vector no_scan_ranges; for (size_t i = 0; i < target_size; i++) { const auto& local_params = request.local_params[i]; @@ -304,8 +294,8 @@ Status PipelineXFragmentContext::_build_pipeline_tasks( _runtime_states[i].get(), this, _pipelines[pip_idx]->pipeline_profile()); pipeline_id_to_task.insert({_pipelines[pip_idx]->id(), task.get()}); - RETURN_IF_ERROR(task->prepare(_runtime_states[i].get())); - _runtime_profile->add_child(_pipelines[pip_idx]->pipeline_profile(), true, nullptr); + _runtime_states[i]->runtime_profile()->add_child( + _pipelines[pip_idx]->pipeline_profile(), true, nullptr); _tasks[i].emplace_back(std::move(task)); } @@ -338,19 +328,7 @@ Status PipelineXFragmentContext::_build_pipeline_tasks( pipeline_id_to_task[dep]->get_downstream_dependency()); } } - - auto scan_ranges = find_with_default(local_params.per_node_scan_ranges, - _pipelines[pip_idx]->operator_xs().front()->id(), - no_scan_ranges); - for (auto& op : _pipelines[pip_idx]->operator_xs()) { - LocalStateInfo info {scan_ranges, task->get_upstream_dependency(op->id())}; - RETURN_IF_ERROR(op->setup_local_state(_runtime_states[i].get(), info)); - } - - LocalSinkStateInfo info {local_params.sender_id, - task->get_downstream_dependency().get()}; - RETURN_IF_ERROR(_pipelines[pip_idx]->sink_x()->setup_local_state( - _runtime_states[i].get(), info)); + RETURN_IF_ERROR(task->prepare(_runtime_states[i].get(), local_params)); } { @@ -363,10 +341,6 @@ Status PipelineXFragmentContext::_build_pipeline_tasks( _build_side_pipelines.clear(); _union_child_pipelines.clear(); _dag.clear(); - // register the profile of child data stream sender - // for (auto& sender : _multi_cast_stream_sink_senders) { - // _sink->profile()->add_child(sender->profile(), true, nullptr); - // } return Status::OK(); } @@ -551,7 +525,7 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN _dag[downstream_pipeline_id].push_back(cur_pipe->id()); DataSinkOperatorXPtr sink; - sink.reset(new AggSinkOperatorX(pool, tnode, descs)); + sink.reset(new AggSinkOperatorX<>(pool, tnode, descs)); RETURN_IF_ERROR(cur_pipe->set_sink(sink)); RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); } @@ -766,19 +740,20 @@ void PipelineXFragmentContext::send_report(bool done) { return; } - // TODO: only send rpc once - FOR_EACH_RUNTIME_STATE( - _report_status_cb( - {exec_status, - _runtime_state->enable_profile() ? _runtime_state->runtime_profile() : nullptr, - _runtime_state->enable_profile() ? runtime_state->load_channel_profile() - : nullptr, - done || !exec_status.ok(), _query_ctx->coord_addr, _query_id, _fragment_id, - runtime_state->fragment_instance_id(), _backend_num, runtime_state.get(), - std::bind(&PipelineFragmentContext::update_status, this, - std::placeholders::_1), - std::bind(&PipelineFragmentContext::cancel, this, std::placeholders::_1, - std::placeholders::_2)});) + std::vector runtime_states(_runtime_states.size()); + for (size_t i = 0; i < _runtime_states.size(); i++) { + runtime_states[i] = _runtime_states[i].get(); + } + + std::vector empty_vector(0); + + _report_status_cb( + {true, exec_status, _runtime_state->enable_profile() ? runtime_states : empty_vector, + nullptr, nullptr, done || !exec_status.ok(), _query_ctx->coord_addr, _query_id, + _fragment_id, TUniqueId(), _backend_num, _runtime_state.get(), + std::bind(&PipelineFragmentContext::update_status, this, std::placeholders::_1), + std::bind(&PipelineFragmentContext::cancel, this, std::placeholders::_1, + std::placeholders::_2)}); } } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp index ae22b95acb..a4f195c20c 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp @@ -54,30 +54,29 @@ PipelineXTask::PipelineXTask(PipelinePtr& pipeline, uint32_t index, RuntimeState _sink->get_dependency(_downstream_dependency); } -Status PipelineXTask::prepare(RuntimeState* state) { +Status PipelineXTask::prepare(RuntimeState* state, const TPipelineInstanceParams& local_params) { DCHECK(_sink); - DCHECK(_cur_state == PipelineTaskState::NOT_READY); + DCHECK(_cur_state == PipelineTaskState::NOT_READY) << get_state_name(_cur_state); _init_profile(); SCOPED_TIMER(_task_profile->total_time_counter()); SCOPED_CPU_TIMER(_task_cpu_timer); SCOPED_TIMER(_prepare_timer); - _task_profile->add_info_string("Sink", - fmt::format("{}(dst_id={})", _sink->get_name(), _sink->id())); - fmt::memory_buffer operator_ids_str; - for (size_t i = 0; i < _operators.size(); i++) { - if (i == 0) { - fmt::format_to( - operator_ids_str, - fmt::format("[{}(node_id={})", _operators[i]->get_name(), _operators[i]->id())); - } else { - fmt::format_to(operator_ids_str, - fmt::format(", {}(node_id={})", _operators[i]->get_name(), - _operators[i]->id())); - } + std::vector no_scan_ranges; + auto scan_ranges = find_with_default(local_params.per_node_scan_ranges, + _operators.front()->id(), no_scan_ranges); + for (int op_idx = _operators.size() - 1; op_idx >= 0; op_idx--) { + LocalStateInfo info { + op_idx == _operators.size() - 1 + ? _pipeline->pipeline_profile() + : state->get_local_state(_operators[op_idx + 1]->id())->profile(), + scan_ranges, get_upstream_dependency(_operators[op_idx]->id())}; + RETURN_IF_ERROR(_operators[op_idx]->setup_local_state(state, info)); } - fmt::format_to(operator_ids_str, "]"); - _task_profile->add_info_string("OperatorIds(source2root)", fmt::to_string(operator_ids_str)); + + LocalSinkStateInfo info {_pipeline->pipeline_profile(), local_params.sender_id, + get_downstream_dependency().get()}; + RETURN_IF_ERROR(_sink->setup_local_state(state, info)); _block = doris::vectorized::Block::create_unique(); @@ -87,6 +86,46 @@ Status PipelineXTask::prepare(RuntimeState* state) { return Status::OK(); } +void PipelineXTask::_init_profile() { + std::stringstream ss; + ss << "PipelineTask" + << " (index=" << _index << ")"; + auto* task_profile = new RuntimeProfile(ss.str()); + _parent_profile->add_child(task_profile, true, nullptr); + _task_profile.reset(task_profile); + _task_cpu_timer = ADD_TIMER(_task_profile, "TaskCpuTime"); + + static const char* exec_time = "ExecuteTime"; + _exec_timer = ADD_TIMER(_task_profile, exec_time); + _prepare_timer = ADD_CHILD_TIMER(_task_profile, "PrepareTime", exec_time); + _open_timer = ADD_CHILD_TIMER(_task_profile, "OpenTime", exec_time); + _get_block_timer = ADD_CHILD_TIMER(_task_profile, "GetBlockTime", exec_time); + _get_block_counter = ADD_COUNTER(_task_profile, "GetBlockCounter", TUnit::UNIT); + _sink_timer = ADD_CHILD_TIMER(_task_profile, "SinkTime", exec_time); + _finalize_timer = ADD_CHILD_TIMER(_task_profile, "FinalizeTime", exec_time); + _close_timer = ADD_CHILD_TIMER(_task_profile, "CloseTime", exec_time); + + _wait_source_timer = ADD_TIMER(_task_profile, "WaitSourceTime"); + _wait_bf_timer = ADD_TIMER(_task_profile, "WaitBfTime"); + _wait_sink_timer = ADD_TIMER(_task_profile, "WaitSinkTime"); + _wait_worker_timer = ADD_TIMER(_task_profile, "WaitWorkerTime"); + + _block_counts = ADD_COUNTER(_task_profile, "NumBlockedTimes", TUnit::UNIT); + _block_by_source_counts = ADD_COUNTER(_task_profile, "NumBlockedBySrcTimes", TUnit::UNIT); + _block_by_sink_counts = ADD_COUNTER(_task_profile, "NumBlockedBySinkTimes", TUnit::UNIT); + _schedule_counts = ADD_COUNTER(_task_profile, "NumScheduleTimes", TUnit::UNIT); + _yield_counts = ADD_COUNTER(_task_profile, "NumYieldTimes", TUnit::UNIT); + _core_change_times = ADD_COUNTER(_task_profile, "CoreChangeTimes", TUnit::UNIT); +} + +void PipelineXTask::_fresh_profile_counter() { + COUNTER_SET(_wait_source_timer, (int64_t)_wait_source_watcher.elapsed_time()); + COUNTER_SET(_wait_bf_timer, (int64_t)_wait_bf_watcher.elapsed_time()); + COUNTER_SET(_schedule_counts, (int64_t)_schedule_time); + COUNTER_SET(_wait_sink_timer, (int64_t)_wait_sink_watcher.elapsed_time()); + COUNTER_SET(_wait_worker_timer, (int64_t)_wait_worker_watcher.elapsed_time()); +} + Status PipelineXTask::_open() { SCOPED_TIMER(_task_profile->total_time_counter()); SCOPED_CPU_TIMER(_task_cpu_timer); @@ -255,19 +294,10 @@ std::string PipelineXTask::debug_string() { fmt::format_to( debug_string_buffer, "\n{}", _opened ? _operators[i]->debug_string(_state, i) : _operators[i]->debug_string(i)); - std::stringstream profile_ss; - _operators[i]->get_runtime_profile()->pretty_print(&profile_ss, std::string(i * 2, ' ')); - fmt::format_to(debug_string_buffer, "\n{}", profile_ss.str()); } fmt::format_to(debug_string_buffer, "\n{}", _opened ? _sink->debug_string(_state, _operators.size()) : _sink->debug_string(_operators.size())); - { - std::stringstream profile_ss; - _sink->get_runtime_profile()->pretty_print(&profile_ss, - std::string(_operators.size() * 2, ' ')); - fmt::format_to(debug_string_buffer, "\n{}", profile_ss.str()); - } return fmt::to_string(debug_string_buffer); } diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.h b/be/src/pipeline/pipeline_x/pipeline_x_task.h index 7e2458b12a..1fad0560b0 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.h @@ -52,7 +52,11 @@ public: PipelineXTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* state, PipelineFragmentContext* fragment_context, RuntimeProfile* parent_profile); - Status prepare(RuntimeState* state) override; + Status prepare(RuntimeState* state) override { + return Status::InternalError("Should not reach here!"); + } + + Status prepare(RuntimeState* state, const TPipelineInstanceParams& local_params); Status execute(bool* eos) override; @@ -68,7 +72,9 @@ public: return true; } for (auto& op : _operators) { - if (!op->can_read(_state)) { + auto dep = op->wait_for_dependency(_state); + if (dep != nullptr) { + dep->start_read_watcher(); return false; } } @@ -118,6 +124,9 @@ public: } private: + void set_close_pipeline_time() override {} + void _init_profile() override; + void _fresh_profile_counter() override; using DependencyMap = std::map; Status _open() override; diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index adbe401356..37732e9036 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -212,15 +212,30 @@ void FragmentMgr::coordinator_callback(const ReportStatusRequest& req) { params.__set_loaded_rows(req.runtime_state->num_rows_load_total()); params.__set_loaded_bytes(req.runtime_state->num_bytes_load_total()); } - if (req.profile == nullptr) { - params.__isset.profile = false; - } else { + params.__isset.detailed_report = req.is_pipeline_x; + if (req.profile != nullptr) { req.profile->to_thrift(¶ms.profile); if (req.load_channel_profile) { req.load_channel_profile->to_thrift(¶ms.loadChannelProfile); } params.__isset.profile = true; params.__isset.loadChannelProfile = true; + } else if (!req.runtime_states.empty()) { + params.__isset.detailed_report = true; + for (auto* rs : req.runtime_states) { + TDetailedReportParams detailed_param; + detailed_param.__set_fragment_instance_id(rs->fragment_instance_id()); + detailed_param.__isset.fragment_instance_id = true; + detailed_param.__isset.profile = true; + detailed_param.__isset.loadChannelProfile = true; + + rs->runtime_profile()->to_thrift(&detailed_param.profile); + rs->load_channel_profile()->to_thrift(&detailed_param.loadChannelProfile); + + params.detailed_report.push_back(detailed_param); + } + } else { + params.__isset.profile = false; } if (!req.runtime_state->output_files().empty()) { diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp index dd62e64fe2..18ef1871ec 100644 --- a/be/src/runtime/plan_fragment_executor.cpp +++ b/be/src/runtime/plan_fragment_executor.cpp @@ -536,7 +536,9 @@ void PlanFragmentExecutor::send_report(bool done) { return; } ReportStatusRequest report_req = { + false, status, + {}, _runtime_state->enable_profile() ? _runtime_state->runtime_profile() : nullptr, _runtime_state->enable_profile() ? _runtime_state->load_channel_profile() : nullptr, done || !status.ok(), diff --git a/be/src/runtime/query_context.h b/be/src/runtime/query_context.h index c27c517ac1..88a8367ff9 100644 --- a/be/src/runtime/query_context.h +++ b/be/src/runtime/query_context.h @@ -41,7 +41,9 @@ namespace doris { struct ReportStatusRequest { + bool is_pipeline_x; const Status& status; + std::vector runtime_states; RuntimeProfile* profile; RuntimeProfile* load_channel_profile; bool done; diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h index 29183a08da..0c11314fb2 100644 --- a/be/src/vec/exec/scan/pip_scanner_context.h +++ b/be/src/vec/exec/scan/pip_scanner_context.h @@ -50,6 +50,12 @@ public: _col_distribute_ids(col_distribute_ids), _need_colocate_distribute(!_col_distribute_ids.empty()) {} + void set_dependency(std::shared_ptr dependency, + std::shared_ptr scanner_done_dependency) override { + _data_dependency = dependency; + _scanner_done_dependency = scanner_done_dependency; + } + Status get_block_from_queue(RuntimeState* state, vectorized::BlockUPtr* block, bool* eos, int id, bool wait = false) override { { @@ -68,17 +74,23 @@ public: *eos = _is_finished || _should_stop; return Status::OK(); } + if (_blocks_queues[id].size_approx() == 0 && _data_dependency) { + _data_dependency->block_reading(); + } } _current_used_bytes -= (*block)->allocated_bytes(); return Status::OK(); } // We should make those method lock free. - bool done() override { return _is_finished || _should_stop || _status_error; } + bool done() override { return _is_finished || _should_stop; } void append_blocks_to_queue(std::vector& blocks) override { const int queue_size = _blocks_queues.size(); const int block_size = blocks.size(); + if (block_size == 0) { + return; + } int64_t local_bytes = 0; if (_need_colocate_distribute) { @@ -128,6 +140,9 @@ public: _next_queue_to_feed = queue + 1 < queue_size ? queue + 1 : 0; } } + if (_data_dependency) { + _data_dependency->set_ready_for_read(); + } _current_used_bytes += local_bytes; } @@ -172,6 +187,9 @@ public: _blocks_queues[i].enqueue(std::move(_colocate_blocks[i])); _colocate_mutable_blocks[i]->clear(); } + if (_data_dependency) { + _data_dependency->set_ready_for_read(); + } } } } @@ -196,6 +214,8 @@ private: std::vector> _colocate_mutable_blocks; std::vector> _colocate_block_mutexs; + std::shared_ptr _data_dependency = nullptr; + void _add_rows_colocate_blocks(vectorized::Block* block, int loc, const std::vector& rows) { int row_wait_add = rows.size(); @@ -218,7 +238,10 @@ private: if (row_add == max_add) { _current_used_bytes += _colocate_blocks[loc]->allocated_bytes(); - { _blocks_queues[loc].enqueue(std::move(_colocate_blocks[loc])); } + _blocks_queues[loc].enqueue(std::move(_colocate_blocks[loc])); + if (_data_dependency) { + _data_dependency->set_ready_for_read(); + } bool get_block_not_empty = true; _colocate_blocks[loc] = get_free_block(&get_block_not_empty, get_block_not_empty); _colocate_mutable_blocks[loc]->set_muatable_columns( @@ -227,5 +250,6 @@ private: } } }; + } // namespace pipeline } // namespace doris diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index 4f53f2ac2a..80c5ef1a09 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -239,6 +239,15 @@ Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::Blo return Status::OK(); } +void ScannerContext::set_should_stop() { + if (_scanner_done_dependency) { + _scanner_done_dependency->set_ready_for_read(); + } + std::lock_guard l(_transfer_lock); + _should_stop = true; + _blocks_queue_added_cv.notify_one(); +} + bool ScannerContext::set_status_on_error(const Status& status, bool need_lock) { std::unique_lock l(_transfer_lock, std::defer_lock); if (need_lock) { @@ -248,6 +257,9 @@ bool ScannerContext::set_status_on_error(const Status& status, bool need_lock) { _process_status = status; _status_error = true; _blocks_queue_added_cv.notify_one(); + if (_scanner_done_dependency) { + _scanner_done_dependency->set_ready_for_read(); + } _should_stop = true; return true; } @@ -315,7 +327,9 @@ void ScannerContext::clear_and_join(Parent* parent, RuntimeState* state) { if (_num_running_scanners == 0 && _num_scheduling_ctx == 0) { break; } else { - DCHECK(!state->enable_pipeline_exec()); + DCHECK(!state->enable_pipeline_exec()) + << " _num_running_scanners: " << _num_running_scanners + << " _num_scheduling_ctx: " << _num_scheduling_ctx; while (!(_num_running_scanners == 0 && _num_scheduling_ctx == 0)) { _ctx_finish_cv.wait(l); } @@ -385,6 +399,9 @@ void ScannerContext::push_back_scanner_and_reschedule(VScannerSPtr scanner) { (--_num_unfinished_scanners) == 0) { _dispose_coloate_blocks_not_in_queue(); _is_finished = true; + if (_scanner_done_dependency) { + _scanner_done_dependency->set_ready_for_read(); + } _blocks_queue_added_cv.notify_one(); } // In pipeline engine, doris will close scanners when `no_schedule`. diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index e486acc001..b4d89dcb62 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -43,6 +43,8 @@ class TupleDescriptor; namespace pipeline { class ScanLocalStateBase; +struct ScannerDoneDependency; +struct DataReadyDependency; } // namespace pipeline namespace taskgroup { @@ -100,13 +102,13 @@ public: return _process_status; } + virtual void set_dependency( + std::shared_ptr dependency, + std::shared_ptr scanner_done_dependency) {} + // Called by ScanNode. // Used to notify the scheduler that this ScannerContext can stop working. - void set_should_stop() { - std::lock_guard l(_transfer_lock); - _should_stop = true; - _blocks_queue_added_cv.notify_one(); - } + void set_should_stop(); // Return true if this ScannerContext need no more process virtual bool done() { return _is_finished || _should_stop; } @@ -264,6 +266,8 @@ protected: RuntimeProfile::HighWaterMarkCounter* _queued_blocks_memory_usage = nullptr; RuntimeProfile::Counter* _newly_create_free_blocks_num = nullptr; RuntimeProfile::Counter* _scanner_wait_batch_timer = nullptr; + + std::shared_ptr _scanner_done_dependency = nullptr; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index 30588538d7..27ef81be22 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -27,6 +27,7 @@ #include #include "common/logging.h" +#include "pipeline/exec/exchange_source_operator.h" #include "runtime/memory/mem_tracker.h" #include "runtime/runtime_state.h" #include "runtime/thread_context.h" @@ -94,6 +95,9 @@ Status VDataStreamRecvr::SenderQueue::_inner_get_batch_without_lock(Block* block auto [next_block, block_byte_size] = std::move(_block_queue.front()); _recvr->update_blocks_memory_usage(-block_byte_size); _block_queue.pop_front(); + if (_block_queue.size() == 0 && _dependency) { + _dependency->block_reading(); + } if (!_pending_closures.empty()) { auto closure_pair = _pending_closures.front(); @@ -164,6 +168,9 @@ 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(); + } } // if done is nullptr, this function can't delay this response if (done != nullptr && _recvr->exceeds_limit(block_byte_size)) { @@ -217,6 +224,9 @@ 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(); + } _data_arrival_cv.notify_one(); } @@ -252,6 +262,9 @@ void VDataStreamRecvr::SenderQueue::decrement_senders(int be_number) { VLOG_FILE << "decremented senders: fragment_instance_id=" << _recvr->fragment_instance_id() << " node_id=" << _recvr->dest_node_id() << " #senders=" << _num_remaining_senders; if (_num_remaining_senders == 0) { + if (_dependency) { + _dependency->set_always_done(); + } _data_arrival_cv.notify_one(); } } @@ -263,6 +276,9 @@ void VDataStreamRecvr::SenderQueue::cancel() { return; } _is_cancelled = true; + if (_dependency) { + _dependency->set_always_done(); + } VLOG_QUERY << "cancelled stream: _fragment_instance_id=" << _recvr->fragment_instance_id() << " node_id=" << _recvr->dest_node_id(); } @@ -289,6 +305,9 @@ void VDataStreamRecvr::SenderQueue::close() { // is clear will be memory leak std::lock_guard l(_lock); _is_cancelled = true; + if (_dependency) { + _dependency->set_always_done(); + } for (auto closure_pair : _pending_closures) { closure_pair.first->Run(); @@ -447,4 +466,44 @@ void VDataStreamRecvr::close() { } } +void VDataStreamRecvr::PipSenderQueue::add_block(Block* block, bool use_move) { + if (block->rows() == 0) { + return; + } + { + std::unique_lock l(_lock); + if (_is_cancelled) { + return; + } + } + BlockUPtr nblock = Block::create_unique(block->get_columns_with_type_and_name()); + + // local exchange should copy the block contented if use move == false + if (use_move) { + block->clear(); + } else { + auto rows = block->rows(); + for (int i = 0; i < nblock->columns(); ++i) { + nblock->get_by_position(i).column = + nblock->get_by_position(i).column->clone_resized(rows); + } + } + materialize_block_inplace(*nblock); + + auto block_mem_size = nblock->allocated_bytes(); + { + std::unique_lock l(_lock); + if (_is_cancelled) { + return; + } + _block_queue.emplace_back(std::move(nblock), block_mem_size); + if (_dependency) { + _dependency->set_ready_for_read(); + } + COUNTER_UPDATE(_recvr->_local_bytes_received_counter, block_mem_size); + _recvr->update_blocks_memory_usage(block_mem_size); + _data_arrival_cv.notify_one(); + } +} + } // namespace doris::vectorized diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index d79e9ed90a..271e14b88f 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -57,12 +57,19 @@ class MemTrackerLimiter; class PQueryStatistics; class RuntimeState; +namespace pipeline { +struct ExchangeDataDependency; +} + namespace vectorized { class VDataStreamMgr; class VSortedRunMerger; +class VDataStreamRecvr; + class VDataStreamRecvr { public: + class SenderQueue; VDataStreamRecvr(VDataStreamMgr* stream_mgr, RuntimeState* state, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id, PlanNodeId dest_node_id, int num_senders, bool is_merging, RuntimeProfile* profile, @@ -75,6 +82,8 @@ public: const std::vector& nulls_first, size_t batch_size, int64_t limit, size_t offset); + std::vector sender_queues() const { return _sender_queues; } + Status add_block(const PBlock& pblock, int sender_id, int be_number, int64_t packet_seq, ::google::protobuf::Closure** done); @@ -117,7 +126,6 @@ private: _blocks_memory_usage->add(size); _blocks_memory_usage_current_value = _blocks_memory_usage->current_value(); } - class SenderQueue; class PipSenderQueue; friend struct BlockSupplierSortCursorImpl; @@ -209,6 +217,10 @@ public: return _block_queue.empty(); } + void set_dependency(std::shared_ptr dependency) { + _dependency = dependency; + } + protected: Status _inner_get_batch_without_lock(Block* block, bool* eos); @@ -228,6 +240,8 @@ protected: std::unordered_map _packet_seq_map; std::deque> _pending_closures; std::unordered_map> _local_closure; + + std::shared_ptr _dependency = nullptr; }; class VDataStreamRecvr::PipSenderQueue : public SenderQueue { @@ -244,42 +258,7 @@ public: return _inner_get_batch_without_lock(block, eos); } - void add_block(Block* block, bool use_move) override { - if (block->rows() == 0) { - return; - } - { - std::unique_lock l(_lock); - if (_is_cancelled) { - return; - } - } - BlockUPtr nblock = Block::create_unique(block->get_columns_with_type_and_name()); - - // local exchange should copy the block contented if use move == false - if (use_move) { - block->clear(); - } else { - auto rows = block->rows(); - for (int i = 0; i < nblock->columns(); ++i) { - nblock->get_by_position(i).column = - nblock->get_by_position(i).column->clone_resized(rows); - } - } - materialize_block_inplace(*nblock); - - auto block_mem_size = nblock->allocated_bytes(); - { - std::unique_lock l(_lock); - if (_is_cancelled) { - return; - } - _block_queue.emplace_back(std::move(nblock), block_mem_size); - COUNTER_UPDATE(_recvr->_local_bytes_received_counter, block_mem_size); - _recvr->update_blocks_memory_usage(block_mem_size); - _data_arrival_cv.notify_one(); - } - } + void add_block(Block* block, bool use_move) override; }; } // namespace vectorized } // namespace doris diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 084da08235..3309c82883 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -80,6 +80,7 @@ import org.apache.doris.task.LoadEtlTask; import org.apache.doris.thrift.PaloInternalServiceVersion; import org.apache.doris.thrift.TBrokerScanRange; import org.apache.doris.thrift.TDescriptorTable; +import org.apache.doris.thrift.TDetailedReportParams; import org.apache.doris.thrift.TErrorTabletInfo; import org.apache.doris.thrift.TEsScanRange; import org.apache.doris.thrift.TExecPlanFragmentParams; @@ -2305,9 +2306,42 @@ public class Coordinator { // update job progress from BE public void updateFragmentExecStatus(TReportExecStatusParams params) { - if (enablePipelineEngine) { + if (enablePipelineXEngine) { PipelineExecContext ctx = pipelineExecContexts.get(Pair.of(params.getFragmentId(), params.getBackendId())); - if (!ctx.updateProfile(params)) { + if (!ctx.updateProfile(params, true)) { + return; + } + + // print fragment instance profile + if (LOG.isDebugEnabled()) { + StringBuilder builder = new StringBuilder(); + ctx.printProfile(builder); + LOG.debug("profile for query_id={} fragment_id={}\n{}", + DebugUtil.printId(queryId), + params.getFragmentId(), + builder.toString()); + } + + Status status = new Status(params.status); + // for now, abort the query if we see any error except if the error is cancelled + // and returned_all_results_ is true. + // (UpdateStatus() initiates cancellation, if it hasn't already been initiated) + if (!(returnedAllResults && status.isCancelled()) && !status.ok()) { + LOG.warn("one instance report fail, query_id={} instance_id={}, error message: {}", + DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId()), + status.getErrorMsg()); + updateStatus(status, params.getFragmentInstanceId()); + } + Preconditions.checkArgument(params.isSetDetailedReport()); + for (TDetailedReportParams param : params.detailed_report) { + if (ctx.fragmentInstancesMap.get(param.fragment_instance_id).getIsDone()) { + // TODO + executionProfile.markOneInstanceDone(param.getFragmentInstanceId()); + } + } + } else if (enablePipelineEngine) { + PipelineExecContext ctx = pipelineExecContexts.get(Pair.of(params.getFragmentId(), params.getBackendId())); + if (!ctx.updateProfile(params, false)) { return; } @@ -2932,27 +2966,51 @@ public class Coordinator { // update profile. // return true if profile is updated. Otherwise, return false. - public synchronized boolean updateProfile(TReportExecStatusParams params) { - RuntimeProfile profile = fragmentInstancesMap.get(params.fragment_instance_id); - if (params.done && profile.getIsDone()) { - // duplicate packet - return false; - } + public synchronized boolean updateProfile(TReportExecStatusParams params, boolean isPipelineX) { + if (isPipelineX) { + for (TDetailedReportParams param : params.detailed_report) { + RuntimeProfile profile = fragmentInstancesMap.get(param.fragment_instance_id); + if (params.done && profile.getIsDone()) { + continue; + } - if (params.isSetProfile()) { - profile.update(params.profile); + if (param.isSetProfile()) { + profile.update(param.profile); + } + if (params.isSetLoadChannelProfile()) { + loadChannelProfile.update(params.loadChannelProfile); + } + if (params.done) { + profile.setIsDone(true); + profileReportProgress++; + } + if (profileReportProgress == numInstances) { + this.done = true; + } + } + return true; + } else { + RuntimeProfile profile = fragmentInstancesMap.get(params.fragment_instance_id); + if (params.done && profile.getIsDone()) { + // duplicate packet + return false; + } + + if (params.isSetProfile()) { + profile.update(params.profile); + } + if (params.isSetLoadChannelProfile()) { + loadChannelProfile.update(params.loadChannelProfile); + } + if (params.done) { + profile.setIsDone(true); + profileReportProgress++; + } + if (profileReportProgress == numInstances) { + this.done = true; + } + return true; } - if (params.isSetLoadChannelProfile()) { - loadChannelProfile.update(params.loadChannelProfile); - } - if (params.done) { - profile.setIsDone(true); - profileReportProgress++; - } - if (profileReportProgress == numInstances) { - this.done = true; - } - return true; } public synchronized void printProfile(StringBuilder builder) { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 65bd2b7151..e5c7b9bb0b 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -390,6 +390,12 @@ enum FrontendServiceVersion { V1 } +struct TDetailedReportParams { + 1: optional Types.TUniqueId fragment_instance_id + 2: optional RuntimeProfile.TRuntimeProfileTree profile + 3: optional RuntimeProfile.TRuntimeProfileTree loadChannelProfile +} + // The results of an INSERT query, sent to the coordinator as part of // TReportExecStatusParams struct TReportExecStatusParams { @@ -447,6 +453,8 @@ struct TReportExecStatusParams { 21: optional RuntimeProfile.TRuntimeProfileTree loadChannelProfile 22: optional i32 finished_scan_ranges + + 23: optional list detailed_report } struct TFeResult { diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql index c3d19b67a2..3a9ce71313 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE LO_ORDERDATE >= 19930101 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql index 6ab6ceea34..a37876b067 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q1.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE LO_ORDERDATE >= 19940101 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql index 70796c2a95..5ba5ee025c 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q1.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q1.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue FROM lineorder_flat WHERE weekofyear(LO_ORDERDATE) = 6 diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql index 57f2ada296..f42f86e685 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql index 9b7a5db502..ae458ca5c3 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql index 3a8a5e74d4..70771a50f7 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q2.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q2.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, P_BRAND FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql index 6b3257f1f3..e6c31d6315 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ C_NATION, S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql index fefe727da8..d9160639bd 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql index c4560b701e..1d6b34f9d5 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql index 4ae5d956e4..2c1ce76817 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q3.4.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q3.4 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ C_CITY, S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, SUM(LO_REVENUE) AS revenue diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql index 87b29bf160..33f491eece 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.1 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, C_NATION, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit FROM lineorder_flat diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql index 8ea28f3f12..a71d014add 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.2 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, S_NATION, P_CATEGORY, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql index 0f7c7401ab..3c247a188f 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_flat_q4.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. --Q4.3 -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ (LO_ORDERDATE DIV 10000) AS YEAR, S_CITY, P_BRAND, SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql index 50b50bc368..260b1ee03b 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql index 77c0262016..b8b54ccaad 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql index 0052db0aac..fec034dd9b 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q1.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_extendedprice*lo_discount) AS +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(lo_extendedprice*lo_discount) AS REVENUE FROM lineorder, date WHERE lo_orderdate = d_datekey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql index a47ec82b51..d71685a91a 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql index 9ab1a95d4d..c2a5f6dc15 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql index b7e6bd7840..19331d9ee5 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q2.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ SUM(lo_revenue), d_year, p_brand +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ SUM(lo_revenue), d_year, p_brand FROM lineorder, date, part, supplier WHERE lo_orderdate = d_datekey AND lo_partkey = p_partkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql index 85c470b708..a99bffe752 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_nation, s_nation, d_year, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ c_nation, s_nation, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql index cd0b320f87..65acd47881 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, sum(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ c_city, s_city, d_year, sum(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql index 89765c02d9..18fe99b85a 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, SUM(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ c_city, s_city, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql index 5cef87a3fe..f59aac3ee6 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q3.4.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ c_city, s_city, d_year, SUM(lo_revenue) +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ c_city, s_city, d_year, SUM(lo_revenue) AS REVENUE FROM customer, lineorder, supplier, date WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql index 3e0227c2ea..029934b62a 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.1.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, c_nation, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ d_year, c_nation, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql index 1338e780ae..fc4bf8402e 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.2.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, s_nation, p_category, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ d_year, s_nation, p_category, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql index d8e6f7c42d..8c7c315ebc 100644 --- a/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql +++ b/regression-test/suites/ssb_sf0.1_p1/sql/pipelinex_q4.3.sql @@ -15,7 +15,7 @@ -- specific language governing permissions and limitations -- under the License. -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ d_year, s_city, p_brand, +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ d_year, s_city, p_brand, SUM(lo_revenue - lo_supplycost) AS PROFIT FROM date, customer, supplier, part, lineorder WHERE lo_custkey = c_custkey diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql index ded6754a97..76d38414cd 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q01.sql @@ -1,5 +1,5 @@ -- tables: lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ l_returnflag, l_linestatus, sum(l_quantity) AS sum_qty, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql index f102f7504d..25ec865314 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q02.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,partsupp,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ s_acctbal, s_name, n_name, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql index 8bd60f0e07..eb6831b7ca 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q03.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ l_orderkey, sum(l_extendedprice * (1 - l_discount)) AS revenue, o_orderdate, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql index 3f44094729..e05c649b93 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q04.sql @@ -1,5 +1,5 @@ -- tables: orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ o_orderpriority, count(*) AS order_count FROM orders diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql index ed179f8b86..b69b05e5e0 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q05.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem,supplier,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ n_name, sum(l_extendedprice * (1 - l_discount)) AS revenue FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql index 2dd86f8c2c..a9a080a45b 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q06.sql @@ -1,6 +1,6 @@ -- tables: lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice * l_discount) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ sum(l_extendedprice * l_discount) AS revenue FROM lineitem WHERE diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql index 6453c1094a..79efbfdae4 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q07.sql @@ -1,5 +1,5 @@ -- tables: supplier,lineitem,orders,customer,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ supp_nation, cust_nation, l_year, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql index e4c46fb084..1faced9805 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q08.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,lineitem,orders,customer,nation,region -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ o_year, sum(CASE WHEN nation = 'BRAZIL' diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql index cee9925fb5..a47a3b5e9c 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q09.sql @@ -1,5 +1,5 @@ -- tables: part,supplier,lineitem,partsupp,orders,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ nation, o_year, sum(amount) AS sum_profit diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql index c95a80fcee..3d49252d3e 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q10.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) AS revenue, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql index b23701e940..66140c9431 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q11.sql @@ -1,5 +1,5 @@ -- tables: partsupp,supplier,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ ps_partkey, sum(ps_supplycost * ps_availqty) AS value FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql index e8893e71e4..99c9c0f574 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q12.sql @@ -1,5 +1,5 @@ -- tables: orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ l_shipmode, sum(CASE WHEN o_orderpriority = '1-URGENT' diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql index 9db2da60ee..21a03f752c 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q13.sql @@ -1,5 +1,5 @@ -- tables: customer -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ c_count, count(*) AS custdist FROM ( diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql index 70d7a57d07..b0e79f8da5 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q14.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ 100.00 * sum(CASE +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0 diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql index 45f75ff985..03c0b97372 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q15.sql @@ -1,4 +1,4 @@ -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ s_suppkey, s_name, s_address, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql index 37a438c796..7df971f830 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q16.sql @@ -1,5 +1,5 @@ -- tables: partsupp,part,supplier -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ p_brand, p_type, p_size, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql index 62f39a750c..3f9203cea0 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q17.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice) / 7.0 AS avg_yearly +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, part diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql index 2eb2505c01..971d457672 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q18.sql @@ -1,5 +1,5 @@ -- tables: customer,orders,lineitem -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ c_name, c_custkey, o_orderkey, diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql index 16e543f87c..8cc1b890c9 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q19.sql @@ -1,5 +1,5 @@ -- tables: lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ sum(l_extendedprice * (1 - l_discount)) AS revenue +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ sum(l_extendedprice * (1 - l_discount)) AS revenue FROM lineitem, part diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql index a2aca56790..f54d44ed8b 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q20.sql @@ -1,5 +1,5 @@ -- tables: supplier,nation,partsupp,lineitem,part -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ s_name, s_address FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql index 7b4874f96c..56d81211db 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q21.sql @@ -1,5 +1,5 @@ -- tables: supplier,lineitem,orders,nation -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ s_name, count(*) AS numwait FROM diff --git a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql index bf784175e0..c3f199f124 100644 --- a/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql +++ b/regression-test/suites/tpch_unique_sql_zstd_p0/sql/pipelinex_q22.sql @@ -1,5 +1,5 @@ -- tables: orders,customer -SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true) */ +SELECT /*+SET_VAR(experimental_enable_pipeline_x_engine=true, disable_streaming_preaggregations=true) */ cntrycode, count(*) AS numcust, sum(c_acctbal) AS totacctbal