From 3358f76a7f6ac5acca6483e3483f33f8596e1924 Mon Sep 17 00:00:00 2001 From: TengJianPing <18241664+jacktengg@users.noreply.github.com> Date: Mon, 11 Mar 2024 09:36:25 +0800 Subject: [PATCH] [feature](spill) Implement spill to disk for hash join, aggregation and sort for pipelineX (#31910) Co-authored-by: Jerry Hu --- be/src/common/config.cpp | 9 + be/src/common/config.h | 8 + be/src/common/daemon.cpp | 11 - be/src/common/daemon.h | 1 - be/src/olap/data_dir.cpp | 6 +- be/src/olap/data_dir.h | 2 +- be/src/olap/olap_define.h | 2 + .../exec/aggregation_sink_operator.cpp | 113 +-- .../pipeline/exec/aggregation_sink_operator.h | 192 +---- .../exec/aggregation_source_operator.cpp | 334 ++++---- .../exec/aggregation_source_operator.h | 35 +- .../distinct_streaming_aggregation_operator.h | 1 - be/src/pipeline/exec/hashjoin_build_sink.cpp | 1 - .../exec/join_build_sink_operator.cpp | 4 + .../pipeline/exec/join_build_sink_operator.h | 1 - be/src/pipeline/exec/join_probe_operator.cpp | 5 + be/src/pipeline/exec/join_probe_operator.h | 1 - .../exec/multi_cast_data_stream_sink.cpp | 2 - .../exec/multi_cast_data_stream_source.h | 1 - be/src/pipeline/exec/operator.h | 4 + .../partitioned_aggregation_sink_operator.cpp | 288 +++++++ .../partitioned_aggregation_sink_operator.h | 342 +++++++++ ...artitioned_aggregation_source_operator.cpp | 253 ++++++ .../partitioned_aggregation_source_operator.h | 106 +++ .../partitioned_hash_join_probe_operator.cpp | 724 ++++++++++++++++++ .../partitioned_hash_join_probe_operator.h | 194 +++++ .../partitioned_hash_join_sink_operator.cpp | 223 ++++++ .../partitioned_hash_join_sink_operator.h | 129 ++++ be/src/pipeline/exec/result_sink_operator.h | 1 - be/src/pipeline/exec/sort_sink_operator.cpp | 24 +- be/src/pipeline/exec/sort_sink_operator.h | 10 + be/src/pipeline/exec/sort_source_operator.cpp | 6 + be/src/pipeline/exec/sort_source_operator.h | 2 + .../exec/spill_sort_sink_operator.cpp | 292 +++++++ .../pipeline/exec/spill_sort_sink_operator.h | 104 +++ .../exec/spill_sort_source_operator.cpp | 261 +++++++ .../exec/spill_sort_source_operator.h | 102 +++ .../streaming_aggregation_source_operator.cpp | 1 - be/src/pipeline/pipeline_x/dependency.cpp | 74 ++ be/src/pipeline/pipeline_x/dependency.h | 114 ++- .../local_exchange_sink_operator.h | 1 - .../local_exchange_source_operator.h | 1 - .../local_exchange/local_exchanger.h | 1 - be/src/pipeline/pipeline_x/operator.cpp | 21 + be/src/pipeline/pipeline_x/operator.h | 64 +- .../pipeline_x_fragment_context.cpp | 96 ++- .../pipeline/pipeline_x/pipeline_x_task.cpp | 27 + be/src/runtime/exec_env.h | 6 + be/src/runtime/exec_env_init.cpp | 14 +- be/src/runtime/runtime_state.h | 36 + be/src/service/doris_main.cpp | 31 +- be/src/vec/common/sort/partition_sorter.h | 2 - be/src/vec/common/sort/sorter.cpp | 197 ++--- be/src/vec/common/sort/sorter.h | 66 +- be/src/vec/common/sort/topn_sorter.cpp | 31 +- be/src/vec/common/sort/topn_sorter.h | 2 - be/src/vec/exec/vsort_node.cpp | 3 - be/src/vec/spill/spill_reader.cpp | 147 ++++ be/src/vec/spill/spill_reader.h | 81 ++ be/src/vec/spill/spill_stream.cpp | 137 ++++ be/src/vec/spill/spill_stream.h | 105 +++ be/src/vec/spill/spill_stream_manager.cpp | 262 +++++++ be/src/vec/spill/spill_stream_manager.h | 91 +++ be/src/vec/spill/spill_writer.cpp | 160 ++++ be/src/vec/spill/spill_writer.h | 92 +++ .../org/apache/doris/qe/SessionVariable.java | 98 ++- gensrc/thrift/PaloInternalService.thrift | 8 + 67 files changed, 5045 insertions(+), 718 deletions(-) create mode 100644 be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp create mode 100644 be/src/pipeline/exec/partitioned_aggregation_sink_operator.h create mode 100644 be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp create mode 100644 be/src/pipeline/exec/partitioned_aggregation_source_operator.h create mode 100644 be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp create mode 100644 be/src/pipeline/exec/partitioned_hash_join_probe_operator.h create mode 100644 be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp create mode 100644 be/src/pipeline/exec/partitioned_hash_join_sink_operator.h create mode 100644 be/src/pipeline/exec/spill_sort_sink_operator.cpp create mode 100644 be/src/pipeline/exec/spill_sort_sink_operator.h create mode 100644 be/src/pipeline/exec/spill_sort_source_operator.cpp create mode 100644 be/src/pipeline/exec/spill_sort_source_operator.h create mode 100644 be/src/vec/spill/spill_reader.cpp create mode 100644 be/src/vec/spill/spill_reader.h create mode 100644 be/src/vec/spill/spill_stream.cpp create mode 100644 be/src/vec/spill/spill_stream.h create mode 100644 be/src/vec/spill/spill_stream_manager.cpp create mode 100644 be/src/vec/spill/spill_stream_manager.h create mode 100644 be/src/vec/spill/spill_writer.cpp create mode 100644 be/src/vec/spill/spill_writer.h diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index f534e70126..3d44528ea3 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1176,6 +1176,15 @@ DEFINE_mDouble(high_disk_avail_level_diff_usages, "0.15"); // create tablet in partition random robin idx lru size, default 10000 DEFINE_Int32(partition_disk_index_lru_size, "10000"); +// limit the storage space that query spill files can use +DEFINE_String(spill_storage_root_path, "${DORIS_HOME}/storage"); +DEFINE_mInt64(spill_storage_limit, "10737418240"); // 10G +DEFINE_mInt32(spill_gc_interval_ms, "2000"); // 2s +DEFINE_Int32(spill_io_thread_pool_per_disk_thread_num, "2"); +DEFINE_Int32(spill_io_thread_pool_queue_size, "1024"); +DEFINE_Int32(spill_async_task_thread_pool_thread_num, "2"); +DEFINE_Int32(spill_async_task_thread_pool_queue_size, "1024"); +DEFINE_mInt32(spill_mem_warning_water_mark_multiplier, "2"); DEFINE_mBool(check_segment_when_build_rowset_meta, "false"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 70713447ca..a5219c4259 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1250,6 +1250,14 @@ DECLARE_mDouble(high_disk_avail_level_diff_usages); // create tablet in partition random robin idx lru size, default 10000 DECLARE_Int32(partition_disk_index_lru_size); +DECLARE_String(spill_storage_root_path); +DECLARE_mInt64(spill_storage_limit); +DECLARE_mInt32(spill_gc_interval_ms); +DECLARE_Int32(spill_io_thread_pool_per_disk_thread_num); +DECLARE_Int32(spill_io_thread_pool_queue_size); +DECLARE_Int32(spill_async_task_thread_pool_thread_num); +DECLARE_Int32(spill_async_task_thread_pool_queue_size); +DECLARE_mInt32(spill_mem_warning_water_mark_multiplier); DECLARE_mBool(check_segment_when_build_rowset_meta); diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index 5274808c9b..cb493ea807 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -44,7 +44,6 @@ #include "olap/options.h" #include "olap/storage_engine.h" #include "olap/tablet_manager.h" -#include "runtime/block_spill_manager.h" #include "runtime/client_cache.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" @@ -348,13 +347,6 @@ void Daemon::calculate_metrics_thread() { } while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(15))); } -// clean up stale spilled files -void Daemon::block_spill_gc_thread() { - while (!_stop_background_threads_latch.wait_for(std::chrono::seconds(60))) { - ExecEnv::GetInstance()->block_spill_mgr()->gc(200); - } -} - void Daemon::report_runtime_query_statistics_thread() { while (!_stop_background_threads_latch.wait_for( std::chrono::milliseconds(config::report_query_statistics_interval_ms))) { @@ -403,9 +395,6 @@ void Daemon::start() { [this]() { this->calculate_metrics_thread(); }, &_threads.emplace_back()); CHECK(st.ok()) << st; } - st = Thread::create( - "Daemon", "block_spill_gc_thread", [this]() { this->block_spill_gc_thread(); }, - &_threads.emplace_back()); st = Thread::create( "Daemon", "je_purge_dirty_pages_thread", [this]() { this->je_purge_dirty_pages_thread(); }, &_threads.emplace_back()); diff --git a/be/src/common/daemon.h b/be/src/common/daemon.h index e88dd73764..5d54ba5b49 100644 --- a/be/src/common/daemon.h +++ b/be/src/common/daemon.h @@ -42,7 +42,6 @@ private: void memory_gc_thread(); void memtable_memory_limiter_tracker_refresh_thread(); void calculate_metrics_thread(); - void block_spill_gc_thread(); void je_purge_dirty_pages_thread() const; void report_runtime_query_statistics_thread(); diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index 8713dc2742..96b6c90129 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -115,7 +115,7 @@ DataDir::~DataDir() { delete _meta; } -Status DataDir::init() { +Status DataDir::init(bool init_meta) { bool exists = false; RETURN_IF_ERROR(io::global_local_filesystem()->exists(_path, &exists)); if (!exists) { @@ -127,7 +127,9 @@ Status DataDir::init() { RETURN_NOT_OK_STATUS_WITH_WARN(_init_cluster_id(), "_init_cluster_id failed"); RETURN_NOT_OK_STATUS_WITH_WARN(_init_capacity_and_create_shards(), "_init_capacity_and_create_shards failed"); - RETURN_NOT_OK_STATUS_WITH_WARN(_init_meta(), "_init_meta failed"); + if (init_meta) { + RETURN_NOT_OK_STATUS_WITH_WARN(_init_meta(), "_init_meta failed"); + } _is_used = true; return Status::OK(); diff --git a/be/src/olap/data_dir.h b/be/src/olap/data_dir.h index 75fde6dc06..424315b79a 100644 --- a/be/src/olap/data_dir.h +++ b/be/src/olap/data_dir.h @@ -54,7 +54,7 @@ public: TabletManager* tablet_manager = nullptr, TxnManager* txn_manager = nullptr); ~DataDir(); - Status init(); + Status init(bool init_meta = true); void stop_bg_worker(); const std::string& path() const { return _path; } diff --git a/be/src/olap/olap_define.h b/be/src/olap/olap_define.h index 349e6a9dbc..49833b8733 100644 --- a/be/src/olap/olap_define.h +++ b/be/src/olap/olap_define.h @@ -97,6 +97,8 @@ static const std::string ERROR_LOG_PREFIX = "error_log"; static const std::string PENDING_DELTA_PREFIX = "pending_delta"; static const std::string INCREMENTAL_DELTA_PREFIX = "incremental_delta"; static const std::string CLONE_PREFIX = "clone"; +static const std::string SPILL_DIR_PREFIX = "spill"; +static const std::string SPILL_GC_DIR_PREFIX = "spill_gc"; // define paths static inline std::string remote_tablet_path(int64_t tablet_id) { diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index e718bb4149..64c946e6da 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -71,7 +71,6 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { Base::_shared_state->total_size_of_aggregate_states = p._total_size_of_aggregate_states; Base::_shared_state->offsets_of_aggregate_states = p._offsets_of_aggregate_states; Base::_shared_state->make_nullable_keys = p._make_nullable_keys; - Base::_shared_state->init_spill_partition_helper(p._spill_partition_count_bits); for (auto& evaluator : p._aggregate_evaluators) { Base::_shared_state->aggregate_evaluators.push_back(evaluator->clone(state, p._pool)); } @@ -86,7 +85,6 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { "SerializeKeyArena", TUnit::BYTES, "MemoryUsage", 1); _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"); @@ -200,6 +198,9 @@ Status AggSinkLocalState::_merge_with_serialized_key(vectorized::Block* block) { } size_t AggSinkLocalState::_memory_usage() const { + if (0 == _get_hash_table_size()) { + return 0; + } size_t usage = 0; if (_agg_arena_pool) { usage += _agg_arena_pool->size(); @@ -209,6 +210,13 @@ size_t AggSinkLocalState::_memory_usage() const { usage += Base::_shared_state->aggregate_data_container->memory_usage(); } + std::visit( + [&](auto&& agg_method) -> void { + auto data = agg_method.hash_table; + usage += data->get_buffer_size_in_bytes(); + }, + _agg_data->method_variant); + return usage; } @@ -274,7 +282,8 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b 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]); + int col_id = AggSharedState::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(); @@ -317,7 +326,8 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b if constexpr (for_spill) { col_id = Base::_shared_state->probe_expr_ctxs.size() + i; } else { - col_id = _get_slot_column_id(Base::_shared_state->aggregate_evaluators[i]); + col_id = AggSharedState::get_slot_column_id( + Base::_shared_state->aggregate_evaluators[i]); } auto column = block->get_by_position(col_id).column; if (column->is_nullable()) { @@ -361,23 +371,13 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b return Status::OK(); } -// 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. -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]=" - << ctxs[0]->root()->debug_string(); - return ((vectorized::VSlotRef*)ctxs[0]->root().get())->column_id(); -} - Status AggSinkLocalState::_merge_without_key(vectorized::Block* block) { SCOPED_TIMER(_merge_timer); DCHECK(_agg_data->without_key != nullptr); 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]); + int col_id = AggSharedState::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(); @@ -465,7 +465,7 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* _places.data(), _agg_arena_pool)); } - if (_should_limit_output) { + if (_should_limit_output && !Base::_shared_state->enable_spill) { _reach_limit = _get_hash_table_size() >= Base::_parent->template cast()._limit; if (_reach_limit && @@ -479,14 +479,14 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* return Status::OK(); } -size_t AggSinkLocalState::_get_hash_table_size() { +size_t AggSinkLocalState::_get_hash_table_size() const { return std::visit([&](auto&& agg_method) { return agg_method.hash_table->size(); }, _agg_data->method_variant); } void AggSinkLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* places, vectorized::ColumnRawPtrs& key_columns, - const size_t num_rows) { + size_t num_rows) { std::visit( [&](auto&& agg_method) -> void { SCOPED_TIMER(_hash_table_compute_timer); @@ -619,56 +619,6 @@ void AggSinkLocalState::_init_hash_method(const vectorized::VExprContextSPtrs& p } } -Status AggSinkLocalState::_reset_hash_table() { - auto& ss = *Base::_shared_state; - return std::visit( - [&](auto&& agg_method) { - auto& hash_table = *agg_method.hash_table; - using HashTableType = std::decay_t; - - agg_method.reset(); - - hash_table.for_each_mapped([&](auto& mapped) { - if (mapped) { - static_cast(_destroy_agg_status(mapped)); - mapped = nullptr; - } - }); - - ss.aggregate_data_container.reset(new vectorized::AggregateDataContainer( - sizeof(typename HashTableType::key_type), - ((ss.total_size_of_aggregate_states + ss.align_aggregate_states - 1) / - ss.align_aggregate_states) * - ss.align_aggregate_states)); - agg_method.hash_table.reset(new HashTableType()); - ss.agg_arena_pool.reset(new vectorized::Arena); - return Status::OK(); - }, - ss.agg_data->method_variant); -} - -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.hash_table; - if (!eos && _memory_usage() < Base::_parent->template cast() - ._external_agg_bytes_threshold) { - return Status::OK(); - } - - if (_get_hash_table_size() == 0) { - return Status::OK(); - } - - RETURN_IF_ERROR(_spill_hash_table(agg_method, hash_table)); - return _reset_hash_table(); - }, - _agg_data->method_variant); -} - AggSinkOperatorX::AggSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, const DescriptorTbl& descs) : DataSinkOperatorX(operator_id, tnode.node_id), @@ -712,13 +662,6 @@ Status AggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { const auto& agg_functions = tnode.agg_node.aggregate_functions; _external_agg_bytes_threshold = state->external_agg_bytes_threshold(); - if (_external_agg_bytes_threshold > 0) { - _spill_partition_count_bits = 4; - if (state->query_options().__isset.external_agg_partition_bits) { - _spill_partition_count_bits = state->query_options().external_agg_partition_bits; - } - } - _is_merge = std::any_of(agg_functions.cbegin(), agg_functions.cend(), [](const auto& e) { return e.nodes[0].agg_expr.is_merge_agg; }); @@ -796,19 +739,27 @@ Status AggSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in_ local_state._shared_state->input_num_rows += in_block->rows(); if (in_block->rows() > 0) { RETURN_IF_ERROR(local_state._executor->execute(&local_state, in_block)); - RETURN_IF_ERROR(local_state.try_spill_disk()); local_state._executor->update_memusage(&local_state); } if (eos) { - if (local_state._shared_state->spill_context.has_data) { - RETURN_IF_ERROR(local_state.try_spill_disk(true)); - RETURN_IF_ERROR(local_state._shared_state->spill_context.prepare_for_reading()); - } local_state._dependency->set_ready_to_read(); } return Status::OK(); } +size_t AggSinkOperatorX::get_revocable_mem_size(RuntimeState* state) const { + auto& local_state = get_local_state(state); + return local_state._memory_usage(); +} + +Status AggSinkOperatorX::reset_hash_table(RuntimeState* state) { + auto& local_state = get_local_state(state); + auto& ss = *local_state.Base::_shared_state; + RETURN_IF_ERROR(ss.reset_hash_table()); + local_state._agg_arena_pool = ss.agg_arena_pool.get(); + return Status::OK(); +} + Status AggSinkLocalState::close(RuntimeState* state, Status exec_status) { SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_close_timer); diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h index 29dbf61169..e3d8baad39 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -20,7 +20,6 @@ #include #include "operator.h" -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "runtime/block_spill_manager.h" #include "runtime/exec_env.h" @@ -58,8 +57,6 @@ public: Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; - Status try_spill_disk(bool eos = false); - protected: friend class AggSinkOperatorX; @@ -106,189 +103,14 @@ protected: void _find_in_hash_table(vectorized::AggregateDataPtr* places, vectorized::ColumnRawPtrs& key_columns, size_t num_rows); void _emplace_into_hash_table(vectorized::AggregateDataPtr* places, - vectorized::ColumnRawPtrs& key_columns, const size_t num_rows); - size_t _get_hash_table_size(); + vectorized::ColumnRawPtrs& key_columns, size_t num_rows); + size_t _get_hash_table_size() const; template Status _merge_with_serialized_key_helper(vectorized::Block* block); - template - Status _serialize_hash_table_to_block(HashTableCtxType& context, HashTableType& hash_table, - vectorized::Block& block, std::vector& keys_) { - 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); - vectorized::MutableColumns key_columns; - - for (int i = 0; i < key_size; ++i) { - key_columns.emplace_back( - Base::_shared_state->probe_expr_ctxs[i]->root()->data_type()->create_column()); - } - - 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(); - value_columns[i] = Base::_shared_state->aggregate_evaluators[i] - ->function() - ->create_serialize_column(); - } - - context.init_iterator(); - const auto size = hash_table.size(); - std::vector keys(size); - if (Base::_shared_state->values.size() < size) { - Base::_shared_state->values.resize(size); - } - - size_t num_rows = 0; - Base::_shared_state->aggregate_data_container->init_once(); - auto& iter = Base::_shared_state->aggregate_data_container->iterator; - - { - 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; - } - } - - { context.insert_keys_into_columns(keys, key_columns, num_rows); } - - if (hash_table.has_null_key_data()) { - // only one key of group by support wrap null key - // here need additional processing logic on the null key / value - CHECK(key_columns.size() == 1); - CHECK(key_columns[0]->is_nullable()); - key_columns[0]->insert_data(nullptr, 0); - - // Here is no need to set `keys[num_rows]`, keep it as default value. - Base::_shared_state->values[num_rows] = - hash_table.template get_null_key_data(); - ++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::_shared_state->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]), - 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], - Base::_shared_state->aggregate_evaluators[i]->function()->get_name()); - } - - block = columns_with_schema; - keys_.swap(keys); - return Status::OK(); - } - Status _destroy_agg_status(vectorized::AggregateDataPtr data); - template - Status _spill_hash_table(HashTableCtxType& agg_method, HashTableType& hash_table) { - vectorized::Block block; - std::vector keys; - RETURN_IF_ERROR(_serialize_hash_table_to_block(agg_method, hash_table, block, keys)); - CHECK_EQ(block.rows(), hash_table.size()); - CHECK_EQ(keys.size(), block.rows()); - - 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, - Base::_shared_state->spill_context.runtime_profile)); - Defer defer {[&]() { - // redundant call is ok - static_cast(writer->close()); - }}; - Base::_shared_state->spill_context.stream_ids.emplace_back(writer->get_id()); - - std::vector partitioned_indices(block.rows()); - 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 = 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 = Base::_shared_state->spill_partition_helper->partition_count - 1; - partitioned_indices[rows] = index; - blocks_rows[index]++; - } - - 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, - /// blocks' count should be equal with partition_count. - RETURN_IF_ERROR(writer->write(block_to_write)); - continue; - } - - vectorized::MutableBlock mutable_block(std::move(block_to_write)); - - for (auto& column : mutable_block.mutable_columns()) { - column->reserve(blocks_rows[i]); - } - - size_t begin = 0; - size_t length = 0; - for (size_t j = 0; j < partitioned_indices.size(); ++j) { - if (partitioned_indices[j] != i) { - if (length > 0) { - mutable_block.add_rows(&block, begin, length); - } - length = 0; - continue; - } - - if (length == 0) { - begin = j; - } - length++; - } - - if (length > 0) { - mutable_block.add_rows(&block, begin, length); - } - - CHECK_EQ(mutable_block.rows(), blocks_rows[i]); - RETURN_IF_ERROR(writer->write(mutable_block.to_block())); - } - RETURN_IF_ERROR(writer->close()); - - return Status::OK(); - } Status _create_agg_status(vectorized::AggregateDataPtr data); - Status _reset_hash_table(); - // We should call this function only at 1st phase. - // 1st phase: is_merge=true, only have one SlotRef. - // 2nd phase: is_merge=false, maybe have multiple exprs. - int _get_slot_column_id(const vectorized::AggFnEvaluator* evaluator); size_t _memory_usage() const; RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; @@ -296,7 +118,6 @@ protected: RuntimeProfile::Counter* _hash_table_input_counter = nullptr; RuntimeProfile::Counter* _build_timer = nullptr; RuntimeProfile::Counter* _expr_timer = nullptr; - RuntimeProfile::Counter* _build_table_convert_timer = nullptr; RuntimeProfile::Counter* _serialize_key_timer = nullptr; RuntimeProfile::Counter* _merge_timer = nullptr; RuntimeProfile::Counter* _serialize_data_timer = nullptr; @@ -346,6 +167,14 @@ public: return _is_colocate ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, _partition_exprs) : DataDistribution(ExchangeType::HASH_SHUFFLE, _partition_exprs); } + size_t get_revocable_mem_size(RuntimeState* state) const; + + vectorized::AggregatedDataVariants* get_agg_data(RuntimeState* state) { + auto& local_state = get_local_state(state); + return local_state._agg_data; + } + + Status reset_hash_table(RuntimeState* state); using DataSinkOperatorX::id; using DataSinkOperatorX::operator_id; @@ -379,7 +208,6 @@ protected: vectorized::VExprContextSPtrs _probe_expr_ctxs; ObjectPool* _pool = nullptr; std::vector _make_nullable_keys; - size_t _spill_partition_count_bits; int64_t _limit; // -1: no limit bool _have_conjuncts; diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index 15ff10ba3d..9c47f1c8cb 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -42,13 +42,19 @@ Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - _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); + + _merge_timer = ADD_TIMER(Base::profile(), "MergeTime"); + _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); + auto& p = _parent->template cast(); if (p._without_key) { if (p._needs_finalize) { @@ -76,6 +82,23 @@ Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { return Status::OK(); } +Status AggLocalState::_create_agg_status(vectorized::AggregateDataPtr data) { + auto& shared_state = *Base::_shared_state; + for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { + try { + shared_state.aggregate_evaluators[i]->create( + data + shared_state.offsets_of_aggregate_states[i]); + } catch (...) { + for (int j = 0; j < i; ++j) { + shared_state.aggregate_evaluators[j]->destroy( + data + shared_state.offsets_of_aggregate_states[j]); + } + throw; + } + } + return Status::OK(); +} + Status AggLocalState::_destroy_agg_status(vectorized::AggregateDataPtr data) { auto& shared_state = *Base::_shared_state; for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { @@ -87,70 +110,6 @@ Status AggLocalState::_destroy_agg_status(vectorized::AggregateDataPtr data) { Status AggLocalState::_serialize_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, bool* eos) { - if (_shared_state->spill_context.has_data) { - return _serialize_with_serialized_key_result_with_spilt_data(state, block, eos); - } else { - return _serialize_with_serialized_key_result_non_spill(state, block, eos); - } -} - -Status AggLocalState::_serialize_with_serialized_key_result_with_spilt_data( - RuntimeState* state, vectorized::Block* block, bool* eos) { - CHECK(!_shared_state->spill_context.stream_ids.empty()); - CHECK(_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) { - break; - } - RETURN_IF_ERROR(_reset_hash_table()); - RETURN_IF_ERROR(_merge_spilt_data()); - _shared_state->aggregate_data_container->init_once(); - } - - RETURN_IF_ERROR(_serialize_with_serialized_key_result_non_spill(state, block, eos)); - if (*eos) { - *eos = _shared_state->spill_context.read_cursor == - _shared_state->spill_partition_helper->partition_count; - } - CHECK(!block->empty() || *eos); - return Status::OK(); -} - -Status AggLocalState::_reset_hash_table() { - auto& ss = *Base::_shared_state; - return std::visit( - [&](auto&& agg_method) { - auto& hash_table = *agg_method.hash_table; - using HashTableType = std::decay_t; - - agg_method.reset(); - - hash_table.for_each_mapped([&](auto& mapped) { - if (mapped) { - static_cast(_destroy_agg_status(mapped)); - mapped = nullptr; - } - }); - - ss.aggregate_data_container = std::make_unique( - sizeof(typename HashTableType::key_type), - ((ss.total_size_of_aggregate_states + ss.align_aggregate_states - 1) / - ss.align_aggregate_states) * - ss.align_aggregate_states); - agg_method.hash_table.reset(new HashTableType()); - ss.agg_arena_pool = std::make_unique(); - return Status::OK(); - }, - ss.agg_data->method_variant); -} - -Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeState* state, - vectorized::Block* block, - bool* eos) { SCOPED_TIMER(_serialize_result_timer); auto& shared_state = *_shared_state; int key_size = _shared_state->probe_expr_ctxs.size(); @@ -243,7 +202,7 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta } } }, - _agg_data->method_variant); + shared_state.agg_data->method_variant); if (!mem_reuse) { vectorized::ColumnsWithTypeAndName columns_with_schema; @@ -263,63 +222,6 @@ Status AggLocalState::_serialize_with_serialized_key_result_non_spill(RuntimeSta Status AggLocalState::_get_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, bool* eos) { - if (_shared_state->spill_context.has_data) { - return _get_result_with_spilt_data(state, block, eos); - } else { - return _get_result_with_serialized_key_non_spill(state, block, eos); - } -} - -Status AggLocalState::_get_result_with_spilt_data(RuntimeState* state, vectorized::Block* block, - bool* eos) { - CHECK(!_shared_state->spill_context.stream_ids.empty()); - CHECK(_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) { - break; - } - RETURN_IF_ERROR(_reset_hash_table()); - RETURN_IF_ERROR(_merge_spilt_data()); - _shared_state->aggregate_data_container->init_once(); - } - - RETURN_IF_ERROR(_get_result_with_serialized_key_non_spill(state, block, eos)); - if (*eos) { - *eos = _shared_state->spill_context.read_cursor == - _shared_state->spill_partition_helper->partition_count; - } - CHECK(!block->empty() || *eos); - return Status::OK(); -} - -Status AggLocalState::_merge_spilt_data() { - CHECK(!_shared_state->spill_context.stream_ids.empty()); - - for (auto& reader : _shared_state->spill_context.readers) { - CHECK_LT(_shared_state->spill_context.read_cursor, reader->block_count()); - reader->seek(_shared_state->spill_context.read_cursor); - vectorized::Block block; - bool eos = false; - RETURN_IF_ERROR(reader->read(&block, &eos)); - - // TODO - // if (!block.empty()) { - // auto st = _merge_with_serialized_key_helper( - // &block); - // RETURN_IF_ERROR(st); - // } - } - _shared_state->spill_context.read_cursor++; - return Status::OK(); -} - -Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* state, - vectorized::Block* block, - bool* eos) { auto& shared_state = *_shared_state; // non-nullable column(id in `_make_nullable_keys`) will be converted to nullable. bool mem_reuse = shared_state.make_nullable_keys.empty() && block->mem_reuse(); @@ -404,7 +306,7 @@ Status AggLocalState::_get_result_with_serialized_key_non_spill(RuntimeState* st } } }, - _agg_data->method_variant); + shared_state.agg_data->method_variant); if (!mem_reuse) { *block = columns_with_schema; @@ -435,7 +337,7 @@ Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Bl } block->clear(); - DCHECK(_agg_data->without_key != nullptr); + DCHECK(shared_state.agg_data->without_key != nullptr); int agg_size = shared_state.aggregate_evaluators.size(); vectorized::MutableColumns value_columns(agg_size); @@ -449,7 +351,7 @@ Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Bl for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { shared_state.aggregate_evaluators[i]->function()->serialize_without_key_to_column( - _agg_data->without_key + shared_state.offsets_of_aggregate_states[i], + shared_state.agg_data->without_key + shared_state.offsets_of_aggregate_states[i], *value_columns[i]); } @@ -470,7 +372,7 @@ Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Bl Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::Block* block, bool* eos) { auto& shared_state = *_shared_state; - DCHECK(_agg_data->without_key != nullptr); + DCHECK(_shared_state->agg_data->without_key != nullptr); block->clear(); auto& p = _parent->cast(); @@ -487,7 +389,8 @@ Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::B for (int i = 0; i < shared_state.aggregate_evaluators.size(); ++i) { auto column = columns[i].get(); shared_state.aggregate_evaluators[i]->insert_result_info( - _agg_data->without_key + shared_state.offsets_of_aggregate_states[i], column); + shared_state.agg_data->without_key + shared_state.offsets_of_aggregate_states[i], + column); } const auto& block_schema = block->get_columns_with_type_and_name(); @@ -546,6 +449,174 @@ void AggLocalState::make_nullable_output_key(vectorized::Block* block) { } } +template +Status AggLocalState::merge_with_serialized_key_helper(vectorized::Block* block) { + SCOPED_TIMER(_merge_timer); + + 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) { + key_columns[i] = block->get_by_position(i).column.get(); + } + + int rows = block->rows(); + if (_places.size() < rows) { + _places.resize(rows); + } + + if constexpr (limit) { + _find_in_hash_table(_places.data(), key_columns, rows); + + for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { + if (Base::_shared_state->aggregate_evaluators[i]->is_merge()) { + int col_id = AggSharedState::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 = + 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); + Base::_shared_state->aggregate_evaluators[i] + ->function() + ->deserialize_and_merge_vec_selected( + _places.data(), _shared_state->offsets_of_aggregate_states[i], + _deserialize_buffer.data(), + (vectorized::ColumnString*)(column.get()), + _shared_state->agg_arena_pool.get(), rows); + } + } else { + RETURN_IF_ERROR( + Base::_shared_state->aggregate_evaluators[i]->execute_batch_add_selected( + block, _shared_state->offsets_of_aggregate_states[i], + _places.data(), _shared_state->agg_arena_pool.get())); + } + } + } else { + _emplace_into_hash_table(_places.data(), key_columns, rows); + + for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { + int col_id = 0; + col_id = Base::_shared_state->probe_expr_ctxs.size() + 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 = + 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); + Base::_shared_state->aggregate_evaluators[i]->function()->deserialize_and_merge_vec( + _places.data(), _shared_state->offsets_of_aggregate_states[i], + _deserialize_buffer.data(), (vectorized::ColumnString*)(column.get()), + _shared_state->agg_arena_pool.get(), rows); + } + } + + if (_should_limit_output) { + _reach_limit = _get_hash_table_size() >= + Base::_parent->template cast()._limit; + } + } + + return Status::OK(); +} +template +Status AggSourceOperatorX::merge_with_serialized_key_helper(RuntimeState* state, + vectorized::Block* block) { + auto& local_state = get_local_state(state); + return local_state.merge_with_serialized_key_helper(block); +} +template Status AggSourceOperatorX::merge_with_serialized_key_helper( + RuntimeState* state, vectorized::Block* block); +template Status AggSourceOperatorX::merge_with_serialized_key_helper( + RuntimeState* state, vectorized::Block* block); + +size_t AggLocalState::_get_hash_table_size() { + return std::visit([&](auto&& agg_method) { return agg_method.hash_table->size(); }, + _shared_state->agg_data->method_variant); +} + +void AggLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* places, + vectorized::ColumnRawPtrs& key_columns, + size_t num_rows) { + std::visit( + [&](auto&& agg_method) -> void { + SCOPED_TIMER(_hash_table_compute_timer); + using HashMethodType = std::decay_t; + using AggState = typename HashMethodType::State; + AggState state(key_columns); + agg_method.init_serialized_keys(key_columns, num_rows); + + auto creator = [this](const auto& ctor, auto& key, auto& origin) { + HashMethodType::try_presis_key_and_origin(key, origin, + *_shared_state->agg_arena_pool); + auto mapped = + Base::_shared_state->aggregate_data_container->append_data(origin); + auto st = _create_agg_status(mapped); + if (!st) { + throw Exception(st.code(), st.to_string()); + } + ctor(key, mapped); + }; + + auto creator_for_null_key = [&](auto& mapped) { + mapped = _shared_state->agg_arena_pool->aligned_alloc( + _shared_state->total_size_of_aggregate_states, + _shared_state->align_aggregate_states); + auto st = _create_agg_status(mapped); + if (!st) { + throw Exception(st.code(), st.to_string()); + } + }; + + SCOPED_TIMER(_hash_table_emplace_timer); + for (size_t i = 0; i < num_rows; ++i) { + places[i] = agg_method.lazy_emplace(state, i, creator, creator_for_null_key); + } + + COUNTER_UPDATE(_hash_table_input_counter, num_rows); + }, + _shared_state->agg_data->method_variant); +} + +void AggLocalState::_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 AggState = typename HashMethodType::State; + AggState state(key_columns); + agg_method.init_serialized_keys(key_columns, num_rows); + + /// For all rows. + for (size_t i = 0; i < num_rows; ++i) { + auto find_result = agg_method.find(state, i); + + if (find_result.is_found()) { + places[i] = find_result.get_mapped(); + } else { + places[i] = nullptr; + } + } + }, + _shared_state->agg_data->method_variant); +} + Status AggLocalState::close(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_close_timer); @@ -559,9 +630,14 @@ Status AggLocalState::close(RuntimeState* state) { [&](auto&& agg_method) { COUNTER_SET(_hash_table_size_counter, int64_t(agg_method.hash_table->size())); }, - _agg_data->method_variant); + _shared_state->agg_data->method_variant); } + vectorized::PODArray tmp_places; + _places.swap(tmp_places); + + std::vector tmp_deserialize_buffer; + _deserialize_buffer.swap(tmp_deserialize_buffer); return Base::close(state); } diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h index 077ac8f7db..1d1f564d41 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.h +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -20,7 +20,6 @@ #include "common/status.h" #include "operator.h" -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "vec/exec/vaggregation_node.h" @@ -61,6 +60,8 @@ public: Status close(RuntimeState* state) override; void make_nullable_output_key(vectorized::Block* block); + template + Status merge_with_serialized_key_helper(vectorized::Block* block); protected: friend class AggSourceOperatorX; @@ -71,18 +72,8 @@ protected: bool* eos); Status _serialize_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, bool* eos); - Status _get_result_with_serialized_key_non_spill(RuntimeState* state, vectorized::Block* block, - bool* eos); - Status _get_result_with_spilt_data(RuntimeState* state, vectorized::Block* block, bool* eos); - - Status _serialize_with_serialized_key_result_non_spill(RuntimeState* state, - vectorized::Block* block, bool* eos); - Status _serialize_with_serialized_key_result_with_spilt_data(RuntimeState* state, - vectorized::Block* block, - bool* eos); + Status _create_agg_status(vectorized::AggregateDataPtr data); Status _destroy_agg_status(vectorized::AggregateDataPtr data); - Status _reset_hash_table(); - Status _merge_spilt_data(); void _make_nullable_output_key(vectorized::Block* block) { if (block->rows() != 0) { auto& shared_state = *Base ::_shared_state; @@ -93,6 +84,14 @@ protected: } } } + void _find_in_hash_table(vectorized::AggregateDataPtr* places, + vectorized::ColumnRawPtrs& key_columns, size_t num_rows); + void _emplace_into_hash_table(vectorized::AggregateDataPtr* places, + vectorized::ColumnRawPtrs& key_columns, size_t num_rows); + size_t _get_hash_table_size(); + + vectorized::PODArray _places; + std::vector _deserialize_buffer; RuntimeProfile::Counter* _get_results_timer = nullptr; RuntimeProfile::Counter* _serialize_result_timer = nullptr; @@ -101,6 +100,12 @@ protected: RuntimeProfile::Counter* _serialize_data_timer = nullptr; RuntimeProfile::Counter* _hash_table_size_counter = nullptr; + RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; + RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; + RuntimeProfile::Counter* _hash_table_input_counter = nullptr; + RuntimeProfile::Counter* _merge_timer = nullptr; + RuntimeProfile::Counter* _deserialize_data_timer = nullptr; + using vectorized_get_result = std::function; @@ -110,7 +115,8 @@ protected: executor _executor; - vectorized::AggregatedDataVariants* _agg_data = nullptr; + bool _should_limit_output = false; + bool _reach_limit = false; }; class AggSourceOperatorX : public OperatorX { @@ -124,6 +130,9 @@ public: bool is_source() const override { return true; } + template + Status merge_with_serialized_key_helper(RuntimeState* state, vectorized::Block* block); + private: friend class AggLocalState; diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h index 17abe3d7ff..d469cf5bf2 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h @@ -23,7 +23,6 @@ #include #include "common/status.h" -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "util/runtime_profile.h" #include "vec/core/block.h" diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 0a6ed9f35a..94ae46690f 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -22,7 +22,6 @@ #include "exprs/bloom_filter_func.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/operator.h" -#include "pipeline/pipeline_x/dependency.h" #include "vec/exec/join/vhash_join_node.h" #include "vec/utils/template_helpers.hpp" diff --git a/be/src/pipeline/exec/join_build_sink_operator.cpp b/be/src/pipeline/exec/join_build_sink_operator.cpp index ffef9d4dee..44c34a3c05 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.cpp +++ b/be/src/pipeline/exec/join_build_sink_operator.cpp @@ -19,6 +19,7 @@ #include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/exec/nested_loop_join_build_operator.h" +#include "pipeline/exec/partitioned_hash_join_sink_operator.h" #include "pipeline/pipeline_x/operator.h" namespace doris::pipeline { @@ -126,5 +127,8 @@ template class JoinBuildSinkLocalState; template class JoinBuildSinkLocalState; +template class JoinBuildSinkOperatorX; +template class JoinBuildSinkLocalState; } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/join_build_sink_operator.h b/be/src/pipeline/exec/join_build_sink_operator.h index f3d8997684..9dbea12e02 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.h +++ b/be/src/pipeline/exec/join_build_sink_operator.h @@ -18,7 +18,6 @@ #pragma once #include "operator.h" -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "vec/exec/join/vjoin_node_base.h" diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 469ceeaff6..03b20fdb4d 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -20,6 +20,7 @@ #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/nested_loop_join_probe_operator.h" #include "pipeline/exec/operator.h" +#include "pipeline/exec/partitioned_hash_join_probe_operator.h" namespace doris::pipeline { @@ -239,4 +240,8 @@ template class JoinProbeOperatorX; template class JoinProbeLocalState; template class JoinProbeOperatorX; +template class JoinProbeLocalState; +template class JoinProbeOperatorX; + } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/join_probe_operator.h b/be/src/pipeline/exec/join_probe_operator.h index e9dfc773eb..9bb716ff36 100644 --- a/be/src/pipeline/exec/join_probe_operator.h +++ b/be/src/pipeline/exec/join_probe_operator.h @@ -18,7 +18,6 @@ #pragma once #include "operator.h" -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "vec/exec/join/vjoin_node_base.h" diff --git a/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp b/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp index 6b5506be26..de9cdeba04 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp @@ -17,8 +17,6 @@ #include "multi_cast_data_stream_sink.h" -#include "pipeline/pipeline_x/dependency.h" - namespace doris::pipeline { OperatorPtr MultiCastDataStreamSinkOperatorBuilder::build_operator() { diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.h b/be/src/pipeline/exec/multi_cast_data_stream_source.h index 4deab7a5ac..3af8c5507b 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -23,7 +23,6 @@ #include "common/status.h" #include "operator.h" -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "vec/exec/runtime_filter_consumer.h" diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index ffbe51d1b9..c93cc8f592 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -250,6 +250,10 @@ public: [[nodiscard]] virtual RuntimeProfile* get_runtime_profile() const = 0; + virtual size_t revocable_mem_size(RuntimeState* state) const { return 0; } + + virtual Status revoke_memory(RuntimeState* state) { return Status::OK(); }; + protected: OperatorBuilderBase* _operator_builder = nullptr; OperatorPtr _child; diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp new file mode 100644 index 0000000000..0a3eb7c706 --- /dev/null +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp @@ -0,0 +1,288 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "partitioned_aggregation_sink_operator.h" + +#include +#include + +#include "aggregation_sink_operator.h" +#include "common/status.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris::pipeline { +PartitionedAggSinkLocalState::PartitionedAggSinkLocalState(DataSinkOperatorXBase* parent, + RuntimeState* state) + : Base(parent, state) { + _finish_dependency = std::make_shared( + parent->operator_id(), parent->node_id(), parent->get_name() + "_FINISH_DEPENDENCY", + state->get_query_ctx()); +} +Status PartitionedAggSinkLocalState::init(doris::RuntimeState* state, + doris::pipeline::LocalSinkStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + + SCOPED_TIMER(Base::exec_time_counter()); + SCOPED_TIMER(Base::_open_timer); + + _init_counters(); + + auto& parent = Base::_parent->template cast(); + Base::_shared_state->init_spill_params(parent._spill_partition_count_bits); + + RETURN_IF_ERROR(setup_in_memory_agg_op(state)); + + for (const auto& probe_expr_ctx : Base::_shared_state->in_mem_shared_state->probe_expr_ctxs) { + key_columns_.emplace_back(probe_expr_ctx->root()->data_type()->create_column()); + } + for (const auto& aggregate_evaluator : + Base::_shared_state->in_mem_shared_state->aggregate_evaluators) { + value_data_types_.emplace_back(aggregate_evaluator->function()->get_serialized_type()); + value_columns_.emplace_back(aggregate_evaluator->function()->create_serialize_column()); + } + + _finish_dependency->block(); + return Status::OK(); +} + +Status PartitionedAggSinkLocalState::open(RuntimeState* state) { + SCOPED_TIMER(Base::exec_time_counter()); + SCOPED_TIMER(Base::_open_timer); + return Base::open(state); +} +Status PartitionedAggSinkLocalState::close(RuntimeState* state, Status exec_status) { + SCOPED_TIMER(Base::exec_time_counter()); + SCOPED_TIMER(Base::_close_timer); + if (Base::_closed) { + return Status::OK(); + } + { + std::unique_lock lk(_spill_lock); + if (_is_spilling) { + _spill_cv.wait(lk); + } + } + return Base::close(state, exec_status); +} + +void PartitionedAggSinkLocalState::_init_counters() { + _internal_runtime_profile = std::make_unique("internal_profile"); + + _hash_table_memory_usage = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "HashTable", + TUnit::BYTES, "MemoryUsage", 1); + _serialize_key_arena_memory_usage = Base::profile()->AddHighWaterMarkCounter( + "SerializeKeyArena", TUnit::BYTES, "MemoryUsage", 1); + + _build_timer = ADD_TIMER(Base::profile(), "BuildTime"); + _serialize_key_timer = ADD_TIMER(Base::profile(), "SerializeKeyTime"); + _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); + + _spill_serialize_hash_table_timer = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillSerializeHashTableTime", "Spill", 1); +} +#define UPDATE_PROFILE(counter, name) \ + do { \ + auto* child_counter = child_profile->get_counter(name); \ + if (child_counter != nullptr) { \ + COUNTER_SET(counter, child_counter->value()); \ + } \ + } while (false) + +void PartitionedAggSinkLocalState::update_profile(RuntimeProfile* child_profile) { + UPDATE_PROFILE(_hash_table_memory_usage, "HashTable"); + UPDATE_PROFILE(_serialize_key_arena_memory_usage, "SerializeKeyArena"); + UPDATE_PROFILE(_build_timer, "BuildTime"); + UPDATE_PROFILE(_serialize_key_timer, "SerializeKeyTime"); + UPDATE_PROFILE(_merge_timer, "MergeTime"); + UPDATE_PROFILE(_expr_timer, "MergeTime"); + UPDATE_PROFILE(_serialize_data_timer, "SerializeDataTime"); + UPDATE_PROFILE(_deserialize_data_timer, "DeserializeAndMergeTime"); + UPDATE_PROFILE(_hash_table_compute_timer, "HashTableComputeTime"); + UPDATE_PROFILE(_hash_table_emplace_timer, "HashTableEmplaceTime"); + UPDATE_PROFILE(_hash_table_input_counter, "HashTableInputCount"); + UPDATE_PROFILE(_max_row_size_counter, "MaxRowSizeInBytes"); +} + +PartitionedAggSinkOperatorX::PartitionedAggSinkOperatorX(ObjectPool* pool, int operator_id, + const TPlanNode& tnode, + const DescriptorTbl& descs) + : DataSinkOperatorX(operator_id, tnode.node_id) { + _agg_sink_operator = std::make_unique(pool, operator_id, tnode, descs); +} + +Status PartitionedAggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(DataSinkOperatorX::init(tnode, state)); + _name = "PARTITIONED_AGGREGATION_SINK_OPERATOR"; + if (state->query_options().__isset.external_agg_partition_bits) { + _spill_partition_count_bits = state->query_options().external_agg_partition_bits; + } + + _agg_sink_operator->set_dests_id(DataSinkOperatorX::dests_id()); + RETURN_IF_ERROR(_agg_sink_operator->set_child( + DataSinkOperatorX::_child_x)); + return _agg_sink_operator->init(tnode, state); +} + +Status PartitionedAggSinkOperatorX::prepare(RuntimeState* state) { + return _agg_sink_operator->prepare(state); +} + +Status PartitionedAggSinkOperatorX::open(RuntimeState* state) { + return _agg_sink_operator->open(state); +} + +Status PartitionedAggSinkOperatorX::close(RuntimeState* state) { + return _agg_sink_operator->close(state); +} +Status PartitionedAggSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in_block, + bool eos) { + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); + RETURN_IF_ERROR(local_state.Base::_shared_state->sink_status); + local_state._eos = eos; + auto* runtime_state = local_state._runtime_state.get(); + RETURN_IF_ERROR(_agg_sink_operator->sink(runtime_state, in_block, false)); + if (eos) { + LOG(INFO) << "agg node " << id() << " sink eos"; + if (revocable_mem_size(state) > 0) { + RETURN_IF_ERROR(revoke_memory(state)); + } else { + local_state._dependency->set_ready_to_read(); + } + } + if (local_state._runtime_state) { + auto* sink_local_state = local_state._runtime_state->get_sink_local_state(); + local_state.update_profile(sink_local_state->profile()); + } + return Status::OK(); +} +Status PartitionedAggSinkOperatorX::revoke_memory(RuntimeState* state) { + auto& local_state = get_local_state(state); + return local_state.revoke_memory(state); +} + +size_t PartitionedAggSinkOperatorX::revocable_mem_size(RuntimeState* state) const { + auto& local_state = get_local_state(state); + if (!local_state.Base::_shared_state->sink_status.ok()) { + return UINT64_MAX; + } + auto* runtime_state = local_state._runtime_state.get(); + auto size = _agg_sink_operator->get_revocable_mem_size(runtime_state); + return size; +} + +Status PartitionedAggSinkLocalState::setup_in_memory_agg_op(RuntimeState* state) { + _runtime_state = RuntimeState::create_unique( + nullptr, state->fragment_instance_id(), state->query_id(), state->fragment_id(), + state->query_options(), TQueryGlobals {}, state->exec_env(), state->get_query_ctx()); + _runtime_state->set_query_mem_tracker(state->query_mem_tracker()); + _runtime_state->set_task_execution_context(state->get_task_execution_context().lock()); + _runtime_state->set_be_number(state->be_number()); + + _runtime_state->set_desc_tbl(&state->desc_tbl()); + _runtime_state->set_pipeline_x_runtime_filter_mgr(state->local_runtime_filter_mgr()); + _runtime_state->set_task_id(state->task_id()); + + auto& parent = Base::_parent->template cast(); + Base::_shared_state->in_mem_shared_state_sptr = + parent._agg_sink_operator->create_shared_state(); + Base::_shared_state->in_mem_shared_state = + static_cast(Base::_shared_state->in_mem_shared_state_sptr.get()); + Base::_shared_state->in_mem_shared_state->enable_spill = true; + + LocalSinkStateInfo info {0, _internal_runtime_profile.get(), + -1, Base::_shared_state->in_mem_shared_state_sptr.get(), + {}, {}}; + RETURN_IF_ERROR(parent._agg_sink_operator->setup_local_state(_runtime_state.get(), info)); + + auto* sink_local_state = _runtime_state->get_sink_local_state(); + DCHECK(sink_local_state != nullptr); + return sink_local_state->open(state); +} + +Status PartitionedAggSinkLocalState::revoke_memory(RuntimeState* state) { + LOG(INFO) << "agg node " << Base::_parent->id() << " revoke_memory" + << ", eos: " << _eos; + RETURN_IF_ERROR(Base::_shared_state->sink_status); + DCHECK(!_is_spilling); + _is_spilling = true; + + // TODO: spill thread may set_ready before the task::execute thread put the task to blocked state + if (!_eos) { + Base::_dependency->Dependency::block(); + } + auto& parent = Base::_parent->template cast(); + Status status; + Defer defer {[&]() { + if (!status.ok()) { + _is_spilling = false; + if (!_eos) { + Base::_dependency->Dependency::set_ready(); + } + } + }}; + status = ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool()->submit_func( + [this, &parent, state] { + SCOPED_ATTACH_TASK(state); + SCOPED_TIMER(Base::_spill_timer); + Defer defer {[&]() { + if (!Base::_shared_state->sink_status.ok()) { + LOG(WARNING) + << "agg node " << Base::_parent->id() + << " revoke_memory error: " << Base::_shared_state->sink_status; + } else { + LOG(INFO) << " agg node " << Base::_parent->id() << " revoke_memory finish" + << ", eos: " << _eos; + } + { + std::unique_lock lk(_spill_lock); + _is_spilling = false; + if (_eos) { + Base::_dependency->set_ready_to_read(); + _finish_dependency->set_ready(); + } else { + Base::_dependency->Dependency::set_ready(); + } + _spill_cv.notify_one(); + } + }}; + auto* runtime_state = _runtime_state.get(); + auto* agg_data = parent._agg_sink_operator->get_agg_data(runtime_state); + Base::_shared_state->sink_status = std::visit( + [&](auto&& agg_method) -> Status { + auto& hash_table = *agg_method.hash_table; + return _spill_hash_table(state, agg_method, hash_table, _eos); + }, + agg_data->method_variant); + RETURN_IF_ERROR(Base::_shared_state->sink_status); + Base::_shared_state->sink_status = + parent._agg_sink_operator->reset_hash_table(runtime_state); + return Base::_shared_state->sink_status; + }); + return status; +} + +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h new file mode 100644 index 0000000000..9628fb5766 --- /dev/null +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h @@ -0,0 +1,342 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once +#include "aggregation_sink_operator.h" +#include "pipeline/pipeline_x/operator.h" +#include "vec/exprs/vexpr.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris::pipeline { +class PartitionedAggSinkOperatorX; +class PartitionedAggSinkLocalState + : public PipelineXSpillSinkLocalState { +public: + ENABLE_FACTORY_CREATOR(PartitionedAggSinkLocalState); + using Base = PipelineXSpillSinkLocalState; + using Parent = PartitionedAggSinkOperatorX; + + PartitionedAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); + ~PartitionedAggSinkLocalState() override = default; + + friend class PartitionedAggSinkOperatorX; + + Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + Status open(RuntimeState* state) override; + Status close(RuntimeState* state, Status exec_status) override; + Dependency* finishdependency() override { return _finish_dependency.get(); } + + Status revoke_memory(RuntimeState* state); + + Status setup_in_memory_agg_op(RuntimeState* state); + + void update_profile(RuntimeProfile* child_profile); + + template + struct TmpSpillInfo { + std::vector keys_; + std::vector values_; + }; + template + Status _spill_hash_table(RuntimeState* state, HashTableCtxType& context, + HashTableType& hash_table, bool eos) { + Status status; + Defer defer {[&]() { + if (!status.ok()) { + Base::_shared_state->close(); + } + }}; + + context.init_iterator(); + + Base::_shared_state->in_mem_shared_state->aggregate_data_container->init_once(); + + static int spill_batch_rows = 4096; + int row_count = 0; + + std::vector> spill_infos( + Base::_shared_state->partition_count); + auto& iter = Base::_shared_state->in_mem_shared_state->aggregate_data_container->iterator; + while (iter != Base::_shared_state->in_mem_shared_state->aggregate_data_container->end()) { + const auto& key = iter.template get_key(); + auto partition_index = Base::_shared_state->get_partition_index(hash_table.hash(key)); + spill_infos[partition_index].keys_.emplace_back(key); + spill_infos[partition_index].values_.emplace_back(iter.get_aggregate_data()); + + if (++row_count == spill_batch_rows) { + row_count = 0; + for (int i = 0; i < Base::_shared_state->partition_count && !state->is_cancelled(); + ++i) { + if (spill_infos[i].keys_.size() >= spill_batch_rows) { + status = _async_spill_partition_and_wait( + state, context, Base::_shared_state->spill_partitions[i], + spill_infos[i].keys_, spill_infos[i].values_, nullptr, false); + RETURN_IF_ERROR(status); + } + } + } + + ++iter; + } + auto hash_null_key_data = hash_table.has_null_key_data(); + for (int i = 0; i < Base::_shared_state->partition_count; ++i) { + auto spill_null_key_data = + (hash_null_key_data && i == Base::_shared_state->partition_count - 1); + if (spill_infos[i].keys_.size() > 0 || spill_null_key_data) { + status = _async_spill_partition_and_wait( + state, context, Base::_shared_state->spill_partitions[i], + spill_infos[i].keys_, spill_infos[i].values_, + spill_null_key_data ? hash_table.template get_null_key_data< + vectorized::AggregateDataPtr>() + : nullptr, + true); + RETURN_IF_ERROR(status); + } + } + + for (auto& partition : Base::_shared_state->spill_partitions) { + status = partition->finish_current_spilling(eos); + RETURN_IF_ERROR(status); + } + if (eos) { + _clear_tmp_data(); + } + return Status::OK(); + } + + template + Status _async_spill_partition_and_wait(RuntimeState* state, HashTableCtxType& context, + AggSpillPartitionSPtr& spill_partition, + std::vector& keys, + std::vector& values, + const vectorized::AggregateDataPtr null_key_data, + bool is_last) { + vectorized::SpillStreamSPtr spill_stream; + auto status = spill_partition->get_spill_stream(state, Base::_parent->node_id(), + Base::profile(), spill_stream); + RETURN_IF_ERROR(status); + spill_stream->set_write_counters(Base::_spill_serialize_block_timer, + Base::_spill_block_count, Base::_spill_data_size, + Base::_spill_write_disk_timer); + + status = to_block(context, keys, values, null_key_data); + RETURN_IF_ERROR(status); + + if (is_last) { + std::vector tmp_keys; + std::vector tmp_values; + keys.swap(tmp_keys); + values.swap(tmp_values); + + } else { + keys.clear(); + values.clear(); + } + + status = spill_stream->prepare_spill(); + RETURN_IF_ERROR(status); + + status = ExecEnv::GetInstance() + ->spill_stream_mgr() + ->get_spill_io_thread_pool(spill_stream->get_spill_root_dir()) + ->submit_func([this, state, &spill_stream] { + (void)state; // avoid ut compile error + SCOPED_ATTACH_TASK(state); + SCOPED_TIMER(_spill_write_disk_timer); + Status status; + Defer defer {[&]() { spill_stream->end_spill(status); }}; + status = spill_stream->spill_block(block_, false); + return status; + }); + if (!status.ok()) { + spill_stream->end_spill(status); + } + RETURN_IF_ERROR(status); + status = spill_partition->wait_spill(state); + _reset_tmp_data(); + return status; + } + + template + Status to_block(HashTableCtxType& context, std::vector& keys, + std::vector& values, + const vectorized::AggregateDataPtr null_key_data) { + SCOPED_TIMER(_spill_serialize_hash_table_timer); + context.insert_keys_into_columns(keys, key_columns_, keys.size()); + + if (null_key_data) { + // only one key of group by support wrap null key + // here need additional processing logic on the null key / value + CHECK(key_columns_.size() == 1); + CHECK(key_columns_[0]->is_nullable()); + key_columns_[0]->insert_data(nullptr, 0); + + values.emplace_back(null_key_data); + } + + for (size_t i = 0; + i < Base::_shared_state->in_mem_shared_state->aggregate_evaluators.size(); ++i) { + Base::_shared_state->in_mem_shared_state->aggregate_evaluators[i] + ->function() + ->serialize_to_column(values, + Base::_shared_state->in_mem_shared_state + ->offsets_of_aggregate_states[i], + value_columns_[i], values.size()); + } + + vectorized::ColumnsWithTypeAndName key_columns_with_schema; + for (int i = 0; i < key_columns_.size(); ++i) { + key_columns_with_schema.emplace_back( + std::move(key_columns_[i]), + Base::_shared_state->in_mem_shared_state->probe_expr_ctxs[i] + ->root() + ->data_type(), + Base::_shared_state->in_mem_shared_state->probe_expr_ctxs[i] + ->root() + ->expr_name()); + } + key_block_ = key_columns_with_schema; + + vectorized::ColumnsWithTypeAndName value_columns_with_schema; + for (int i = 0; i < value_columns_.size(); ++i) { + value_columns_with_schema.emplace_back( + std::move(value_columns_[i]), value_data_types_[i], + Base::_shared_state->in_mem_shared_state->aggregate_evaluators[i] + ->function() + ->get_name()); + } + value_block_ = value_columns_with_schema; + + for (const auto& column : key_block_.get_columns_with_type_and_name()) { + block_.insert(column); + } + for (const auto& column : value_block_.get_columns_with_type_and_name()) { + block_.insert(column); + } + return Status::OK(); + } + + void _reset_tmp_data() { + block_.clear(); + key_columns_.clear(); + value_columns_.clear(); + key_block_.clear_column_data(); + value_block_.clear_column_data(); + key_columns_ = key_block_.mutate_columns(); + value_columns_ = value_block_.mutate_columns(); + } + + void _clear_tmp_data() { + { + vectorized::Block empty_block; + block_.swap(empty_block); + } + { + vectorized::Block empty_block; + key_block_.swap(empty_block); + } + { + vectorized::Block empty_block; + value_block_.swap(empty_block); + } + { + vectorized::MutableColumns cols; + key_columns_.swap(cols); + } + { + vectorized::MutableColumns cols; + value_columns_.swap(cols); + } + + vectorized::DataTypes tmp_value_data_types; + value_data_types_.swap(tmp_value_data_types); + } + + void _init_counters(); + + std::unique_ptr _runtime_state; + + bool _eos = false; + std::shared_ptr _finish_dependency; + bool _is_spilling = false; + std::mutex _spill_lock; + std::condition_variable _spill_cv; + + // temp structures during spilling + vectorized::MutableColumns key_columns_; + vectorized::MutableColumns value_columns_; + vectorized::DataTypes value_data_types_; + vectorized::Block block_; + vectorized::Block key_block_; + vectorized::Block value_block_; + + std::unique_ptr _internal_runtime_profile; + RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; + RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; + RuntimeProfile::Counter* _hash_table_input_counter = nullptr; + RuntimeProfile::Counter* _build_timer = nullptr; + RuntimeProfile::Counter* _expr_timer = nullptr; + RuntimeProfile::Counter* _serialize_key_timer = nullptr; + RuntimeProfile::Counter* _merge_timer = nullptr; + RuntimeProfile::Counter* _serialize_data_timer = nullptr; + RuntimeProfile::Counter* _deserialize_data_timer = nullptr; + RuntimeProfile::Counter* _max_row_size_counter = nullptr; + RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; + RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage = nullptr; + + RuntimeProfile::Counter* _spill_serialize_hash_table_timer = nullptr; +}; + +class PartitionedAggSinkOperatorX : public DataSinkOperatorX { +public: + PartitionedAggSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, + const DescriptorTbl& descs); + ~PartitionedAggSinkOperatorX() override = default; + Status init(const TDataSink& tsink) override { + return Status::InternalError("{} should not init with TPlanNode", + DataSinkOperatorX::_name); + } + + Status init(const TPlanNode& tnode, RuntimeState* state) override; + + Status prepare(RuntimeState* state) override; + + Status open(RuntimeState* state) override; + + Status close(RuntimeState* state) override; + + Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos) override; + + DataDistribution required_data_distribution() const override { + return _agg_sink_operator->required_data_distribution(); + } + + Status set_child(OperatorXPtr child) override { + RETURN_IF_ERROR(DataSinkOperatorX::set_child(child)); + return _agg_sink_operator->set_child(child); + } + size_t revocable_mem_size(RuntimeState* state) const override; + + Status revoke_memory(RuntimeState* state) override; + +private: + friend class PartitionedAggSinkLocalState; + std::unique_ptr _agg_sink_operator; + + size_t _spill_partition_count_bits = 4; +}; +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp new file mode 100644 index 0000000000..909de78478 --- /dev/null +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp @@ -0,0 +1,253 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "partitioned_aggregation_source_operator.h" + +#include + +#include "aggregation_source_operator.h" +#include "common/exception.h" +#include "common/status.h" +#include "pipeline/exec/operator.h" +#include "util/runtime_profile.h" +#include "vec//utils/util.hpp" +#include "vec/spill/spill_stream_manager.h" + +namespace doris::pipeline { + +PartitionedAggLocalState::PartitionedAggLocalState(RuntimeState* state, OperatorXBase* parent) + : Base(state, parent) {} + +Status PartitionedAggLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_open_timer); + _init_counters(); + return Status::OK(); +} + +Status PartitionedAggLocalState::open(RuntimeState* state) { + if (_opened) { + return Status::OK(); + } + _opened = true; + RETURN_IF_ERROR(setup_in_memory_agg_op(state)); + return Base::open(state); +} + +void PartitionedAggLocalState::_init_counters() { + _internal_runtime_profile = std::make_unique("internal_profile"); + _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); + + _merge_timer = ADD_TIMER(profile(), "MergeTime"); + _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); +} + +#define UPDATE_PROFILE(counter, name) \ + do { \ + auto* child_counter = child_profile->get_counter(name); \ + if (child_counter != nullptr) { \ + COUNTER_SET(counter, child_counter->value()); \ + } \ + } while (false) + +void PartitionedAggLocalState::update_profile(RuntimeProfile* child_profile) { + UPDATE_PROFILE(_get_results_timer, "GetResultsTime"); + UPDATE_PROFILE(_serialize_result_timer, "SerializeResultTime"); + UPDATE_PROFILE(_hash_table_iterate_timer, "HashTableIterateTime"); + UPDATE_PROFILE(_insert_keys_to_column_timer, "InsertKeysToColumnTime"); + UPDATE_PROFILE(_serialize_data_timer, "SerializeDataTime"); + UPDATE_PROFILE(_hash_table_size_counter, "HashTableSize"); +} + +Status PartitionedAggLocalState::close(RuntimeState* state) { + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_close_timer); + if (_closed) { + return Status::OK(); + } + { + std::unique_lock lk(_merge_spill_lock); + if (_is_merging) { + _merge_spill_cv.wait(lk); + } + } + return Base::close(state); +} +PartitionedAggSourceOperatorX::PartitionedAggSourceOperatorX(ObjectPool* pool, + const TPlanNode& tnode, + int operator_id, + const DescriptorTbl& descs) + : Base(pool, tnode, operator_id, descs) { + _agg_source_operator = std::make_unique(pool, tnode, operator_id, descs); +} + +Status PartitionedAggSourceOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::init(tnode, state)); + _op_name = "PARTITIONED_AGGREGATION_OPERATOR"; + return _agg_source_operator->init(tnode, state); +} + +Status PartitionedAggSourceOperatorX::prepare(RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::prepare(state)); + return _agg_source_operator->prepare(state); +} + +Status PartitionedAggSourceOperatorX::open(RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::open(state)); + return _agg_source_operator->open(state); +} + +Status PartitionedAggSourceOperatorX::close(RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::close(state)); + return _agg_source_operator->close(state); +} + +Status PartitionedAggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, + bool* eos) { + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + RETURN_IF_ERROR(local_state._status); + + RETURN_IF_ERROR(local_state.initiate_merge_spill_partition_agg_data(state)); + auto* runtime_state = local_state._runtime_state.get(); + RETURN_IF_ERROR(_agg_source_operator->get_block(runtime_state, block, eos)); + if (local_state._runtime_state) { + auto* source_local_state = + local_state._runtime_state->get_local_state(_agg_source_operator->operator_id()); + local_state.update_profile(source_local_state->profile()); + } + if (*eos) { + if (!local_state._shared_state->spill_partitions.empty()) { + *eos = false; + } + } + local_state.reached_limit(block, eos); + return Status::OK(); +} + +Status PartitionedAggLocalState::setup_in_memory_agg_op(RuntimeState* state) { + _runtime_state = RuntimeState::create_unique( + nullptr, state->fragment_instance_id(), state->query_id(), state->fragment_id(), + state->query_options(), TQueryGlobals {}, state->exec_env(), state->get_query_ctx()); + _runtime_state->set_query_mem_tracker(state->query_mem_tracker()); + _runtime_state->set_task_execution_context(state->get_task_execution_context().lock()); + _runtime_state->set_be_number(state->be_number()); + + _runtime_state->set_desc_tbl(&state->desc_tbl()); + _runtime_state->resize_op_id_to_local_state(state->max_operator_id()); + _runtime_state->set_pipeline_x_runtime_filter_mgr(state->local_runtime_filter_mgr()); + + auto& parent = Base::_parent->template cast(); + + DCHECK(Base::_shared_state->in_mem_shared_state); + LocalStateInfo state_info { + _internal_runtime_profile.get(), {}, Base::_shared_state->in_mem_shared_state, {}, 0}; + + RETURN_IF_ERROR( + parent._agg_source_operator->setup_local_state(_runtime_state.get(), state_info)); + + auto* source_local_state = + _runtime_state->get_local_state(parent._agg_source_operator->operator_id()); + DCHECK(source_local_state != nullptr); + return source_local_state->open(state); +} + +Status PartitionedAggLocalState::initiate_merge_spill_partition_agg_data(RuntimeState* state) { + DCHECK(!_is_merging); + Base::_shared_state->in_mem_shared_state->aggregate_data_container->init_once(); + if (Base::_shared_state->in_mem_shared_state->aggregate_data_container->iterator != + Base::_shared_state->in_mem_shared_state->aggregate_data_container->end() || + _shared_state->spill_partitions.empty()) { + return Status::OK(); + } + + _is_merging = true; + LOG(INFO) << "agg node " << _parent->node_id() << " merge spilled agg data"; + + RETURN_IF_ERROR(Base::_shared_state->in_mem_shared_state->reset_hash_table()); + _dependency->Dependency::block(); + + RETURN_IF_ERROR( + ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool()->submit_func( + [this, state] { + SCOPED_ATTACH_TASK(state); + Defer defer {[&]() { + if (!_status.ok()) { + LOG(WARNING) << "agg node " << _parent->node_id() + << " merge spilled agg data error: " << _status; + } else if (_shared_state->spill_partitions.empty()) { + LOG(INFO) << "agg node " << _parent->node_id() + << " merge spilled agg data finish"; + } + Base::_shared_state->in_mem_shared_state->aggregate_data_container + ->init_once(); + { + std::unique_lock lk(_merge_spill_lock); + _is_merging = false; + _dependency->Dependency::set_ready(); + _merge_spill_cv.notify_one(); + } + }}; + bool has_agg_data = false; + auto& parent = Base::_parent->template cast(); + while (!state->is_cancelled() && !has_agg_data && + !_shared_state->spill_partitions.empty()) { + for (auto& stream : + _shared_state->spill_partitions[0]->spill_streams_) { + stream->set_read_counters(Base::_spill_read_data_time, + Base::_spill_deserialize_time, + Base::_spill_read_bytes); + vectorized::Block block; + bool eos = false; + while (!eos) { + { + SCOPED_TIMER(Base::_spill_recover_time); + _status = stream->read_next_block_sync(&block, &eos); + } + RETURN_IF_ERROR(_status); + + if (!block.empty()) { + has_agg_data = true; + _status = parent._agg_source_operator + ->merge_with_serialized_key_helper( + _runtime_state.get(), &block); + RETURN_IF_ERROR(_status); + } + } + (void)ExecEnv::GetInstance() + ->spill_stream_mgr() + ->delete_spill_stream(stream); + } + _shared_state->spill_partitions.pop_front(); + } + if (_shared_state->spill_partitions.empty()) { + _shared_state->close(); + } + return _status; + })); + return Status::OK(); +} +} // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h new file mode 100644 index 0000000000..cc408c07e2 --- /dev/null +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h @@ -0,0 +1,106 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +#pragma once + +#include + +#include "common/status.h" +#include "operator.h" +#include "pipeline/pipeline_x/operator.h" + +namespace doris { +class ExecNode; +class RuntimeState; + +namespace pipeline { + +class PartitionedAggSourceOperatorX; +class PartitionedAggLocalState; + +class PartitionedAggLocalState final : public PipelineXSpillLocalState { +public: + ENABLE_FACTORY_CREATOR(PartitionedAggLocalState); + using Base = PipelineXSpillLocalState; + using Parent = PartitionedAggSourceOperatorX; + PartitionedAggLocalState(RuntimeState* state, OperatorXBase* parent); + ~PartitionedAggLocalState() override = default; + + Status init(RuntimeState* state, LocalStateInfo& info) override; + Status open(RuntimeState* state) override; + Status close(RuntimeState* state) override; + + Status initiate_merge_spill_partition_agg_data(RuntimeState* state); + Status setup_in_memory_agg_op(RuntimeState* state); + + void update_profile(RuntimeProfile* child_profile); + +protected: + void _init_counters(); + + friend class PartitionedAggSourceOperatorX; + std::unique_ptr _runtime_state; + + bool _opened = false; + Status _status; + std::unique_ptr> _spill_merge_promise; + std::future _spill_merge_future; + bool _current_partition_eos = true; + bool _is_merging = false; + std::mutex _merge_spill_lock; + std::condition_variable _merge_spill_cv; + + std::unique_ptr _internal_runtime_profile; + RuntimeProfile::Counter* _get_results_timer = nullptr; + RuntimeProfile::Counter* _serialize_result_timer = nullptr; + RuntimeProfile::Counter* _hash_table_iterate_timer = nullptr; + RuntimeProfile::Counter* _insert_keys_to_column_timer = nullptr; + RuntimeProfile::Counter* _serialize_data_timer = nullptr; + RuntimeProfile::Counter* _hash_table_size_counter = nullptr; + + RuntimeProfile::Counter* _merge_timer = nullptr; + RuntimeProfile::Counter* _deserialize_data_timer = nullptr; + RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; + RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; + RuntimeProfile::Counter* _hash_table_input_counter = nullptr; +}; +class AggSourceOperatorX; +class PartitionedAggSourceOperatorX : public OperatorX { +public: + using Base = OperatorX; + PartitionedAggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, + const DescriptorTbl& descs); + ~PartitionedAggSourceOperatorX() override = default; + + Status init(const TPlanNode& tnode, RuntimeState* state) override; + Status prepare(RuntimeState* state) override; + + Status open(RuntimeState* state) override; + + Status close(RuntimeState* state) override; + + Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos) override; + + bool is_source() const override { return true; } + +private: + friend class PartitionedAggLocalState; + Status _initiate_merge_spill_partition_agg_data(RuntimeState* state); + + std::unique_ptr _agg_source_operator; +}; +} // namespace pipeline +} // namespace doris diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp new file mode 100644 index 0000000000..f37f28d59c --- /dev/null +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp @@ -0,0 +1,724 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "partitioned_hash_join_probe_operator.h" + +#include "util/mem_info.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris::pipeline { + +PartitionedHashJoinProbeLocalState::PartitionedHashJoinProbeLocalState(RuntimeState* state, + OperatorXBase* parent) + : JoinProbeLocalState(state, parent) {} + +Status PartitionedHashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(JoinProbeLocalState::init(state, info)); + _internal_runtime_profile.reset(new RuntimeProfile("internal_profile")); + auto& p = _parent->cast(); + + _partitioned_blocks.resize(p._partition_count); + _probe_spilling_streams.resize(p._partition_count); + _partitioner = std::make_unique(p._partition_count); + RETURN_IF_ERROR(_partitioner->init(p._probe_exprs)); + RETURN_IF_ERROR(_partitioner->prepare(state, p._child_x->row_desc())); + + _spill_and_partition_label = ADD_LABEL_COUNTER(profile(), "SpillAndPartition"); + _partition_timer = ADD_CHILD_TIMER(profile(), "PartitionTime", "SpillAndPartition"); + _partition_shuffle_timer = + ADD_CHILD_TIMER(profile(), "PartitionShuffleTime", "SpillAndPartition"); + _spill_build_rows = + ADD_CHILD_COUNTER(profile(), "SpillBuildRows", TUnit::UNIT, "SpillAndPartition"); + _recovery_build_rows = + ADD_CHILD_COUNTER(profile(), "RecoveryBuildRows", TUnit::UNIT, "SpillAndPartition"); + _spill_probe_rows = + ADD_CHILD_COUNTER(profile(), "SpillProbeRows", TUnit::UNIT, "SpillAndPartition"); + _recovery_probe_rows = + ADD_CHILD_COUNTER(profile(), "RecoveryProbeRows", TUnit::UNIT, "SpillAndPartition"); + _spill_build_blocks = + ADD_CHILD_COUNTER(profile(), "SpillBuildBlocks", TUnit::UNIT, "SpillAndPartition"); + _recovery_build_blocks = + ADD_CHILD_COUNTER(profile(), "RecoveryBuildBlocks", TUnit::UNIT, "SpillAndPartition"); + _spill_probe_blocks = + ADD_CHILD_COUNTER(profile(), "SpillProbeBlocks", TUnit::UNIT, "SpillAndPartition"); + _recovery_probe_blocks = + ADD_CHILD_COUNTER(profile(), "RecoveryProbeBlocks", TUnit::UNIT, "SpillAndPartition"); + + // Build phase + _build_phase_label = ADD_LABEL_COUNTER(profile(), "BuildPhase"); + _build_rows_counter = ADD_CHILD_COUNTER(profile(), "BuildRows", TUnit::UNIT, "BuildPhase"); + _publish_runtime_filter_timer = + ADD_CHILD_TIMER(profile(), "PublishRuntimeFilterTime", "BuildPhase"); + _runtime_filter_compute_timer = + ADD_CHILD_TIMER(profile(), "RuntimeFilterComputeTime", "BuildPhase"); + _build_table_timer = ADD_CHILD_TIMER(profile(), "BuildTableTime", "BuildPhase"); + _build_side_merge_block_timer = + ADD_CHILD_TIMER(profile(), "BuildSideMergeBlockTime", "BuildPhase"); + _build_table_insert_timer = ADD_CHILD_TIMER(profile(), "BuildTableInsertTime", "BuildPhase"); + _build_expr_call_timer = ADD_CHILD_TIMER(profile(), "BuildExprCallTime", "BuildPhase"); + _build_side_compute_hash_timer = + ADD_CHILD_TIMER(profile(), "BuildSideHashComputingTime", "BuildPhase"); + _allocate_resource_timer = ADD_CHILD_TIMER(profile(), "AllocateResourceTime", "BuildPhase"); + + // Probe phase + _probe_phase_label = ADD_LABEL_COUNTER(profile(), "ProbePhase"); + _probe_next_timer = ADD_CHILD_TIMER(profile(), "ProbeFindNextTime", "ProbePhase"); + _probe_expr_call_timer = ADD_CHILD_TIMER(profile(), "ProbeExprCallTime", "ProbePhase"); + _search_hashtable_timer = + ADD_CHILD_TIMER(profile(), "ProbeWhenSearchHashTableTime", "ProbePhase"); + _build_side_output_timer = + ADD_CHILD_TIMER(profile(), "ProbeWhenBuildSideOutputTime", "ProbePhase"); + _probe_side_output_timer = + ADD_CHILD_TIMER(profile(), "ProbeWhenProbeSideOutputTime", "ProbePhase"); + _probe_process_hashtable_timer = + ADD_CHILD_TIMER(profile(), "ProbeWhenProcessHashTableTime", "ProbePhase"); + _process_other_join_conjunct_timer = + ADD_CHILD_TIMER(profile(), "OtherJoinConjunctTime", "ProbePhase"); + _init_probe_side_timer = ADD_CHILD_TIMER(profile(), "InitProbeSideTime", "ProbePhase"); + return Status::OK(); +} +#define UPDATE_PROFILE(counter, name) \ + do { \ + auto* child_counter = child_profile->get_counter(name); \ + if (child_counter != nullptr) { \ + COUNTER_UPDATE(counter, child_counter->value()); \ + } \ + } while (false) + +void PartitionedHashJoinProbeLocalState::update_build_profile(RuntimeProfile* child_profile) { + UPDATE_PROFILE(_build_rows_counter, "BuildRows"); + UPDATE_PROFILE(_publish_runtime_filter_timer, "PublishRuntimeFilterTime"); + UPDATE_PROFILE(_runtime_filter_compute_timer, "RuntimeFilterComputeTime"); + UPDATE_PROFILE(_build_table_timer, "BuildTableTime"); + UPDATE_PROFILE(_build_side_merge_block_timer, "BuildSideMergeBlockTime"); + UPDATE_PROFILE(_build_table_insert_timer, "BuildTableInsertTime"); + UPDATE_PROFILE(_build_expr_call_timer, "BuildExprCallTime"); + UPDATE_PROFILE(_build_side_compute_hash_timer, "BuildSideHashComputingTime"); + UPDATE_PROFILE(_allocate_resource_timer, "AllocateResourceTime"); +} + +void PartitionedHashJoinProbeLocalState::update_probe_profile(RuntimeProfile* child_profile) { + UPDATE_PROFILE(_probe_timer, "ProbeTime"); + UPDATE_PROFILE(_join_filter_timer, "JoinFilterTimer"); + UPDATE_PROFILE(_build_output_block_timer, "BuildOutputBlock"); + UPDATE_PROFILE(_probe_rows_counter, "ProbeRows"); + UPDATE_PROFILE(_probe_next_timer, "ProbeFindNextTime"); + UPDATE_PROFILE(_probe_expr_call_timer, "ProbeExprCallTime"); + UPDATE_PROFILE(_search_hashtable_timer, "ProbeWhenSearchHashTableTime"); + UPDATE_PROFILE(_build_side_output_timer, "ProbeWhenBuildSideOutputTime"); + UPDATE_PROFILE(_probe_side_output_timer, "ProbeWhenProbeSideOutputTime"); + UPDATE_PROFILE(_probe_process_hashtable_timer, "ProbeWhenProcessHashTableTime"); + UPDATE_PROFILE(_process_other_join_conjunct_timer, "OtherJoinConjunctTime"); + UPDATE_PROFILE(_init_probe_side_timer, "InitProbeSideTime"); +} + +#undef UPDATE_PROFILE + +Status PartitionedHashJoinProbeLocalState::open(RuntimeState* state) { + RETURN_IF_ERROR(PipelineXLocalStateBase::open(state)); + return _partitioner->open(state); +} +Status PartitionedHashJoinProbeLocalState::close(RuntimeState* state) { + RETURN_IF_ERROR(JoinProbeLocalState::close(state)); + return Status::OK(); +} + +Status PartitionedHashJoinProbeLocalState::spill_build_block(RuntimeState* state, + uint32_t partition_index) { + auto& partitioned_build_blocks = _shared_state->partitioned_build_blocks; + auto& mutable_block = partitioned_build_blocks[partition_index]; + if (!mutable_block || mutable_block->rows() == 0) { + --_spilling_task_count; + return Status::OK(); + } + + auto& build_spilling_stream = _shared_state->spilled_streams[partition_index]; + if (!build_spilling_stream) { + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( + state, build_spilling_stream, print_id(state->query_id()), "hash_build_sink", + _parent->id(), std::numeric_limits::max(), + std::numeric_limits::max(), _runtime_profile.get())); + RETURN_IF_ERROR(build_spilling_stream->prepare_spill()); + } + + auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool( + build_spilling_stream->get_spill_root_dir()); + return spill_io_pool->submit_func([state, &build_spilling_stream, &mutable_block, this] { + (void)state; // avoid ut compile error + SCOPED_ATTACH_TASK(state); + if (_spill_status_ok) { + auto build_block = mutable_block->to_block(); + DCHECK_EQ(mutable_block->rows(), 0); + auto st = build_spilling_stream->spill_block(build_block, false); + if (!st.ok()) { + std::unique_lock lock(_spill_lock); + _spill_status_ok = false; + _spill_status = std::move(st); + } else { + COUNTER_UPDATE(_spill_build_rows, build_block.rows()); + COUNTER_UPDATE(_spill_build_blocks, 1); + } + } + --_spilling_task_count; + + if (_spilling_task_count == 0) { + std::unique_lock lock(_spill_lock); + _dependency->set_ready(); + } + }); +} + +Status PartitionedHashJoinProbeLocalState::spill_probe_blocks(RuntimeState* state, + uint32_t partition_index) { + auto& spilling_stream = _probe_spilling_streams[partition_index]; + if (!spilling_stream) { + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( + state, spilling_stream, print_id(state->query_id()), "hash_probe", _parent->id(), + std::numeric_limits::max(), std::numeric_limits::max(), + _runtime_profile.get())); + RETURN_IF_ERROR(spilling_stream->prepare_spill()); + } + + auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool( + spilling_stream->get_spill_root_dir()); + + auto& blocks = _probe_blocks[partition_index]; + + if (!blocks.empty()) { + return spill_io_pool->submit_func([state, &blocks, &spilling_stream, this] { + (void)state; // avoid ut compile error + SCOPED_ATTACH_TASK(state); + for (auto& block : blocks) { + if (_spill_status_ok) { + auto st = spilling_stream->spill_block(block, false); + if (!st.ok()) { + std::unique_lock lock(_spill_lock); + _spill_status_ok = false; + _spill_status = std::move(st); + break; + } + COUNTER_UPDATE(_spill_probe_rows, block.rows()); + } else { + break; + } + } + + COUNTER_UPDATE(_spill_probe_blocks, blocks.size()); + blocks.clear(); + --_spilling_task_count; + + if (_spilling_task_count == 0) { + std::unique_lock lock(_spill_lock); + _dependency->set_ready(); + } + }); + } else { + --_spilling_task_count; + if (_spilling_task_count == 0) { + std::unique_lock lock(_spill_lock); + _dependency->set_ready(); + } + } + return Status::OK(); +} + +Status PartitionedHashJoinProbeLocalState::finish_spilling(uint32_t partition_index) { + auto& build_spilling_stream = _shared_state->spilled_streams[partition_index]; + if (build_spilling_stream) { + build_spilling_stream->end_spill(Status::OK()); + RETURN_IF_ERROR(build_spilling_stream->spill_eof()); + } + + auto& probe_spilling_stream = _probe_spilling_streams[partition_index]; + + if (probe_spilling_stream) { + probe_spilling_stream->end_spill(Status::OK()); + RETURN_IF_ERROR(probe_spilling_stream->spill_eof()); + } + + return Status::OK(); +} + +Status PartitionedHashJoinProbeLocalState::recovery_build_blocks_from_disk(RuntimeState* state, + uint32_t partition_index, + bool& has_data) { + auto& spilled_stream = _shared_state->spilled_streams[partition_index]; + has_data = false; + if (!spilled_stream) { + return Status::OK(); + } + + auto& mutable_block = _shared_state->partitioned_build_blocks[partition_index]; + DCHECK(mutable_block != nullptr); + + auto read_func = [this, state, &spilled_stream, &mutable_block] { + Defer defer([this] { --_spilling_task_count; }); + (void)state; // avoid ut compile error + SCOPED_ATTACH_TASK(state); + DCHECK_EQ(_spill_status_ok.load(), true); + + bool eos = false; + while (!eos) { + vectorized::Block block; + auto st = spilled_stream->read_next_block_sync(&block, &eos); + if (!st.ok()) { + std::unique_lock lock(_spill_lock); + _spill_status_ok = false; + _spill_status = std::move(st); + break; + } + COUNTER_UPDATE(_recovery_build_rows, block.rows()); + COUNTER_UPDATE(_recovery_build_blocks, 1); + + if (block.empty()) { + continue; + } + + if (mutable_block->empty()) { + *mutable_block = std::move(block); + } else { + st = mutable_block->merge(std::move(block)); + if (!st.ok()) { + std::unique_lock lock(_spill_lock); + _spill_status_ok = false; + _spill_status = std::move(st); + break; + } + } + } + + ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(spilled_stream); + spilled_stream.reset(); + _dependency->set_ready(); + }; + + auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool(); + has_data = true; + _dependency->block(); + + ++_spilling_task_count; + auto st = spill_io_pool->submit_func(read_func); + if (!st.ok()) { + --_spilling_task_count; + } + return st; +} + +Status PartitionedHashJoinProbeLocalState::recovery_probe_blocks_from_disk(RuntimeState* state, + uint32_t partition_index, + bool& has_data) { + auto& spilled_stream = _probe_spilling_streams[partition_index]; + has_data = false; + if (!spilled_stream) { + return Status::OK(); + } + + auto& blocks = _probe_blocks[partition_index]; + + /// TODO: maybe recovery more blocks each time. + auto read_func = [this, state, &spilled_stream, &blocks] { + Defer defer([this] { --_spilling_task_count; }); + (void)state; // avoid ut compile error + SCOPED_ATTACH_TASK(state); + DCHECK_EQ(_spill_status_ok.load(), true); + + vectorized::Block block; + bool eos = false; + auto st = spilled_stream->read_next_block_sync(&block, &eos); + if (!st.ok()) { + std::unique_lock lock(_spill_lock); + _spill_status_ok = false; + _spill_status = std::move(st); + } else { + COUNTER_UPDATE(_recovery_probe_rows, block.rows()); + COUNTER_UPDATE(_recovery_probe_blocks, 1); + blocks.emplace_back(std::move(block)); + } + + if (eos) { + ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(spilled_stream); + spilled_stream.reset(); + } + + _dependency->set_ready(); + }; + + auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool(); + DCHECK(spill_io_pool != nullptr); + _dependency->block(); + has_data = true; + ++_spilling_task_count; + auto st = spill_io_pool->submit_func(read_func); + if (!st.ok()) { + --_spilling_task_count; + } + return st; +} + +PartitionedHashJoinProbeOperatorX::PartitionedHashJoinProbeOperatorX(ObjectPool* pool, + const TPlanNode& tnode, + int operator_id, + const DescriptorTbl& descs, + uint32_t partition_count) + : JoinProbeOperatorX(pool, tnode, operator_id, descs), + _join_distribution(tnode.hash_join_node.__isset.dist_type ? tnode.hash_join_node.dist_type + : TJoinDistributionType::NONE), + _distribution_partition_exprs(tnode.__isset.distribute_expr_lists + ? tnode.distribute_expr_lists[0] + : std::vector {}), + _tnode(tnode), + _descriptor_tbl(descs), + _partition_count(partition_count) {} + +Status PartitionedHashJoinProbeOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(JoinProbeOperatorX::init(tnode, state)); + _op_name = "PARTITIONED_HASH_JOIN_PROBE_OPERATOR"; + auto tnode_ = _tnode; + tnode_.runtime_filters.clear(); + + for (auto& conjunct : tnode.hash_join_node.eq_join_conjuncts) { + _probe_exprs.emplace_back(conjunct.left); + } + + _sink_operator = + std::make_unique(_pool, 0, tnode_, _descriptor_tbl, false); + _probe_operator = std::make_unique(_pool, tnode_, 0, _descriptor_tbl); + RETURN_IF_ERROR(_sink_operator->init(tnode_, state)); + return _probe_operator->init(tnode_, state); +} +Status PartitionedHashJoinProbeOperatorX::prepare(RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::prepare(state)); + RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_expr_ctxs, state, *_intermediate_row_desc)); + RETURN_IF_ERROR(_probe_operator->set_child(_child_x)); + RETURN_IF_ERROR(_probe_operator->set_child(_build_side_child)); + RETURN_IF_ERROR(_sink_operator->set_child(_build_side_child)); + RETURN_IF_ERROR(_probe_operator->prepare(state)); + RETURN_IF_ERROR(_sink_operator->prepare(state)); + return Status::OK(); +} + +Status PartitionedHashJoinProbeOperatorX::open(RuntimeState* state) { + RETURN_IF_ERROR(JoinProbeOperatorX::open(state)); + RETURN_IF_ERROR(_probe_operator->open(state)); + RETURN_IF_ERROR(_sink_operator->open(state)); + return Status::OK(); +} + +Status PartitionedHashJoinProbeOperatorX::push(RuntimeState* state, vectorized::Block* input_block, + bool eos) const { + auto& local_state = get_local_state(state); + const auto rows = input_block->rows(); + auto& partitioned_blocks = local_state._partitioned_blocks; + if (rows == 0) { + if (eos) { + for (uint32_t i = 0; i != _partition_count; ++i) { + if (partitioned_blocks[i] && !partitioned_blocks[i]->empty()) { + local_state._probe_blocks[i].emplace_back(partitioned_blocks[i]->to_block()); + partitioned_blocks[i].reset(); + } + } + } + return Status::OK(); + } + { + SCOPED_TIMER(local_state._partition_timer); + RETURN_IF_ERROR(local_state._partitioner->do_partitioning(state, input_block, + local_state._mem_tracker.get())); + } + + std::vector partition_indexes[_partition_count]; + auto* channel_ids = reinterpret_cast(local_state._partitioner->get_channel_ids()); + for (uint32_t i = 0; i != rows; ++i) { + partition_indexes[channel_ids[i]].emplace_back(i); + } + + SCOPED_TIMER(local_state._partition_shuffle_timer); + for (uint32_t i = 0; i != _partition_count; ++i) { + const auto count = partition_indexes[i].size(); + if (UNLIKELY(count == 0)) { + continue; + } + + if (!partitioned_blocks[i]) { + partitioned_blocks[i] = + vectorized::MutableBlock::create_unique(input_block->clone_empty()); + } + partitioned_blocks[i]->add_rows(input_block, &(partition_indexes[i][0]), + &(partition_indexes[i][count])); + + if (partitioned_blocks[i]->rows() > 2 * 1024 * 1024 || + (eos && partitioned_blocks[i]->rows() > 0)) { + local_state._probe_blocks[i].emplace_back(partitioned_blocks[i]->to_block()); + partitioned_blocks[i].reset(); + } + } + + return Status::OK(); +} + +Status PartitionedHashJoinProbeOperatorX::_setup_internal_operators( + PartitionedHashJoinProbeLocalState& local_state, RuntimeState* state) const { + if (local_state._runtime_state) { + _update_profile_from_internal_states(local_state); + } + + local_state._runtime_state = RuntimeState::create_unique( + nullptr, state->fragment_instance_id(), state->query_id(), state->fragment_id(), + state->query_options(), TQueryGlobals {}, state->exec_env(), state->get_query_ctx()); + + local_state._runtime_state->set_query_mem_tracker(state->query_mem_tracker()); + + local_state._runtime_state->set_task_execution_context( + state->get_task_execution_context().lock()); + local_state._runtime_state->set_be_number(state->be_number()); + + local_state._runtime_state->set_desc_tbl(&state->desc_tbl()); + local_state._runtime_state->resize_op_id_to_local_state(-1); + local_state._runtime_state->set_pipeline_x_runtime_filter_mgr( + state->local_runtime_filter_mgr()); + + local_state._in_mem_shared_state_sptr = _sink_operator->create_shared_state(); + + // set sink local state + LocalSinkStateInfo info {0, local_state._internal_runtime_profile.get(), + -1, local_state._in_mem_shared_state_sptr.get(), + {}, {}}; + RETURN_IF_ERROR(_sink_operator->setup_local_state(local_state._runtime_state.get(), info)); + + LocalStateInfo state_info {local_state._internal_runtime_profile.get(), + {}, + local_state._in_mem_shared_state_sptr.get(), + {}, + 0}; + RETURN_IF_ERROR( + _probe_operator->setup_local_state(local_state._runtime_state.get(), state_info)); + + auto* sink_local_state = local_state._runtime_state->get_sink_local_state(); + DCHECK(sink_local_state != nullptr); + RETURN_IF_ERROR(sink_local_state->open(state)); + + auto* probe_local_state = + local_state._runtime_state->get_local_state(_probe_operator->operator_id()); + DCHECK(probe_local_state != nullptr); + RETURN_IF_ERROR(probe_local_state->open(state)); + + auto& partitioned_block = + local_state._shared_state->partitioned_build_blocks[local_state._partition_cursor]; + vectorized::Block block; + if (partitioned_block && partitioned_block->rows() > 0) { + block = partitioned_block->to_block(); + partitioned_block.reset(); + } + RETURN_IF_ERROR(_sink_operator->sink(local_state._runtime_state.get(), &block, true)); + return Status::OK(); +} + +Status PartitionedHashJoinProbeOperatorX::pull(doris::RuntimeState* state, + vectorized::Block* output_block, bool* eos) const { + auto& local_state = get_local_state(state); + if (!local_state._spill_status_ok) { + DCHECK_NE(local_state._spill_status.code(), 0); + return local_state._spill_status; + } + + if (_should_revoke_memory(state)) { + bool wait_for_io = false; + RETURN_IF_ERROR((const_cast(this)) + ->_revoke_memory(state, wait_for_io)); + if (wait_for_io) { + return Status::OK(); + } + } + + if (local_state._need_to_setup_internal_operators) { + *eos = false; + bool has_data = false; + CHECK_EQ(local_state._dependency->is_blocked_by(), nullptr); + RETURN_IF_ERROR(local_state.recovery_build_blocks_from_disk( + state, local_state._partition_cursor, has_data)); + if (has_data) { + return Status::OK(); + } + RETURN_IF_ERROR(_setup_internal_operators(local_state, state)); + local_state._need_to_setup_internal_operators = false; + } + + auto partition_index = local_state._partition_cursor; + bool in_mem_eos_; + auto* runtime_state = local_state._runtime_state.get(); + auto& probe_blocks = local_state._probe_blocks[partition_index]; + while (_probe_operator->need_more_input_data(runtime_state)) { + if (probe_blocks.empty()) { + *eos = false; + bool has_data = false; + RETURN_IF_ERROR( + local_state.recovery_probe_blocks_from_disk(state, partition_index, has_data)); + if (!has_data) { + vectorized::Block block; + RETURN_IF_ERROR(_probe_operator->push(runtime_state, &block, true)); + break; + } else { + return Status::OK(); + } + } + + auto block = std::move(probe_blocks.back()); + probe_blocks.pop_back(); + RETURN_IF_ERROR(_probe_operator->push(runtime_state, &block, false)); + } + + RETURN_IF_ERROR( + _probe_operator->pull(local_state._runtime_state.get(), output_block, &in_mem_eos_)); + + *eos = false; + if (in_mem_eos_) { + local_state._partition_cursor++; + if (local_state._partition_cursor == _partition_count) { + *eos = true; + } else { + RETURN_IF_ERROR(local_state.finish_spilling(local_state._partition_cursor)); + local_state._need_to_setup_internal_operators = true; + } + } + + return Status::OK(); +} + +bool PartitionedHashJoinProbeOperatorX::need_more_input_data(RuntimeState* state) const { + auto& local_state = get_local_state(state); + return !local_state._child_eos; +} + +bool PartitionedHashJoinProbeOperatorX::need_data_from_children(RuntimeState* state) const { + auto& local_state = get_local_state(state); + if (local_state._spilling_task_count != 0) { + return true; + } + + return JoinProbeOperatorX::need_data_from_children(state); +} + +size_t PartitionedHashJoinProbeOperatorX::revocable_mem_size(RuntimeState* state) const { + auto& local_state = get_local_state(state); + size_t mem_size = 0; + + auto& partitioned_build_blocks = local_state._shared_state->partitioned_build_blocks; + auto& probe_blocks = local_state._probe_blocks; + for (uint32_t i = local_state._partition_cursor + 1; i < _partition_count; ++i) { + auto& build_block = partitioned_build_blocks[i]; + if (build_block && build_block->rows() > 0) { + mem_size += build_block->allocated_bytes(); + } + + for (auto& block : probe_blocks[i]) { + mem_size += block.allocated_bytes(); + } + } + return mem_size; +} + +Status PartitionedHashJoinProbeOperatorX::_revoke_memory(RuntimeState* state, bool& wait_for_io) { + auto& local_state = get_local_state(state); + wait_for_io = false; + if (_partition_count > (local_state._partition_cursor + 1)) { + local_state._spilling_task_count = + (_partition_count - local_state._partition_cursor - 1) * 2; + } else { + return Status::OK(); + } + + for (uint32_t i = local_state._partition_cursor + 1; i < _partition_count; ++i) { + RETURN_IF_ERROR(local_state.spill_build_block(state, i)); + RETURN_IF_ERROR(local_state.spill_probe_blocks(state, i)); + } + + if (local_state._spilling_task_count > 0) { + std::unique_lock lock(local_state._spill_lock); + if (local_state._spilling_task_count > 0) { + local_state._dependency->block(); + wait_for_io = true; + } + } + return Status::OK(); +} + +bool PartitionedHashJoinProbeOperatorX::_should_revoke_memory(RuntimeState* state) const { + auto sys_mem_available = MemInfo::sys_mem_available(); + auto sys_mem_warning_water_mark = doris::MemInfo::sys_mem_available_warning_water_mark(); + + if (sys_mem_available < + sys_mem_warning_water_mark * config::spill_mem_warning_water_mark_multiplier) { + const auto revocable_size = revocable_mem_size(state); + const auto min_revocable_size = state->min_revocable_mem(); + return min_revocable_size > 0 && revocable_size > min_revocable_size; + } + return false; +} + +void PartitionedHashJoinProbeOperatorX::_update_profile_from_internal_states( + PartitionedHashJoinProbeLocalState& local_state) const { + if (local_state._runtime_state) { + auto* sink_local_state = local_state._runtime_state->get_sink_local_state(); + local_state.update_build_profile(sink_local_state->profile()); + auto* probe_local_state = + local_state._runtime_state->get_local_state(_probe_operator->operator_id()); + local_state.update_probe_profile(probe_local_state->profile()); + } +} + +Status PartitionedHashJoinProbeOperatorX::get_block(RuntimeState* state, vectorized::Block* block, + bool* eos) { + *eos = false; + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + if (need_more_input_data(state)) { + local_state._child_block->clear_column_data(); + + if (_should_revoke_memory(state)) { + bool wait_for_io = false; + RETURN_IF_ERROR(_revoke_memory(state, wait_for_io)); + if (wait_for_io) { + return Status::OK(); + } + } + + RETURN_IF_ERROR(_child_x->get_block_after_projects(state, local_state._child_block.get(), + &local_state._child_eos)); + + if (local_state._child_eos) { + RETURN_IF_ERROR(local_state.finish_spilling(0)); + } else if (local_state._child_block->rows() == 0) { + return Status::OK(); + } + { + SCOPED_TIMER(local_state.exec_time_counter()); + RETURN_IF_ERROR(push(state, local_state._child_block.get(), local_state._child_eos)); + } + } + + if (!need_more_input_data(state)) { + SCOPED_TIMER(local_state.exec_time_counter()); + RETURN_IF_ERROR(pull(state, block, eos)); + local_state.add_num_rows_returned(block->rows()); + if (*eos) { + _update_profile_from_internal_states(local_state); + } + } + return Status::OK(); +} + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h new file mode 100644 index 0000000000..27c4a9e0bb --- /dev/null +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h @@ -0,0 +1,194 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "common/status.h" +#include "operator.h" +#include "pipeline/exec/hashjoin_build_sink.h" +#include "pipeline/exec/hashjoin_probe_operator.h" +#include "pipeline/exec/join_build_sink_operator.h" +#include "pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h" // LocalExchangeChannelIds +#include "pipeline/pipeline_x/operator.h" +#include "vec/runtime/partitioner.h" + +namespace doris { +class RuntimeState; + +namespace pipeline { + +using PartitionerType = vectorized::XXHashPartitioner; + +class PartitionedHashJoinProbeOperatorX; + +class PartitionedHashJoinProbeLocalState final + : public JoinProbeLocalState { +public: + using Parent = PartitionedHashJoinProbeOperatorX; + ENABLE_FACTORY_CREATOR(PartitionedHashJoinProbeLocalState); + PartitionedHashJoinProbeLocalState(RuntimeState* state, OperatorXBase* parent); + ~PartitionedHashJoinProbeLocalState() override = default; + + void add_tuple_is_null_column(vectorized::Block* block) override {} + + Status init(RuntimeState* state, LocalStateInfo& info) override; + Status open(RuntimeState* state) override; + Status close(RuntimeState* state) override; + + Status spill_build_block(RuntimeState* state, uint32_t partition_index); + Status spill_probe_blocks(RuntimeState* state, uint32_t partition_index); + + Status recovery_build_blocks_from_disk(RuntimeState* state, uint32_t partition_index, + bool& has_data); + Status recovery_probe_blocks_from_disk(RuntimeState* state, uint32_t partition_index, + bool& has_data); + + Status finish_spilling(uint32_t partition_index); + + void update_build_profile(RuntimeProfile* child_profile); + void update_probe_profile(RuntimeProfile* child_profile); + + friend class PartitionedHashJoinProbeOperatorX; + +private: + std::shared_ptr _in_mem_shared_state_sptr; + uint32_t _partition_cursor {0}; + + std::mutex _spill_lock; + Status _spill_status; + + std::atomic _spilling_task_count {0}; + std::atomic _spill_status_ok {true}; + + std::vector> _partitioned_blocks; + std::map> _probe_blocks; + + std::vector _probe_spilling_streams; + + std::unique_ptr _partitioner; + std::unique_ptr _runtime_state; + std::unique_ptr _internal_runtime_profile; + + bool _need_to_setup_internal_operators {true}; + + RuntimeProfile::Counter* _spill_and_partition_label = nullptr; + RuntimeProfile::Counter* _partition_timer = nullptr; + RuntimeProfile::Counter* _partition_shuffle_timer = nullptr; + RuntimeProfile::Counter* _spill_build_rows = nullptr; + RuntimeProfile::Counter* _spill_build_blocks = nullptr; + RuntimeProfile::Counter* _recovery_build_rows = nullptr; + RuntimeProfile::Counter* _recovery_build_blocks = nullptr; + RuntimeProfile::Counter* _spill_probe_rows = nullptr; + RuntimeProfile::Counter* _spill_probe_blocks = nullptr; + RuntimeProfile::Counter* _recovery_probe_rows = nullptr; + RuntimeProfile::Counter* _recovery_probe_blocks = nullptr; + + RuntimeProfile::Counter* _build_phase_label = nullptr; + RuntimeProfile::Counter* _build_rows_counter = nullptr; + RuntimeProfile::Counter* _publish_runtime_filter_timer = nullptr; + RuntimeProfile::Counter* _runtime_filter_compute_timer = nullptr; + + RuntimeProfile::Counter* _build_table_timer = nullptr; + RuntimeProfile::Counter* _build_expr_call_timer = nullptr; + RuntimeProfile::Counter* _build_table_insert_timer = nullptr; + RuntimeProfile::Counter* _build_side_compute_hash_timer = nullptr; + RuntimeProfile::Counter* _build_side_merge_block_timer = nullptr; + + RuntimeProfile::Counter* _allocate_resource_timer = nullptr; + + RuntimeProfile::Counter* _probe_phase_label = nullptr; + RuntimeProfile::Counter* _probe_expr_call_timer = nullptr; + RuntimeProfile::Counter* _probe_next_timer = nullptr; + RuntimeProfile::Counter* _probe_side_output_timer = nullptr; + RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; + RuntimeProfile::Counter* _search_hashtable_timer = nullptr; + RuntimeProfile::Counter* _init_probe_side_timer = nullptr; + RuntimeProfile::Counter* _build_side_output_timer = nullptr; + RuntimeProfile::Counter* _process_other_join_conjunct_timer = nullptr; +}; + +class PartitionedHashJoinProbeOperatorX final + : public JoinProbeOperatorX { +public: + PartitionedHashJoinProbeOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, + const DescriptorTbl& descs, uint32_t partition_count); + Status init(const TPlanNode& tnode, RuntimeState* state) override; + Status prepare(RuntimeState* state) override; + Status open(RuntimeState* state) override; + + [[nodiscard]] Status get_block(RuntimeState* state, vectorized::Block* block, + bool* eos) override; + + Status push(RuntimeState* state, vectorized::Block* input_block, bool eos) const override; + Status pull(doris::RuntimeState* state, vectorized::Block* output_block, + bool* eos) const override; + + bool need_more_input_data(RuntimeState* state) const override; + DataDistribution required_data_distribution() const override { + if (_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { + return {ExchangeType::NOOP}; + } + return (_join_distribution == TJoinDistributionType::BUCKET_SHUFFLE || + _join_distribution == TJoinDistributionType::COLOCATE + ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, + _distribution_partition_exprs) + : DataDistribution(ExchangeType::HASH_SHUFFLE, + _distribution_partition_exprs)); + } + + bool is_shuffled_hash_join() const override { + return _join_distribution == TJoinDistributionType::PARTITIONED; + } + + size_t revocable_mem_size(RuntimeState* state) const override; + + bool need_data_from_children(RuntimeState* state) const override; + +private: + Status _revoke_memory(RuntimeState* state, bool& wait_for_io); + + friend class PartitionedHashJoinProbeLocalState; + + [[nodiscard]] Status _setup_internal_operators(PartitionedHashJoinProbeLocalState& local_state, + RuntimeState* state) const; + + bool _should_revoke_memory(RuntimeState* state) const; + + void _update_profile_from_internal_states( + PartitionedHashJoinProbeLocalState& local_state) const; + + const TJoinDistributionType::type _join_distribution; + + std::unique_ptr _sink_operator; + std::unique_ptr _probe_operator; + + // probe expr + std::vector _probe_exprs; + + const std::vector _distribution_partition_exprs; + + const TPlanNode _tnode; + const DescriptorTbl _descriptor_tbl; + + const uint32_t _partition_count; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp new file mode 100644 index 0000000000..5cbf2b15ec --- /dev/null +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp @@ -0,0 +1,223 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "partitioned_hash_join_sink_operator.h" + +#include "pipeline/exec/operator.h" +#include "util/mem_info.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris::pipeline { + +Status PartitionedHashJoinSinkLocalState::init(doris::RuntimeState* state, + doris::pipeline::LocalSinkStateInfo& info) { + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); + auto& p = _parent->cast(); + _shared_state->partitioned_build_blocks.resize(p._partition_count); + _shared_state->spilled_streams.resize(p._partition_count); + + _partitioner = std::make_unique(p._partition_count); + RETURN_IF_ERROR(_partitioner->init(p._build_exprs)); + + _partition_timer = ADD_TIMER(profile(), "PartitionTime"); + _partition_shuffle_timer = ADD_TIMER(profile(), "PartitionShuffleTime"); + + return _partitioner->prepare(state, p._child_x->row_desc()); +} + +Status PartitionedHashJoinSinkLocalState::open(RuntimeState* state) { + RETURN_IF_ERROR(PipelineXSinkLocalState::open(state)); + return _partitioner->open(state); +} + +Status PartitionedHashJoinSinkLocalState::revoke_memory(RuntimeState* state) { + DCHECK_EQ(_spilling_streams_count, 0); + _spilling_streams_count = _shared_state->partitioned_build_blocks.size(); + for (size_t i = 0; i != _shared_state->partitioned_build_blocks.size(); ++i) { + vectorized::SpillStreamSPtr& spilling_stream = _shared_state->spilled_streams[i]; + auto& mutable_block = _shared_state->partitioned_build_blocks[i]; + + if (!mutable_block || mutable_block->rows() == 0) { + continue; + } + + if (!spilling_stream) { + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( + state, spilling_stream, print_id(state->query_id()), "hash_build_sink", + _parent->id(), std::numeric_limits::max(), + std::numeric_limits::max(), _profile)); + RETURN_IF_ERROR(spilling_stream->prepare_spill()); + } + + auto* spill_io_pool = + ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool(); + DCHECK(spill_io_pool != nullptr); + auto st = spill_io_pool->submit_func([this, state, spilling_stream, i] { + (void)state; // avoid ut compile error + SCOPED_ATTACH_TASK(state); + _spill_to_disk(i, spilling_stream); + }); + + if (!st.ok()) { + --_spilling_streams_count; + return st; + } + } + + if (_spilling_streams_count > 0) { + std::unique_lock lock(_spill_lock); + if (_spilling_streams_count > 0) { + _dependency->block(); + } + } + return Status::OK(); +} + +void PartitionedHashJoinSinkLocalState::_spill_to_disk( + uint32_t partition_index, const vectorized::SpillStreamSPtr& spilling_stream) { + auto& partitioned_block = _shared_state->partitioned_build_blocks[partition_index]; + + if (_spill_status_ok) { + auto block = partitioned_block->to_block(); + partitioned_block = vectorized::MutableBlock::create_unique(block.clone_empty()); + auto st = spilling_stream->spill_block(block, false); + if (!st.ok()) { + _spill_status_ok = false; + std::lock_guard l(_spill_status_lock); + _spill_status = st; + } + } + + --_spilling_streams_count; + DCHECK_GE(_spilling_streams_count, 0); + + if (_spilling_streams_count == 0) { + std::unique_lock lock(_spill_lock); + _dependency->set_ready(); + } +} + +PartitionedHashJoinSinkOperatorX::PartitionedHashJoinSinkOperatorX( + ObjectPool* pool, int operator_id, const TPlanNode& tnode, const DescriptorTbl& descs, + bool use_global_rf, uint32_t partition_count) + : JoinBuildSinkOperatorX(pool, operator_id, tnode, + descs), + _join_distribution(tnode.hash_join_node.__isset.dist_type ? tnode.hash_join_node.dist_type + : TJoinDistributionType::NONE), + _distribution_partition_exprs(tnode.__isset.distribute_expr_lists + ? tnode.distribute_expr_lists[1] + : std::vector {}), + _tnode(tnode), + _descriptor_tbl(descs), + _partition_count(partition_count) {} + +Status PartitionedHashJoinSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(JoinBuildSinkOperatorX::init(tnode, state)); + _name = "PARTITIONED_HASH_JOIN_SINK_OPERATOR"; + const std::vector& eq_join_conjuncts = tnode.hash_join_node.eq_join_conjuncts; + std::vector partition_exprs; + for (const auto& eq_join_conjunct : eq_join_conjuncts) { + vectorized::VExprContextSPtr ctx; + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(eq_join_conjunct.right, ctx)); + _build_exprs.emplace_back(eq_join_conjunct.right); + partition_exprs.emplace_back(eq_join_conjunct.right); + } + + return Status::OK(); +} + +Status PartitionedHashJoinSinkOperatorX::prepare(RuntimeState* state) { + return Status::OK(); +} + +Status PartitionedHashJoinSinkOperatorX::open(RuntimeState* state) { + return Status::OK(); +} + +Status PartitionedHashJoinSinkOperatorX::sink(RuntimeState* state, vectorized::Block* in_block, + bool eos) { + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + if (!local_state._spill_status_ok) { + DCHECK_NE(local_state._spill_status.code(), 0); + return local_state._spill_status; + } + + const auto rows = in_block->rows(); + + if (rows > 0) { + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); + /// TODO: DO NOT execute build exprs twice(when partition and building hash table) + { + SCOPED_TIMER(local_state._partition_timer); + RETURN_IF_ERROR(local_state._partitioner->do_partitioning( + state, in_block, local_state._mem_tracker.get())); + } + + SCOPED_TIMER(local_state._partition_shuffle_timer); + auto* channel_ids = + reinterpret_cast(local_state._partitioner->get_channel_ids()); + std::vector partition_indexes[_partition_count]; + for (uint32_t i = 0; i != rows; ++i) { + partition_indexes[channel_ids[i]].emplace_back(i); + } + + auto& partitioned_blocks = local_state._shared_state->partitioned_build_blocks; + for (uint32_t i = 0; i != _partition_count; ++i) { + const auto count = partition_indexes[i].size(); + if (UNLIKELY(count == 0)) { + continue; + } + + if (UNLIKELY(!partitioned_blocks[i])) { + partitioned_blocks[i] = + vectorized::MutableBlock::create_unique(in_block->clone_empty()); + } + partitioned_blocks[i]->add_rows(in_block, &(partition_indexes[i][0]), + &(partition_indexes[i][count])); + } + } + + if (eos) { + local_state._dependency->set_ready_to_read(); + } + + return Status::OK(); +} + +size_t PartitionedHashJoinSinkOperatorX::revocable_mem_size(RuntimeState* state) const { + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + auto& partitioned_blocks = local_state._shared_state->partitioned_build_blocks; + + size_t mem_size = 0; + for (uint32_t i = 0; i != _partition_count; ++i) { + auto& block = partitioned_blocks[i]; + if (block) { + mem_size += block->allocated_bytes(); + } + } + return mem_size; +} + +Status PartitionedHashJoinSinkOperatorX::revoke_memory(RuntimeState* state) { + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + return local_state.revoke_memory(state); +} + +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h new file mode 100644 index 0000000000..152b36459b --- /dev/null +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h @@ -0,0 +1,129 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "common/status.h" +#include "operator.h" +#include "pipeline/exec/hashjoin_build_sink.h" +#include "pipeline/exec/hashjoin_probe_operator.h" +#include "pipeline/exec/join_build_sink_operator.h" +#include "pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h" // LocalExchangeChannelIds +#include "pipeline/pipeline_x/operator.h" +#include "vec/runtime/partitioner.h" + +namespace doris { +class ExecNode; +class RuntimeState; + +namespace pipeline { + +using PartitionerType = vectorized::XXHashPartitioner; + +class PartitionedHashJoinSinkOperatorX; + +class PartitionedHashJoinSinkLocalState + : public PipelineXSinkLocalState { +public: + using Parent = PartitionedHashJoinSinkOperatorX; + ENABLE_FACTORY_CREATOR(PartitionedHashJoinSinkLocalState); + ~PartitionedHashJoinSinkLocalState() override = default; + Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + Status open(RuntimeState* state) override; + Status revoke_memory(RuntimeState* state); + +protected: + PartitionedHashJoinSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalState(parent, state) {} + + void _spill_to_disk(uint32_t partition_index, + const vectorized::SpillStreamSPtr& spilling_stream); + + friend class PartitionedHashJoinSinkOperatorX; + + std::atomic_int _spilling_streams_count {0}; + std::atomic _spill_status_ok {true}; + std::mutex _spill_lock; + + Status _spill_status; + std::mutex _spill_status_lock; + + std::unique_ptr _partitioner; + + RuntimeProfile::Counter* _partition_timer = nullptr; + RuntimeProfile::Counter* _partition_shuffle_timer = nullptr; +}; + +class PartitionedHashJoinSinkOperatorX + : public JoinBuildSinkOperatorX { +public: + PartitionedHashJoinSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, + const DescriptorTbl& descs, bool use_global_rf, + uint32_t partition_count); + + Status init(const TDataSink& tsink) override { + return Status::InternalError("{} should not init with TDataSink", + PartitionedHashJoinSinkOperatorX::_name); + } + + Status init(const TPlanNode& tnode, RuntimeState* state) override; + + Status prepare(RuntimeState* state) override; + Status open(RuntimeState* state) override; + + Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos) override; + + bool should_dry_run(RuntimeState* state) override { return false; } + + size_t revocable_mem_size(RuntimeState* state) const override; + + Status revoke_memory(RuntimeState* state) override; + + DataDistribution required_data_distribution() const override { + if (_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { + return {ExchangeType::NOOP}; + } + + return _join_distribution == TJoinDistributionType::BUCKET_SHUFFLE || + _join_distribution == TJoinDistributionType::COLOCATE + ? DataDistribution(ExchangeType::BUCKET_HASH_SHUFFLE, + _distribution_partition_exprs) + : DataDistribution(ExchangeType::HASH_SHUFFLE, + _distribution_partition_exprs); + } + + bool is_shuffled_hash_join() const override { + return _join_distribution == TJoinDistributionType::PARTITIONED; + } + +private: + friend class PartitionedHashJoinSinkLocalState; + + const TJoinDistributionType::type _join_distribution; + + std::vector _build_exprs; + + const std::vector _distribution_partition_exprs; + const TPlanNode _tnode; + const DescriptorTbl _descriptor_tbl; + const uint32_t _partition_count; +}; + +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index d01be2272f..564f05eef4 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -20,7 +20,6 @@ #include #include "operator.h" -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" #include "vec/sink/vresult_sink.h" diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index e86f4915de..657dd7d62d 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -149,10 +149,9 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); if (in_block->rows() > 0) { + COUNTER_UPDATE(local_state._sort_blocks_memory_usage, (int64_t)in_block->bytes()); RETURN_IF_ERROR(local_state._shared_state->sorter->append_block(in_block)); local_state._mem_tracker->set_consumption(local_state._shared_state->sorter->data_size()); - COUNTER_SET(local_state._sort_blocks_memory_usage, - (int64_t)local_state._shared_state->sorter->data_size()); RETURN_IF_CANCELLED(state); // update runtime predicate @@ -176,4 +175,25 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in return Status::OK(); } +size_t SortSinkOperatorX::get_revocable_mem_size(RuntimeState* state) const { + auto& local_state = get_local_state(state); + return local_state._shared_state->sorter->data_size(); +} + +Status SortSinkOperatorX::prepare_for_spill(RuntimeState* state) { + auto& local_state = get_local_state(state); + return local_state._shared_state->sorter->prepare_for_read(); +} + +Status SortSinkOperatorX::merge_sort_read_for_spill(RuntimeState* state, + doris::vectorized::Block* block, int batch_size, + bool* eos) { + auto& local_state = get_local_state(state); + return local_state._shared_state->sorter->merge_sort_read_for_spill(state, block, batch_size, + eos); +} +void SortSinkOperatorX::reset(RuntimeState* state) { + auto& local_state = get_local_state(state); + local_state._shared_state->sorter->reset(); +} } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index c737e82e1e..f23437f72f 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -96,6 +96,16 @@ public: return DataSinkOperatorX::required_data_distribution(); } + bool is_full_sort() const { return _algorithm == SortAlgorithm::FULL_SORT; } + + size_t get_revocable_mem_size(RuntimeState* state) const; + + Status prepare_for_spill(RuntimeState* state); + + Status merge_sort_read_for_spill(RuntimeState* state, doris::vectorized::Block* block, + int batch_size, bool* eos); + void reset(RuntimeState* state); + private: friend class SortSinkLocalState; diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp index af1dc66bc4..f7d6b6ae56 100644 --- a/be/src/pipeline/exec/sort_source_operator.cpp +++ b/be/src/pipeline/exec/sort_source_operator.cpp @@ -41,4 +41,10 @@ Status SortSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* bl return Status::OK(); } +const vectorized::SortDescription& SortSourceOperatorX::get_sort_description( + RuntimeState* state) const { + auto& local_state = get_local_state(state); + return local_state._shared_state->sorter->get_sort_description(); +} + } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/sort_source_operator.h b/be/src/pipeline/exec/sort_source_operator.h index 39436ac26d..1af30bfd05 100644 --- a/be/src/pipeline/exec/sort_source_operator.h +++ b/be/src/pipeline/exec/sort_source_operator.h @@ -64,6 +64,8 @@ public: bool is_source() const override { return true; } + const vectorized::SortDescription& get_sort_description(RuntimeState* state) const; + private: friend class SortLocalState; }; diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.cpp b/be/src/pipeline/exec/spill_sort_sink_operator.cpp new file mode 100644 index 0000000000..15edcf3c8d --- /dev/null +++ b/be/src/pipeline/exec/spill_sort_sink_operator.cpp @@ -0,0 +1,292 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "spill_sort_sink_operator.h" + +#include "pipeline/exec/sort_sink_operator.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris::pipeline { +SpillSortSinkLocalState::SpillSortSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : Base(parent, state) { + _finish_dependency = std::make_shared( + parent->operator_id(), parent->node_id(), parent->get_name() + "_FINISH_DEPENDENCY", + state->get_query_ctx()); +} +Status SpillSortSinkLocalState::init(doris::RuntimeState* state, + doris::pipeline::LocalSinkStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_open_timer); + + _init_counters(); + + RETURN_IF_ERROR(setup_in_memory_sort_op(state)); + + auto& parent = Base::_parent->template cast(); + Base::_shared_state->enable_spill = parent._enable_spill; + Base::_shared_state->in_mem_shared_state->sorter->set_enable_spill(parent._enable_spill); + if (parent._enable_spill) { + _finish_dependency->block(); + } + return Status::OK(); +} +void SpillSortSinkLocalState::_init_counters() { + _internal_runtime_profile = std::make_unique("internal_profile"); + + _partial_sort_timer = ADD_TIMER(_profile, "PartialSortTime"); + _merge_block_timer = ADD_TIMER(_profile, "MergeBlockTime"); + _sort_blocks_memory_usage = + ADD_CHILD_COUNTER_WITH_LEVEL(_profile, "SortBlocks", TUnit::BYTES, "MemoryUsage", 1); + + _spill_merge_sort_timer = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillMergeSortTime", "Spill", 1); +} +#define UPDATE_PROFILE(counter, name) \ + do { \ + auto* child_counter = child_profile->get_counter(name); \ + if (child_counter != nullptr) { \ + COUNTER_SET(counter, child_counter->value()); \ + } \ + } while (false) + +void SpillSortSinkLocalState::update_profile(RuntimeProfile* child_profile) { + UPDATE_PROFILE(_partial_sort_timer, "PartialSortTime"); + UPDATE_PROFILE(_merge_block_timer, "MergeBlockTime"); + UPDATE_PROFILE(_sort_blocks_memory_usage, "SortBlocks"); +} +Status SpillSortSinkLocalState::open(RuntimeState* state) { + RETURN_IF_ERROR(Base::open(state)); + return Status::OK(); +} +Status SpillSortSinkLocalState::close(RuntimeState* state, Status execsink_status) { + { + std::unique_lock lk(_spill_lock); + if (_is_spilling) { + _spill_cv.wait(lk); + } + } + return Status::OK(); +} +Status SpillSortSinkLocalState::setup_in_memory_sort_op(RuntimeState* state) { + _runtime_state = RuntimeState::create_unique( + nullptr, state->fragment_instance_id(), state->query_id(), state->fragment_id(), + state->query_options(), TQueryGlobals {}, state->exec_env(), state->get_query_ctx()); + _runtime_state->set_query_mem_tracker(state->query_mem_tracker()); + _runtime_state->set_task_execution_context(state->get_task_execution_context().lock()); + _runtime_state->set_be_number(state->be_number()); + + _runtime_state->set_desc_tbl(&state->desc_tbl()); + _runtime_state->set_pipeline_x_runtime_filter_mgr(state->local_runtime_filter_mgr()); + + auto& parent = Base::_parent->template cast(); + Base::_shared_state->in_mem_shared_state_sptr = + parent._sort_sink_operator->create_shared_state(); + Base::_shared_state->in_mem_shared_state = + static_cast(Base::_shared_state->in_mem_shared_state_sptr.get()); + + LocalSinkStateInfo info {0, _internal_runtime_profile.get(), + -1, Base::_shared_state->in_mem_shared_state, + {}, {}}; + RETURN_IF_ERROR(parent._sort_sink_operator->setup_local_state(_runtime_state.get(), info)); + auto* sink_local_state = _runtime_state->get_sink_local_state(); + DCHECK(sink_local_state != nullptr); + + _profile->add_info_string("TOP-N", *sink_local_state->profile()->get_info_string("TOP-N")); + + return sink_local_state->open(state); +} + +SpillSortSinkOperatorX::SpillSortSinkOperatorX(ObjectPool* pool, int operator_id, + const TPlanNode& tnode, const DescriptorTbl& descs) + : DataSinkOperatorX(operator_id, tnode.node_id) { + _sort_sink_operator = std::make_unique(pool, operator_id, tnode, descs); +} + +Status SpillSortSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(DataSinkOperatorX::init(tnode, state)); + _name = "SPILL_SORT_SINK_OPERATOR"; + + _sort_sink_operator->set_dests_id(DataSinkOperatorX::dests_id()); + RETURN_IF_ERROR(_sort_sink_operator->set_child(DataSinkOperatorX::_child_x)); + return _sort_sink_operator->init(tnode, state); +} + +Status SpillSortSinkOperatorX::prepare(RuntimeState* state) { + RETURN_IF_ERROR(DataSinkOperatorX::prepare(state)); + RETURN_IF_ERROR(_sort_sink_operator->prepare(state)); + _enable_spill = _sort_sink_operator->is_full_sort(); + LOG(INFO) << "spill sort sink, enable spill: " << _enable_spill; + return Status::OK(); +} +Status SpillSortSinkOperatorX::open(RuntimeState* state) { + RETURN_IF_ERROR(DataSinkOperatorX::open(state)); + return _sort_sink_operator->open(state); +} +Status SpillSortSinkOperatorX::close(RuntimeState* state) { + RETURN_IF_ERROR(DataSinkOperatorX::close(state)); + return _sort_sink_operator->close(state); +} +Status SpillSortSinkOperatorX::revoke_memory(RuntimeState* state) { + if (!_enable_spill) { + return Status::OK(); + } + auto& local_state = get_local_state(state); + return local_state.revoke_memory(state); +} +size_t SpillSortSinkOperatorX::revocable_mem_size(RuntimeState* state) const { + if (!_enable_spill) { + return 0; + } + auto& local_state = get_local_state(state); + if (!local_state.Base::_shared_state->sink_status.ok()) { + return UINT64_MAX; + } + return _sort_sink_operator->get_revocable_mem_size(local_state._runtime_state.get()); +} +Status SpillSortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in_block, + bool eos) { + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + RETURN_IF_ERROR(local_state.Base::_shared_state->sink_status); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); + if (in_block->rows() > 0) { + local_state._shared_state->update_spill_block_batch_row_count(in_block); + } + local_state._eos = eos; + RETURN_IF_ERROR(_sort_sink_operator->sink(local_state._runtime_state.get(), in_block, false)); + local_state._mem_tracker->set_consumption( + local_state._shared_state->in_mem_shared_state->sorter->data_size()); + if (eos) { + if (_enable_spill) { + if (revocable_mem_size(state) > 0) { + RETURN_IF_ERROR(revoke_memory(state)); + } else { + local_state._dependency->set_ready_to_read(); + } + } else { + RETURN_IF_ERROR( + local_state._shared_state->in_mem_shared_state->sorter->prepare_for_read()); + local_state._dependency->set_ready_to_read(); + } + } + return Status::OK(); +} +Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state) { + DCHECK(!_is_spilling); + _is_spilling = true; + + LOG(INFO) << "sort node " << Base::_parent->id() << " revoke_memory" + << ", eos: " << _eos; + RETURN_IF_ERROR(Base::_shared_state->sink_status); + + auto status = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( + state, _spilling_stream, print_id(state->query_id()), "sort", _parent->id(), + _shared_state->spill_block_batch_row_count, + SpillSortSharedState::SORT_BLOCK_SPILL_BATCH_BYTES, profile()); + RETURN_IF_ERROR(status); + + _spilling_stream->set_write_counters(Base::_spill_serialize_block_timer, + Base::_spill_block_count, Base::_spill_data_size, + Base::_spill_write_disk_timer); + + status = _spilling_stream->prepare_spill(); + RETURN_IF_ERROR(status); + _shared_state->sorted_streams.emplace_back(_spilling_stream); + + auto& parent = Base::_parent->template cast(); + + // TODO: spill thread may set_ready before the task::execute thread put the task to blocked state + if (!_eos) { + Base::_dependency->Dependency::block(); + } + status = + ExecEnv::GetInstance() + ->spill_stream_mgr() + ->get_spill_io_thread_pool(_spilling_stream->get_spill_root_dir()) + ->submit_func([this, state, &parent] { + SCOPED_ATTACH_TASK(state); + Defer defer {[&]() { + if (!_shared_state->sink_status.ok()) { + LOG(WARNING) + << "sort node " << _parent->id() + << " revoke memory error: " << _shared_state->sink_status; + } else { + LOG(INFO) + << "sort node " << _parent->id() << " revoke memory finish"; + } + + _spilling_stream->end_spill(_shared_state->sink_status); + if (!_shared_state->sink_status.ok()) { + _shared_state->clear(); + } + + { + std::unique_lock lk(_spill_lock); + _spilling_stream.reset(); + _is_spilling = false; + if (_eos) { + _dependency->set_ready_to_read(); + _finish_dependency->set_ready(); + } else { + _dependency->Dependency::set_ready(); + } + _spill_cv.notify_one(); + } + }}; + + _shared_state->sink_status = + parent._sort_sink_operator->prepare_for_spill(_runtime_state.get()); + RETURN_IF_ERROR(_shared_state->sink_status); + + auto* sink_local_state = _runtime_state->get_sink_local_state(); + update_profile(sink_local_state->profile()); + + bool eos = false; + vectorized::Block block; + while (!eos && !state->is_cancelled()) { + { + SCOPED_TIMER(_spill_merge_sort_timer); + _shared_state->sink_status = + parent._sort_sink_operator->merge_sort_read_for_spill( + _runtime_state.get(), &block, + _shared_state->spill_block_batch_row_count, &eos); + } + RETURN_IF_ERROR(_shared_state->sink_status); + { + SCOPED_TIMER(Base::_spill_timer); + _shared_state->sink_status = + _spilling_stream->spill_block(block, eos); + } + RETURN_IF_ERROR(_shared_state->sink_status); + block.clear_column_data(); + } + parent._sort_sink_operator->reset(_runtime_state.get()); + + return Status::OK(); + }); + if (!status.ok()) { + _is_spilling = false; + _spilling_stream->end_spill(status); + + if (!_eos) { + Base::_dependency->Dependency::set_ready(); + } + } + return status; +} +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.h b/be/src/pipeline/exec/spill_sort_sink_operator.h new file mode 100644 index 0000000000..aba85abece --- /dev/null +++ b/be/src/pipeline/exec/spill_sort_sink_operator.h @@ -0,0 +1,104 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include "operator.h" +#include "pipeline/pipeline_x/operator.h" +#include "sort_sink_operator.h" + +namespace doris::pipeline { +class SpillSortSinkLocalState; +class SpillSortSinkOperatorX; + +class SpillSortSinkLocalState : public PipelineXSpillSinkLocalState { + ENABLE_FACTORY_CREATOR(SpillSortSinkLocalState); + +public: + using Base = PipelineXSpillSinkLocalState; + using Parent = SpillSortSinkOperatorX; + SpillSortSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); + ~SpillSortSinkLocalState() override = default; + + Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + Status open(RuntimeState* state) override; + Status close(RuntimeState* state, Status exec_status) override; + Dependency* finishdependency() override { return _finish_dependency.get(); } + + Status setup_in_memory_sort_op(RuntimeState* state); + Status revoke_memory(RuntimeState* state); + +private: + void _init_counters(); + void update_profile(RuntimeProfile* child_profile); + + friend class SpillSortSinkOperatorX; + + std::unique_ptr _runtime_state; + std::unique_ptr _internal_runtime_profile; + RuntimeProfile::Counter* _partial_sort_timer = nullptr; + RuntimeProfile::Counter* _merge_block_timer = nullptr; + RuntimeProfile::Counter* _sort_blocks_memory_usage = nullptr; + + RuntimeProfile::Counter* _spill_merge_sort_timer = nullptr; + + bool _eos = false; + bool _is_spilling = false; + vectorized::SpillStreamSPtr _spilling_stream; + std::shared_ptr _finish_dependency; + std::mutex _spill_lock; + std::condition_variable _spill_cv; +}; + +class SpillSortSinkOperatorX final : public DataSinkOperatorX { +public: + using LocalStateType = SpillSortSinkLocalState; + SpillSortSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, + const DescriptorTbl& descs); + Status init(const TDataSink& tsink) override { + return Status::InternalError("{} should not init with TPlanNode", + DataSinkOperatorX::_name); + } + + Status init(const TPlanNode& tnode, RuntimeState* state) override; + + Status prepare(RuntimeState* state) override; + Status open(RuntimeState* state) override; + Status close(RuntimeState* state) override; + Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos) override; + DataDistribution required_data_distribution() const override { + return _sort_sink_operator->required_data_distribution(); + } + Status set_child(OperatorXPtr child) override { + RETURN_IF_ERROR(DataSinkOperatorX::set_child(child)); + return _sort_sink_operator->set_child(child); + } + + size_t revocable_mem_size(RuntimeState* state) const override; + + Status revoke_memory(RuntimeState* state) override; + + using DataSinkOperatorX::id; + using DataSinkOperatorX::operator_id; + using DataSinkOperatorX::get_local_state; + +private: + friend class SpillSortSinkLocalState; + std::unique_ptr _sort_sink_operator; + bool _enable_spill = false; +}; +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/spill_sort_source_operator.cpp b/be/src/pipeline/exec/spill_sort_source_operator.cpp new file mode 100644 index 0000000000..f82c015a17 --- /dev/null +++ b/be/src/pipeline/exec/spill_sort_source_operator.cpp @@ -0,0 +1,261 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "spill_sort_source_operator.h" + +#include "common/status.h" +#include "sort_source_operator.h" +#include "util/runtime_profile.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris::pipeline { +SpillSortLocalState::SpillSortLocalState(RuntimeState* state, OperatorXBase* parent) + : Base(state, parent) { + if (state->external_sort_bytes_threshold() > 0) { + _external_sort_bytes_threshold = state->external_sort_bytes_threshold(); + } +} +Status SpillSortLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + _internal_runtime_profile = std::make_unique("internal_profile"); + _spill_timer = ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillMergeSortTime", "Spill", 1); + _spill_merge_sort_timer = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillMergeSortTime", "Spill", 1); + _spill_serialize_block_timer = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillSerializeBlockTime", "Spill", 1); + _spill_write_disk_timer = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteDiskTime", "Spill", 1); + _spill_data_size = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteDataSize", + TUnit::BYTES, "Spill", 1); + _spill_block_count = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteBlockCount", + TUnit::UNIT, "Spill", 1); + return Status::OK(); +} + +Status SpillSortLocalState::open(RuntimeState* state) { + if (_opened) { + return Status::OK(); + } + RETURN_IF_ERROR(setup_in_memory_sort_op(state)); + return Base::open(state); +} + +Status SpillSortLocalState::close(RuntimeState* state) { + if (_closed) { + return Status::OK(); + } + { + std::unique_lock lk(_merge_spill_lock); + if (_is_merging) { + _merge_spill_cv.wait(lk); + } + } + RETURN_IF_ERROR(Base::close(state)); + for (auto& stream : _current_merging_streams) { + (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + } + _current_merging_streams.clear(); + return Status::OK(); +} +int SpillSortLocalState::_calc_spill_blocks_to_merge() const { + int count = _external_sort_bytes_threshold / SpillSortSharedState::SORT_BLOCK_SPILL_BATCH_BYTES; + return std::max(2, count); +} +Status SpillSortLocalState::initiate_merge_sort_spill_streams(RuntimeState* state) { + auto& parent = Base::_parent->template cast(); + LOG(INFO) << "sort node " << _parent->node_id() << " merge spill data"; + DCHECK(!_is_merging); + _is_merging = true; + _dependency->Dependency::block(); + + Status status; + Defer defer {[&]() { + if (!status.ok()) { + _is_merging = false; + _dependency->Dependency::set_ready(); + } + }}; + + auto spill_func = [this, state, &parent] { + SCOPED_TIMER(_spill_merge_sort_timer); + SCOPED_ATTACH_TASK(state); + Defer defer {[&]() { + if (!_status.ok()) { + LOG(WARNING) << "sort node " << _parent->node_id() + << " merge spill data error: " << _status; + } else { + LOG(INFO) << "sort node " << _parent->node_id() << " merge spill data finish"; + } + { + std::unique_lock lk(_merge_spill_lock); + _is_merging = false; + _dependency->Dependency::set_ready(); + _merge_spill_cv.notify_one(); + } + }}; + vectorized::Block merge_sorted_block; + vectorized::SpillStreamSPtr tmp_stream; + while (true) { + int max_stream_count = _calc_spill_blocks_to_merge(); + LOG(INFO) << "sort node " << _parent->id() << " merge spill streams, streams count: " + << _shared_state->sorted_streams.size() + << ", curren merge max stream count: " << max_stream_count; + { + SCOPED_TIMER(Base::_spill_recover_time); + _status = _create_intermediate_merger( + max_stream_count, + parent._sort_source_operator->get_sort_description(_runtime_state.get())); + } + RETURN_IF_ERROR(_status); + + // all the remaining streams can be merged in a run + if (_shared_state->sorted_streams.empty()) { + return Status::OK(); + } + + { + _status = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( + state, tmp_stream, print_id(state->query_id()), "sort", _parent->id(), + _shared_state->spill_block_batch_row_count, + SpillSortSharedState::SORT_BLOCK_SPILL_BATCH_BYTES, profile()); + RETURN_IF_ERROR(_status); + _status = tmp_stream->prepare_spill(); + RETURN_IF_ERROR(_status); + + Defer defer {[&]() { tmp_stream->end_spill(_status); }}; + _shared_state->sorted_streams.emplace_back(tmp_stream); + + bool eos = false; + tmp_stream->set_write_counters(_spill_serialize_block_timer, _spill_block_count, + _spill_data_size, _spill_write_disk_timer); + while (!eos && !state->is_cancelled()) { + merge_sorted_block.clear_column_data(); + { + SCOPED_TIMER(Base::_spill_recover_time); + _status = _merger->get_next(&merge_sorted_block, &eos); + } + RETURN_IF_ERROR(_status); + _status = tmp_stream->spill_block(merge_sorted_block, eos); + RETURN_IF_ERROR(_status); + } + } + for (auto& stream : _current_merging_streams) { + (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + } + _current_merging_streams.clear(); + } + DCHECK(false); + return Status::OK(); + }; + return ExecEnv::GetInstance()->spill_stream_mgr()->get_async_task_thread_pool()->submit_func( + spill_func); +} + +Status SpillSortLocalState::_create_intermediate_merger( + int num_blocks, const vectorized::SortDescription& sort_description) { + std::vector child_block_suppliers; + _merger = std::make_unique( + sort_description, _shared_state->spill_block_batch_row_count, + Base::_shared_state->in_mem_shared_state->sorter->limit(), + Base::_shared_state->in_mem_shared_state->sorter->offset(), profile()); + + _current_merging_streams.clear(); + for (int i = 0; i < num_blocks && !_shared_state->sorted_streams.empty(); ++i) { + auto stream = _shared_state->sorted_streams.front(); + stream->set_read_counters(Base::_spill_read_data_time, Base::_spill_deserialize_time, + Base::_spill_read_bytes); + _current_merging_streams.emplace_back(stream); + child_block_suppliers.emplace_back( + std::bind(std::mem_fn(&vectorized::SpillStream::read_next_block_sync), stream.get(), + std::placeholders::_1, std::placeholders::_2)); + + _shared_state->sorted_streams.pop_front(); + } + RETURN_IF_ERROR(_merger->prepare(child_block_suppliers)); + return Status::OK(); +} +Status SpillSortLocalState::setup_in_memory_sort_op(RuntimeState* state) { + _runtime_state = RuntimeState::create_unique( + nullptr, state->fragment_instance_id(), state->query_id(), state->fragment_id(), + state->query_options(), TQueryGlobals {}, state->exec_env(), state->get_query_ctx()); + _runtime_state->set_query_mem_tracker(state->query_mem_tracker()); + _runtime_state->set_task_execution_context(state->get_task_execution_context().lock()); + _runtime_state->set_be_number(state->be_number()); + + _runtime_state->set_desc_tbl(&state->desc_tbl()); + _runtime_state->resize_op_id_to_local_state(state->max_operator_id()); + _runtime_state->set_pipeline_x_runtime_filter_mgr(state->local_runtime_filter_mgr()); + + DCHECK(_shared_state->in_mem_shared_state); + LocalStateInfo state_info { + _internal_runtime_profile.get(), {}, _shared_state->in_mem_shared_state, {}, 0}; + + auto& parent = Base::_parent->template cast(); + RETURN_IF_ERROR( + parent._sort_source_operator->setup_local_state(_runtime_state.get(), state_info)); + + auto* source_local_state = + _runtime_state->get_local_state(parent._sort_source_operator->operator_id()); + DCHECK(source_local_state != nullptr); + return source_local_state->open(state); +} +SpillSortSourceOperatorX::SpillSortSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, + int operator_id, const DescriptorTbl& descs) + : Base(pool, tnode, operator_id, descs) { + _sort_source_operator = std::make_unique(pool, tnode, operator_id, descs); +} +Status SpillSortSourceOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::init(tnode, state)); + _op_name = "SPILL_SORT_SOURCE_OPERATOR"; + return _sort_source_operator->init(tnode, state); +} + +Status SpillSortSourceOperatorX::prepare(RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::prepare(state)); + return _sort_source_operator->prepare(state); +} + +Status SpillSortSourceOperatorX::open(RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::open(state)); + return _sort_source_operator->open(state); +} + +Status SpillSortSourceOperatorX::close(RuntimeState* state) { + RETURN_IF_ERROR(OperatorXBase::close(state)); + return _sort_source_operator->close(state); +} + +Status SpillSortSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, + bool* eos) { + auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); + RETURN_IF_ERROR(local_state._status); + + if (!local_state.Base::_shared_state->enable_spill) { + RETURN_IF_ERROR( + _sort_source_operator->get_block(local_state._runtime_state.get(), block, eos)); + } else { + if (!local_state._merger) { + return local_state.initiate_merge_sort_spill_streams(state); + } else { + RETURN_IF_ERROR(local_state._merger->get_next(block, eos)); + } + } + local_state.reached_limit(block, eos); + return Status::OK(); +} +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/spill_sort_source_operator.h b/be/src/pipeline/exec/spill_sort_source_operator.h new file mode 100644 index 0000000000..ae76c4e1b5 --- /dev/null +++ b/be/src/pipeline/exec/spill_sort_source_operator.h @@ -0,0 +1,102 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "common/status.h" +#include "operator.h" +#include "pipeline/pipeline_x/operator.h" + +namespace doris { +class ExecNode; +class RuntimeState; + +namespace pipeline { +class SpillSortSourceOperatorX; +class SpillSortLocalState; + +class SpillSortLocalState final : public PipelineXSpillLocalState { +public: + ENABLE_FACTORY_CREATOR(SpillSortLocalState); + using Base = PipelineXSpillLocalState; + using Parent = SpillSortSourceOperatorX; + SpillSortLocalState(RuntimeState* state, OperatorXBase* parent); + ~SpillSortLocalState() override = default; + + Status init(RuntimeState* state, LocalStateInfo& info) override; + Status open(RuntimeState* state) override; + Status close(RuntimeState* state) override; + + Status setup_in_memory_sort_op(RuntimeState* state); + + Status initiate_merge_sort_spill_streams(RuntimeState* state); + +protected: + int _calc_spill_blocks_to_merge() const; + Status _create_intermediate_merger(int num_blocks, + const vectorized::SortDescription& sort_description); + friend class SpillSortSourceOperatorX; + std::unique_ptr _runtime_state; + + bool _opened = false; + Status _status; + + int64_t _external_sort_bytes_threshold = 134217728; // 128M + std::vector _current_merging_streams; + std::unique_ptr _merger; + bool _is_merging = false; + std::mutex _merge_spill_lock; + std::condition_variable _merge_spill_cv; + + std::unique_ptr _internal_runtime_profile; + // counters for spill merge sort + RuntimeProfile::Counter* _spill_timer = nullptr; + RuntimeProfile::Counter* _spill_merge_sort_timer = nullptr; + RuntimeProfile::Counter* _spill_serialize_block_timer = nullptr; + RuntimeProfile::Counter* _spill_write_disk_timer = nullptr; + RuntimeProfile::Counter* _spill_data_size = nullptr; + RuntimeProfile::Counter* _spill_block_count = nullptr; +}; +class SortSourceOperatorX; +class SpillSortSourceOperatorX : public OperatorX { +public: + using Base = OperatorX; + SpillSortSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, + const DescriptorTbl& descs); + ~SpillSortSourceOperatorX() override = default; + + Status init(const TPlanNode& tnode, RuntimeState* state) override; + Status prepare(RuntimeState* state) override; + + Status open(RuntimeState* state) override; + + Status close(RuntimeState* state) override; + + Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos) override; + + bool is_source() const override { return true; } + +private: + friend class SpillSortLocalState; + Status _initiate_merge_spill_partition_agg_data(RuntimeState* state); + + std::unique_ptr _sort_source_operator; +}; +} // namespace pipeline +} // namespace doris \ No newline at end of file diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp index a658572083..9c9a014046 100644 --- a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp @@ -21,7 +21,6 @@ #include "pipeline/exec/data_queue.h" #include "pipeline/exec/operator.h" -#include "pipeline/pipeline_x/dependency.h" #include "runtime/descriptors.h" #include "util/runtime_profile.h" #include "vec/core/block.h" diff --git a/be/src/pipeline/pipeline_x/dependency.cpp b/be/src/pipeline/pipeline_x/dependency.cpp index 3305341a51..016410b54b 100644 --- a/be/src/pipeline/pipeline_x/dependency.cpp +++ b/be/src/pipeline/pipeline_x/dependency.cpp @@ -26,6 +26,7 @@ #include "pipeline/pipeline_x/pipeline_x_task.h" #include "runtime/exec_env.h" #include "runtime/memory/mem_tracker.h" +#include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { @@ -203,4 +204,77 @@ LocalExchangeSharedState::LocalExchangeSharedState(int num_instances) { mem_trackers.resize(num_instances, nullptr); } +Status AggSharedState::reset_hash_table() { + return std::visit( + [&](auto&& agg_method) { + auto& hash_table = *agg_method.hash_table; + using HashTableType = std::decay_t; + + agg_method.reset(); + + hash_table.for_each_mapped([&](auto& mapped) { + if (mapped) { + static_cast(_destroy_agg_status(mapped)); + mapped = nullptr; + } + }); + + aggregate_data_container.reset(new vectorized::AggregateDataContainer( + sizeof(typename HashTableType::key_type), + ((total_size_of_aggregate_states + align_aggregate_states - 1) / + align_aggregate_states) * + align_aggregate_states)); + agg_method.hash_table.reset(new HashTableType()); + agg_arena_pool.reset(new vectorized::Arena); + return Status::OK(); + }, + agg_data->method_variant); +} + +void PartitionedAggSharedState::init_spill_params(size_t spill_partition_count_bits) { + partition_count_bits = spill_partition_count_bits; + partition_count = (1 << spill_partition_count_bits); + max_partition_index = partition_count - 1; + + for (int i = 0; i < partition_count; ++i) { + spill_partitions.emplace_back(std::make_shared()); + } +} + +Status AggSpillPartition::get_spill_stream(RuntimeState* state, int node_id, + RuntimeProfile* profile, + vectorized::SpillStreamSPtr& spill_stream) { + if (spilling_stream_) { + spill_stream = spilling_stream_; + return Status::OK(); + } + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( + state, spilling_stream_, print_id(state->query_id()), "agg", node_id, + std::numeric_limits::max(), std::numeric_limits::max(), profile)); + spill_streams_.emplace_back(spilling_stream_); + spill_stream = spilling_stream_; + return Status::OK(); +} +void AggSpillPartition::close() { + if (spilling_stream_) { + (void)spilling_stream_->wait_spill(); + spilling_stream_.reset(); + } + for (auto& stream : spill_streams_) { + (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + } + spill_streams_.clear(); +} + +void PartitionedAggSharedState::close() { + for (auto partition : spill_partitions) { + partition->close(); + } +} +void SpillSortSharedState::clear() { + for (auto& stream : sorted_streams) { + (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + } + sorted_streams.clear(); +} } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_x/dependency.h b/be/src/pipeline/pipeline_x/dependency.h index 596ef8bc6f..db2b0341fe 100644 --- a/be/src/pipeline/pipeline_x/dependency.h +++ b/be/src/pipeline/pipeline_x/dependency.h @@ -42,6 +42,7 @@ #include "vec/exec/vanalytic_eval_node.h" #include "vec/exec/vpartition_sort_node.h" #include "vec/exec/vset_operation_node.h" +#include "vec/spill/spill_stream.h" namespace doris::pipeline { @@ -336,17 +337,24 @@ public: _close_with_serialized_key(); } } - void init_spill_partition_helper(size_t spill_partition_count_bits) { - spill_partition_helper = - std::make_unique(spill_partition_count_bits); + + Status reset_hash_table(); + + // We should call this function only at 1st phase. + // 1st phase: is_merge=true, only have one SlotRef. + // 2nd phase: is_merge=false, maybe have multiple exprs. + static int 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]=" + << ctxs[0]->root()->debug_string(); + return ((vectorized::VSlotRef*)ctxs[0]->root().get())->column_id(); } vectorized::AggregatedDataVariantsUPtr agg_data = nullptr; std::unique_ptr aggregate_data_container; - vectorized::AggSpillContext spill_context; vectorized::ArenaUPtr agg_arena_pool; std::vector aggregate_evaluators; - std::unique_ptr spill_partition_helper; // group by k1,k2 vectorized::VExprContextSPtrs probe_expr_ctxs; size_t input_num_rows = 0; @@ -368,6 +376,8 @@ public: bool agg_data_created_without_key = false; std::atomic ready_to_execute = false; + bool enable_spill = false; + private: void _close_with_serialized_key() { std::visit( @@ -406,11 +416,100 @@ private: } }; +struct AggSpillPartition; +struct PartitionedAggSharedState : public BasicSharedState { +public: + PartitionedAggSharedState() = default; + ~PartitionedAggSharedState() override = default; + + void init_spill_params(size_t spill_partition_count_bits); + + void close(); + + AggSharedState* in_mem_shared_state = nullptr; + std::shared_ptr in_mem_shared_state_sptr; + + size_t partition_count_bits; + size_t partition_count; + size_t max_partition_index; + Status sink_status; + std::deque> spill_partitions; + + size_t get_partition_index(size_t hash_value) const { + return (hash_value >> (32 - partition_count_bits)) & max_partition_index; + } +}; + +struct AggSpillPartition { + static constexpr int64_t AGG_SPILL_FILE_SIZE = 1024 * 1024 * 1024; // 1G + + AggSpillPartition() = default; + + void close(); + + Status get_spill_stream(RuntimeState* state, int node_id, RuntimeProfile* profile, + vectorized::SpillStreamSPtr& spilling_stream); + + // wait for current bock spilling to finish + Status wait_spill(RuntimeState* state) { + DCHECK(spilling_stream_); + auto status = spilling_stream_->wait_spill(); + RETURN_IF_ERROR(status); + // avoid small spill files + if (spilling_stream_->get_written_bytes() >= AGG_SPILL_FILE_SIZE) { + status = spilling_stream_->spill_eof(); + spilling_stream_.reset(); + } + return status; + } + + Status finish_current_spilling(bool eos = false) { + if (spilling_stream_) { + if (eos || spilling_stream_->get_written_bytes() >= AGG_SPILL_FILE_SIZE) { + auto status = spilling_stream_->spill_eof(); + spilling_stream_.reset(); + return status; + } + } + return Status::OK(); + } + + std::deque spill_streams_; + vectorized::SpillStreamSPtr spilling_stream_; +}; +using AggSpillPartitionSPtr = std::shared_ptr; struct SortSharedState : public BasicSharedState { public: std::unique_ptr sorter; }; +struct SpillSortSharedState : public BasicSharedState { + SpillSortSharedState() = default; + ~SpillSortSharedState() override = default; + + // This number specifies the maximum size of sub blocks + static constexpr int SORT_BLOCK_SPILL_BATCH_BYTES = 8 * 1024 * 1024; + void update_spill_block_batch_row_count(const vectorized::Block* block) { + auto rows = block->rows(); + if (rows > 0 && 0 == avg_row_bytes) { + avg_row_bytes = std::max((std::size_t)1, block->bytes() / rows); + spill_block_batch_row_count = + (SORT_BLOCK_SPILL_BATCH_BYTES + avg_row_bytes - 1) / avg_row_bytes; + LOG(INFO) << "spill sort block batch row count: " << spill_block_batch_row_count; + } + } + void clear(); + + SortSharedState* in_mem_shared_state = nullptr; + bool enable_spill = false; + Status sink_status; + std::shared_ptr in_mem_shared_state_sptr; + + std::deque sorted_streams; + size_t avg_row_bytes = 0; + int spill_block_batch_row_count; +}; + struct UnionSharedState : public BasicSharedState { public: UnionSharedState(int child_count = 1) : data_queue(child_count), _child_count(child_count) {}; @@ -476,6 +575,11 @@ struct HashJoinSharedState : public JoinSharedState { bool probe_ignore_null = false; }; +struct PartitionedHashJoinSharedState : public HashJoinSharedState { + std::vector> partitioned_build_blocks; + std::vector spilled_streams; +}; + struct NestedLoopJoinSharedState : public JoinSharedState { // if true, left child has no more rows to process bool left_side_eos = false; diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h index debfd113f3..6e787b21a6 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h @@ -17,7 +17,6 @@ #pragma once -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" namespace doris::pipeline { diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h index f0d7309c21..53e6aef332 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h @@ -17,7 +17,6 @@ #pragma once -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" namespace doris::pipeline { diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchanger.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchanger.h index f2e3ae1a4c..69e9f79d3e 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchanger.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchanger.h @@ -17,7 +17,6 @@ #pragma once -#include "pipeline/pipeline_x/dependency.h" #include "pipeline/pipeline_x/operator.h" namespace doris::pipeline { diff --git a/be/src/pipeline/pipeline_x/operator.cpp b/be/src/pipeline/pipeline_x/operator.cpp index a29516a29a..0c890b8304 100644 --- a/be/src/pipeline/pipeline_x/operator.cpp +++ b/be/src/pipeline/pipeline_x/operator.cpp @@ -49,6 +49,10 @@ #include "pipeline/exec/olap_table_sink_v2_operator.h" #include "pipeline/exec/partition_sort_sink_operator.h" #include "pipeline/exec/partition_sort_source_operator.h" +#include "pipeline/exec/partitioned_aggregation_sink_operator.h" +#include "pipeline/exec/partitioned_aggregation_source_operator.h" +#include "pipeline/exec/partitioned_hash_join_probe_operator.h" +#include "pipeline/exec/partitioned_hash_join_sink_operator.h" #include "pipeline/exec/repeat_operator.h" #include "pipeline/exec/result_file_sink_operator.h" #include "pipeline/exec/result_sink_operator.h" @@ -59,7 +63,11 @@ #include "pipeline/exec/set_source_operator.h" #include "pipeline/exec/sort_sink_operator.h" #include "pipeline/exec/sort_source_operator.h" +#include "pipeline/exec/spill_sort_sink_operator.h" +#include "pipeline/exec/spill_sort_source_operator.h" #include "pipeline/exec/streaming_aggregation_operator.h" +#include "pipeline/exec/streaming_aggregation_sink_operator.h" +#include "pipeline/exec/streaming_aggregation_source_operator.h" #include "pipeline/exec/table_function_operator.h" #include "pipeline/exec/union_sink_operator.h" #include "pipeline/exec/union_source_operator.h" @@ -535,8 +543,10 @@ DECLARE_OPERATOR_X(OlapTableSinkLocalState) DECLARE_OPERATOR_X(OlapTableSinkV2LocalState) DECLARE_OPERATOR_X(AnalyticSinkLocalState) DECLARE_OPERATOR_X(SortSinkLocalState) +DECLARE_OPERATOR_X(SpillSortSinkLocalState) DECLARE_OPERATOR_X(LocalExchangeSinkLocalState) DECLARE_OPERATOR_X(AggSinkLocalState) +DECLARE_OPERATOR_X(PartitionedAggSinkLocalState) DECLARE_OPERATOR_X(ExchangeSinkLocalState) DECLARE_OPERATOR_X(NestedLoopJoinBuildSinkLocalState) DECLARE_OPERATOR_X(UnionSinkLocalState) @@ -546,6 +556,7 @@ DECLARE_OPERATOR_X(SetProbeSinkLocalState) DECLARE_OPERATOR_X(SetProbeSinkLocalState) DECLARE_OPERATOR_X(SetSinkLocalState) DECLARE_OPERATOR_X(SetSinkLocalState) +DECLARE_OPERATOR_X(PartitionedHashJoinSinkLocalState) #undef DECLARE_OPERATOR_X @@ -557,7 +568,9 @@ DECLARE_OPERATOR_X(FileScanLocalState) DECLARE_OPERATOR_X(EsScanLocalState) DECLARE_OPERATOR_X(AnalyticLocalState) DECLARE_OPERATOR_X(SortLocalState) +DECLARE_OPERATOR_X(SpillSortLocalState) DECLARE_OPERATOR_X(AggLocalState) +DECLARE_OPERATOR_X(PartitionedAggLocalState) DECLARE_OPERATOR_X(TableFunctionLocalState) DECLARE_OPERATOR_X(ExchangeLocalState) DECLARE_OPERATOR_X(RepeatLocalState) @@ -573,6 +586,7 @@ DECLARE_OPERATOR_X(DataGenLocalState) DECLARE_OPERATOR_X(SchemaScanLocalState) DECLARE_OPERATOR_X(MetaScanLocalState) DECLARE_OPERATOR_X(LocalExchangeSourceLocalState) +DECLARE_OPERATOR_X(PartitionedHashJoinProbeLocalState) #undef DECLARE_OPERATOR_X @@ -580,6 +594,7 @@ template class StreamingOperatorX; template class StreamingOperatorX; template class StatefulOperatorX; +template class StatefulOperatorX; template class StatefulOperatorX; template class StatefulOperatorX; template class StatefulOperatorX; @@ -587,10 +602,13 @@ template class StatefulOperatorX; template class StatefulOperatorX; template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; +template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; @@ -600,10 +618,13 @@ template class PipelineXSinkLocalState; template class PipelineXSinkLocalState; template class PipelineXLocalState; +template class PipelineXLocalState; template class PipelineXLocalState; +template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; +template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; template class PipelineXLocalState; diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index b9c02a935a..06ba93a36f 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -372,6 +372,35 @@ protected: SharedStateArg* _shared_state = nullptr; }; +template +class PipelineXSpillLocalState : public PipelineXLocalState { +public: + using Base = PipelineXLocalState; + PipelineXSpillLocalState(RuntimeState* state, OperatorXBase* parent) + : PipelineXLocalState(state, parent) {} + ~PipelineXSpillLocalState() override = default; + + Status init(RuntimeState* state, LocalStateInfo& info) override { + RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); + _spill_counters = ADD_LABEL_COUNTER_WITH_LEVEL(Base::profile(), "Spill", 1); + _spill_recover_time = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillRecoverTime", "Spill", 1); + _spill_read_data_time = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillReadDataTime", "Spill", 1); + _spill_deserialize_time = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillDeserializeTime", "Spill", 1); + _spill_read_bytes = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillReadDataSize", + TUnit::BYTES, "Spill", 1); + return Status::OK(); + } + + RuntimeProfile::Counter* _spill_counters = nullptr; + RuntimeProfile::Counter* _spill_recover_time; + RuntimeProfile::Counter* _spill_read_data_time; + RuntimeProfile::Counter* _spill_deserialize_time; + RuntimeProfile::Counter* _spill_read_bytes; +}; + class DataSinkOperatorXBase; class PipelineXSinkLocalStateBase { @@ -638,6 +667,38 @@ protected: SharedStateType* _shared_state = nullptr; }; +template +class PipelineXSpillSinkLocalState : public PipelineXSinkLocalState { +public: + using Base = PipelineXSinkLocalState; + PipelineXSpillSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : Base(parent, state) {} + ~PipelineXSpillSinkLocalState() override = default; + + Status init(RuntimeState* state, LocalSinkStateInfo& info) override { + RETURN_IF_ERROR(Base::init(state, info)); + + _spill_counters = ADD_LABEL_COUNTER_WITH_LEVEL(Base::profile(), "Spill", 1); + _spill_timer = ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillTime", "Spill", 1); + _spill_serialize_block_timer = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillSerializeBlockTime", "Spill", 1); + _spill_write_disk_timer = + ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteDiskTime", "Spill", 1); + _spill_data_size = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteDataSize", + TUnit::BYTES, "Spill", 1); + _spill_block_count = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteBlockCount", + TUnit::UNIT, "Spill", 1); + return Status::OK(); + } + + RuntimeProfile::Counter* _spill_counters = nullptr; + RuntimeProfile::Counter* _spill_timer = nullptr; + RuntimeProfile::Counter* _spill_serialize_block_timer = nullptr; + RuntimeProfile::Counter* _spill_write_disk_timer = nullptr; + RuntimeProfile::Counter* _spill_data_size = nullptr; + RuntimeProfile::Counter* _spill_block_count = nullptr; +}; + /** * StreamingOperatorX indicates operators which always processes block in streaming way (one-in-one-out). */ @@ -672,7 +733,8 @@ public: using OperatorX::get_local_state; - [[nodiscard]] Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos) final; + [[nodiscard]] Status get_block(RuntimeState* state, vectorized::Block* block, + bool* eos) override; [[nodiscard]] virtual Status pull(RuntimeState* state, vectorized::Block* block, bool* eos) const = 0; 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 1a7f141168..58fc3cb843 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -24,7 +24,6 @@ #include #include -#include // IWYU pragma: no_include #include // IWYU pragma: keep #include @@ -64,6 +63,10 @@ #include "pipeline/exec/olap_table_sink_v2_operator.h" #include "pipeline/exec/partition_sort_sink_operator.h" #include "pipeline/exec/partition_sort_source_operator.h" +#include "pipeline/exec/partitioned_aggregation_sink_operator.h" +#include "pipeline/exec/partitioned_aggregation_source_operator.h" +#include "pipeline/exec/partitioned_hash_join_probe_operator.h" +#include "pipeline/exec/partitioned_hash_join_sink_operator.h" #include "pipeline/exec/repeat_operator.h" #include "pipeline/exec/result_file_sink_operator.h" #include "pipeline/exec/result_sink_operator.h" @@ -74,6 +77,8 @@ #include "pipeline/exec/set_source_operator.h" #include "pipeline/exec/sort_sink_operator.h" #include "pipeline/exec/sort_source_operator.h" +#include "pipeline/exec/spill_sort_sink_operator.h" +#include "pipeline/exec/spill_sort_source_operator.h" #include "pipeline/exec/streaming_aggregation_operator.h" #include "pipeline/exec/table_function_operator.h" #include "pipeline/exec/union_sink_operator.h" @@ -507,6 +512,7 @@ Status PipelineXFragmentContext::_build_pipeline_tasks( runtime_state->set_per_fragment_instance_idx(local_params.sender_id); runtime_state->set_num_per_fragment_instances(request.num_senders); runtime_state->resize_op_id_to_local_state(max_operator_id()); + runtime_state->set_max_operator_id(max_operator_id()); runtime_state->set_load_stream_per_node(request.load_stream_per_node); runtime_state->set_total_load_streams(request.total_load_streams); runtime_state->set_num_local_sink(request.num_local_sink); @@ -573,6 +579,7 @@ Status PipelineXFragmentContext::_build_pipeline_tasks( auto& task_runtime_state = _task_runtime_states.back(); init_runtime_state(task_runtime_state); auto cur_task_id = _total_tasks++; + task_runtime_state->set_task_id(cur_task_id); auto task = std::make_unique( pipeline, cur_task_id, get_task_runtime_state(cur_task_id), this, pipeline_id_to_profile[pip_idx].get(), get_local_exchange_state(pipeline), @@ -981,7 +988,11 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN op.reset(new StreamingAggOperatorX(pool, next_operator_id(), tnode, descs)); RETURN_IF_ERROR(cur_pipe->add_operator(op)); } else { - op.reset(new AggSourceOperatorX(pool, tnode, next_operator_id(), descs)); + if (_runtime_state->enable_agg_spill() && !tnode.agg_node.grouping_exprs.empty()) { + op.reset(new PartitionedAggSourceOperatorX(pool, tnode, next_operator_id(), descs)); + } else { + op.reset(new AggSourceOperatorX(pool, tnode, next_operator_id(), descs)); + } RETURN_IF_ERROR(cur_pipe->add_operator(op)); const auto downstream_pipeline_id = cur_pipe->id(); @@ -992,7 +1003,12 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN _dag[downstream_pipeline_id].push_back(cur_pipe->id()); DataSinkOperatorXPtr sink; - sink.reset(new AggSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + if (_runtime_state->enable_agg_spill() && !tnode.agg_node.grouping_exprs.empty()) { + sink.reset(new PartitionedAggSinkOperatorX(pool, next_sink_operator_id(), tnode, + descs)); + } else { + sink.reset(new AggSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + } sink->set_dests_id({op->operator_id()}); RETURN_IF_ERROR(cur_pipe->set_sink(sink)); RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); @@ -1000,25 +1016,53 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN break; } case TPlanNodeType::HASH_JOIN_NODE: { - op.reset(new HashJoinProbeOperatorX(pool, tnode, next_operator_id(), descs)); - RETURN_IF_ERROR(cur_pipe->add_operator(op)); + const auto is_broadcast_join = tnode.hash_join_node.__isset.is_broadcast_join && + tnode.hash_join_node.is_broadcast_join; + const auto enable_join_spill = _runtime_state->enable_join_spill(); + if (enable_join_spill && !is_broadcast_join) { + const uint32_t partition_count = 16; + op.reset(new PartitionedHashJoinProbeOperatorX(pool, tnode, next_operator_id(), descs, + partition_count)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); - const auto downstream_pipeline_id = cur_pipe->id(); - if (_dag.find(downstream_pipeline_id) == _dag.end()) { - _dag.insert({downstream_pipeline_id, {}}); + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + PipelinePtr build_side_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); + + DataSinkOperatorXPtr sink; + sink.reset(new PartitionedHashJoinSinkOperatorX(pool, next_sink_operator_id(), tnode, + descs, _need_local_merge, + partition_count)); + sink->set_dests_id({op->operator_id()}); + RETURN_IF_ERROR(build_side_pipe->set_sink(sink)); + RETURN_IF_ERROR(build_side_pipe->sink_x()->init(tnode, _runtime_state.get())); + + _pipeline_parent_map.push(op->node_id(), cur_pipe); + _pipeline_parent_map.push(op->node_id(), build_side_pipe); + } else { + op.reset(new HashJoinProbeOperatorX(pool, tnode, next_operator_id(), descs)); + RETURN_IF_ERROR(cur_pipe->add_operator(op)); + + const auto downstream_pipeline_id = cur_pipe->id(); + if (_dag.find(downstream_pipeline_id) == _dag.end()) { + _dag.insert({downstream_pipeline_id, {}}); + } + PipelinePtr build_side_pipe = add_pipeline(cur_pipe); + _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); + + DataSinkOperatorXPtr sink; + sink.reset(new HashJoinBuildSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, + _need_local_merge)); + sink->set_dests_id({op->operator_id()}); + RETURN_IF_ERROR(build_side_pipe->set_sink(sink)); + RETURN_IF_ERROR(build_side_pipe->sink_x()->init(tnode, _runtime_state.get())); + + _pipeline_parent_map.push(op->node_id(), cur_pipe); + _pipeline_parent_map.push(op->node_id(), build_side_pipe); } - PipelinePtr build_side_pipe = add_pipeline(cur_pipe); - _dag[downstream_pipeline_id].push_back(build_side_pipe->id()); - - DataSinkOperatorXPtr sink; - sink.reset(new HashJoinBuildSinkOperatorX(pool, next_sink_operator_id(), tnode, descs, - _need_local_merge)); - sink->set_dests_id({op->operator_id()}); - RETURN_IF_ERROR(build_side_pipe->set_sink(sink)); - RETURN_IF_ERROR(build_side_pipe->sink_x()->init(tnode, _runtime_state.get())); - - _pipeline_parent_map.push(op->node_id(), cur_pipe); - _pipeline_parent_map.push(op->node_id(), build_side_pipe); break; } case TPlanNodeType::CROSS_JOIN_NODE: { @@ -1065,7 +1109,11 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN break; } case TPlanNodeType::SORT_NODE: { - op.reset(new SortSourceOperatorX(pool, tnode, next_operator_id(), descs)); + if (_runtime_state->enable_sort_spill()) { + op.reset(new SpillSortSourceOperatorX(pool, tnode, next_operator_id(), descs)); + } else { + op.reset(new SortSourceOperatorX(pool, tnode, next_operator_id(), descs)); + } RETURN_IF_ERROR(cur_pipe->add_operator(op)); const auto downstream_pipeline_id = cur_pipe->id(); @@ -1076,7 +1124,11 @@ Status PipelineXFragmentContext::_create_operator(ObjectPool* pool, const TPlanN _dag[downstream_pipeline_id].push_back(cur_pipe->id()); DataSinkOperatorXPtr sink; - sink.reset(new SortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + if (_runtime_state->enable_sort_spill()) { + sink.reset(new SpillSortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + } else { + sink.reset(new SortSinkOperatorX(pool, next_sink_operator_id(), tnode, descs)); + } sink->set_dests_id({op->operator_id()}); RETURN_IF_ERROR(cur_pipe->set_sink(sink)); RETURN_IF_ERROR(cur_pipe->sink_x()->init(tnode, _runtime_state.get())); diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp index 91a27b8e30..9620976f8a 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp @@ -36,6 +36,7 @@ #include "runtime/thread_context.h" #include "util/container_util.hpp" #include "util/defer_op.h" +#include "util/mem_info.h" #include "util/runtime_profile.h" namespace doris { @@ -267,6 +268,32 @@ Status PipelineXTask::execute(bool* eos) { _block->clear_column_data(_root->row_desc().num_materialized_slots()); auto* block = _block.get(); + auto sys_mem_available = doris::MemInfo::sys_mem_available(); + auto sys_mem_warning_water_mark = doris::MemInfo::sys_mem_available_warning_water_mark(); + auto query_mem = query_context()->query_mem_tracker->consumption(); + auto sink_revocable_mem_size = _sink->revocable_mem_size(_state); + if (sink_revocable_mem_size > 0) { + VLOG_ROW << "sys mem available: " + << PrettyPrinter::print(sys_mem_available, TUnit::BYTES) + << ",\nsys_mem_available_warning_water_mark: " + << PrettyPrinter::print(sys_mem_warning_water_mark, TUnit::BYTES) + << ",\nquery mem limit: " + << PrettyPrinter::print(_state->query_mem_limit(), TUnit::BYTES) + << ",\nquery mem: " << PrettyPrinter::print(query_mem, TUnit::BYTES) + << ",\nmin revocable mem: " + << PrettyPrinter::print(_state->min_revocable_mem(), TUnit::BYTES) + << ",\nrevocable mem: " + << PrettyPrinter::print(_sink->revocable_mem_size(_state), TUnit::BYTES); + } + if (sys_mem_available < sys_mem_warning_water_mark * config::spill_mem_warning_water_mark_multiplier /*&& + (double)query_mem >= (double)_state->query_mem_limit() * 0.8*/) { + if (_state->min_revocable_mem() > 0 && + sink_revocable_mem_size >= _state->min_revocable_mem()) { + RETURN_IF_ERROR(_sink->revoke_memory(_state)); + continue; + } + } + // Pull block from operator chain if (!_dry_run) { SCOPED_TIMER(_get_block_timer); diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 1fe5d14372..0b7e2101a4 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -43,6 +43,7 @@ namespace doris { namespace vectorized { class VDataStreamMgr; class ScannerScheduler; +class SpillStreamManager; class DeltaWriterV2Pool; } // namespace vectorized namespace pipeline { @@ -127,6 +128,7 @@ public: // Initial exec environment. must call this to init all [[nodiscard]] static Status init(ExecEnv* env, const std::vector& store_paths, + const std::vector& spill_store_paths, const std::set& broken_paths); // Stop all threads and delete resources. @@ -202,6 +204,7 @@ public: std::shared_ptr new_load_stream_mgr() { return _new_load_stream_mgr; } SmallFileMgr* small_file_mgr() { return _small_file_mgr; } BlockSpillManager* block_spill_mgr() { return _block_spill_mgr; } + doris::vectorized::SpillStreamManager* spill_stream_mgr() { return _spill_stream_mgr; } GroupCommitMgr* group_commit_mgr() { return _group_commit_mgr; } const std::vector& store_paths() const { return _store_paths; } @@ -285,6 +288,7 @@ private: ExecEnv(); [[nodiscard]] Status _init(const std::vector& store_paths, + const std::vector& spill_store_paths, const std::set& broken_paths); void _destroy(); @@ -295,6 +299,7 @@ private: inline static std::atomic_bool _s_ready {false}; std::vector _store_paths; + std::vector _spill_store_paths; io::FileCacheFactory* _file_cache_factory = nullptr; UserFunctionCache* _user_function_cache = nullptr; @@ -395,6 +400,7 @@ private: std::unique_ptr _pipeline_tracer_ctx; std::unique_ptr _tmp_file_dirs; + doris::vectorized::SpillStreamManager* _spill_stream_mgr = nullptr; }; template <> diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index de3b9a2f32..04c2faeed2 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -99,6 +99,7 @@ #include "vec/runtime/vdata_stream_mgr.h" #include "vec/sink/delta_writer_v2_pool.h" #include "vec/sink/load_stream_stub_pool.h" +#include "vec/spill/spill_stream_manager.h" #if !defined(__SANITIZE_ADDRESS__) && !defined(ADDRESS_SANITIZER) && !defined(LEAK_SANITIZER) && \ !defined(THREAD_SANITIZER) && !defined(USE_JEMALLOC) @@ -137,11 +138,13 @@ static void init_doris_metrics(const std::vector& store_paths) { } Status ExecEnv::init(ExecEnv* env, const std::vector& store_paths, + const std::vector& spill_store_paths, const std::set& broken_paths) { - return env->_init(store_paths, broken_paths); + return env->_init(store_paths, spill_store_paths, broken_paths); } Status ExecEnv::_init(const std::vector& store_paths, + const std::vector& spill_store_paths, const std::set& broken_paths) { //Only init once before be destroyed if (ready()) { @@ -203,6 +206,7 @@ Status ExecEnv::_init(const std::vector& store_paths, // NOTE: runtime query statistics mgr could be visited by query and daemon thread // so it should be created before all query begin and deleted after all query and daemon thread stoppped _runtime_query_statistics_mgr = new RuntimeQueryStatiticsMgr(); + init_file_cache_factory(); _pipeline_tracer_ctx = std::make_unique(); // before query RETURN_IF_ERROR(init_pipeline_task_scheduler()); @@ -228,6 +232,7 @@ Status ExecEnv::_init(const std::vector& store_paths, _load_stream_stub_pool = std::make_unique(); _delta_writer_v2_pool = std::make_unique(); _wal_manager = WalManager::create_shared(this, config::group_commit_wal_path); + _spill_stream_mgr = new vectorized::SpillStreamManager(spill_store_paths); _backend_client_cache->init_metrics("backend"); _frontend_client_cache->init_metrics("frontend"); @@ -246,7 +251,7 @@ Status ExecEnv::_init(const std::vector& store_paths, return status; } - static_cast(_init_mem_env()); + RETURN_IF_ERROR(_init_mem_env()); RETURN_IF_ERROR(_memtable_memory_limiter->init(MemInfo::mem_limit())); RETURN_IF_ERROR(_load_channel_mgr->init(MemInfo::mem_limit())); @@ -277,6 +282,8 @@ Status ExecEnv::_init(const std::vector& store_paths, _workload_sched_mgr = new WorkloadSchedPolicyMgr(); _workload_sched_mgr->start(this); + RETURN_IF_ERROR(_spill_stream_mgr->init()); + _s_ready = true; return Status::OK(); @@ -477,6 +484,7 @@ Status ExecEnv::_init_mem_env() { << ", origin config value: " << config::inverted_index_query_cache_limit; RETURN_IF_ERROR(_block_spill_mgr->init()); + return Status::OK(); } @@ -545,6 +553,7 @@ void ExecEnv::destroy() { _delta_writer_v2_pool.reset(); _load_stream_stub_pool.reset(); SAFE_STOP(_storage_engine); + SAFE_STOP(_spill_stream_mgr); SAFE_SHUTDOWN(_buffered_reader_prefetch_thread_pool); SAFE_SHUTDOWN(_s3_file_upload_thread_pool); SAFE_SHUTDOWN(_join_node_thread_pool); @@ -560,6 +569,7 @@ void ExecEnv::destroy() { // _experimental_mem_tracker.reset(); // _orphan_mem_tracker.reset(); + SAFE_DELETE(_spill_stream_mgr); SAFE_DELETE(_block_spill_mgr); SAFE_DELETE(_inverted_index_query_cache); SAFE_DELETE(_inverted_index_searcher_cache); diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 156a0998d4..e14721d485 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -108,6 +108,13 @@ public: return _query_options.__isset.scan_queue_mem_limit ? _query_options.scan_queue_mem_limit : _query_options.mem_limit / 20; } + int64_t query_mem_limit() const { + if (_query_options.__isset.mem_limit && (_query_options.mem_limit > 0)) { + return _query_options.mem_limit; + } + return 0; + } + ObjectPool* obj_pool() const { return _obj_pool.get(); } const DescriptorTbl& desc_tbl() const { return *_desc_tbl; } @@ -581,6 +588,33 @@ public: doris::IRuntimeFilter** producer_filter); bool is_nereids() const; + bool enable_join_spill() const { + return _query_options.__isset.enable_join_spill && _query_options.enable_join_spill; + } + + bool enable_sort_spill() const { + return _query_options.__isset.enable_sort_spill && _query_options.enable_sort_spill; + } + + bool enable_agg_spill() const { + return _query_options.__isset.enable_agg_spill && _query_options.enable_agg_spill; + } + + int64_t min_revocable_mem() const { + if (_query_options.__isset.min_revocable_mem) { + return _query_options.min_revocable_mem; + } + return 0; + } + + void set_max_operator_id(int max_operator_id) { _max_operator_id = max_operator_id; } + + int max_operator_id() const { return _max_operator_id; } + + void set_task_id(int id) { _task_id = id; } + + int task_id() const { return _task_id; } + private: Status create_error_log_file(); @@ -689,6 +723,8 @@ private: std::ofstream* _error_log_file = nullptr; // error file path, absolute path std::vector _tablet_commit_infos; std::vector _error_tablet_infos; + int _max_operator_id = 0; + int _task_id = -1; std::vector> _op_id_to_local_state; diff --git a/be/src/service/doris_main.cpp b/be/src/service/doris_main.cpp index a3ea8f14e0..a970a52fd0 100644 --- a/be/src/service/doris_main.cpp +++ b/be/src/service/doris_main.cpp @@ -404,6 +404,14 @@ int main(int argc, char** argv) { LOG(ERROR) << "parse config storage path failed, path=" << doris::config::storage_root_path; exit(-1); } + + std::vector spill_paths; + olap_res = doris::parse_conf_store_paths(doris::config::spill_storage_root_path, &spill_paths); + if (!olap_res) { + LOG(ERROR) << "parse config spill storage path failed, path=" + << doris::config::spill_storage_root_path; + exit(-1); + } std::set broken_paths; doris::parse_conf_broken_store_paths(doris::config::broken_storage_path, &broken_paths); @@ -435,6 +443,25 @@ int main(int argc, char** argv) { exit(-1); } + it = spill_paths.begin(); + for (; it != spill_paths.end();) { + if (!doris::check_datapath_rw(it->path)) { + if (doris::config::ignore_broken_disk) { + LOG(WARNING) << "read write test file failed, path=" << it->path; + it = spill_paths.erase(it); + } else { + LOG(ERROR) << "read write test file failed, path=" << it->path; + exit(-1); + } + } else { + ++it; + } + } + if (spill_paths.empty()) { + LOG(ERROR) << "All spill disks are broken, exit."; + exit(-1); + } + // initialize libcurl here to avoid concurrent initialization auto curl_ret = curl_global_init(CURL_GLOBAL_ALL); if (curl_ret != 0) { @@ -485,8 +512,8 @@ int main(int argc, char** argv) { doris::ThreadLocalHandle::create_thread_local_if_not_exits(); // init exec env - auto exec_env(doris::ExecEnv::GetInstance()); - status = doris::ExecEnv::init(doris::ExecEnv::GetInstance(), paths, broken_paths); + auto* exec_env(doris::ExecEnv::GetInstance()); + status = doris::ExecEnv::init(doris::ExecEnv::GetInstance(), paths, spill_paths, broken_paths); if (status != Status::OK()) { LOG(ERROR) << "failed to init doris storage engine, res=" << status; exit(-1); diff --git a/be/src/vec/common/sort/partition_sorter.h b/be/src/vec/common/sort/partition_sorter.h index ab4b4dac77..77dcb68371 100644 --- a/be/src/vec/common/sort/partition_sorter.h +++ b/be/src/vec/common/sort/partition_sorter.h @@ -90,8 +90,6 @@ public: size_t data_size() const override { return _state->data_size(); } - bool is_spilled() const override { return false; } - Status partition_sort_read(Block* block, bool* eos, int batch_size); int64 get_output_rows() const { return _output_total_rows; } void reset_sorter_state(RuntimeState* runtime_state); diff --git a/be/src/vec/common/sort/sorter.cpp b/be/src/vec/common/sort/sorter.cpp index 644db862f0..53fc201123 100644 --- a/be/src/vec/common/sort/sorter.cpp +++ b/be/src/vec/common/sort/sorter.cpp @@ -27,14 +27,11 @@ #include #include "common/object_pool.h" -#include "runtime/block_spill_manager.h" #include "runtime/exec_env.h" #include "runtime/thread_context.h" #include "vec/columns/column.h" #include "vec/columns/column_nullable.h" #include "vec/core/block.h" -#include "vec/core/block_spill_reader.h" -#include "vec/core/block_spill_writer.h" #include "vec/core/column_with_type_and_name.h" #include "vec/core/sort_block.h" #include "vec/data_types/data_type.h" @@ -58,47 +55,28 @@ namespace doris::vectorized { // each sub block is serialized in PBlock format and appended // to the spill file. // -// This number specifies the maximum size of sub blocks -static constexpr int BLOCK_SPILL_BATCH_BYTES = 8 * 1024 * 1024; +void MergeSorterState::reset() { + auto empty_queue = std::priority_queue(); + priority_queue_.swap(empty_queue); + std::vector empty_cursors(0); + cursors_.swap(empty_cursors); + std::vector empty_blocks(0); + sorted_blocks_.swap(empty_blocks); + in_mem_sorted_bocks_size_ = 0; +} Status MergeSorterState::add_sorted_block(Block& block) { auto rows = block.rows(); if (0 == rows) { return Status::OK(); } - if (0 == avg_row_bytes_) { - avg_row_bytes_ = std::max((std::size_t)1, block.bytes() / rows); - spill_block_batch_size_ = (BLOCK_SPILL_BATCH_BYTES + avg_row_bytes_ - 1) / avg_row_bytes_; - } - - auto bytes_used = data_size(); - auto total_bytes_used = bytes_used + block.bytes(); - if (is_spilled_ || (external_sort_bytes_threshold_ > 0 && - total_bytes_used >= external_sort_bytes_threshold_)) { - is_spilled_ = true; - BlockSpillWriterUPtr spill_block_writer; - RETURN_IF_ERROR(ExecEnv::GetInstance()->block_spill_mgr()->get_writer( - spill_block_batch_size_, spill_block_writer, block_spill_profile_)); - - RETURN_IF_ERROR(spill_block_writer->write(block)); - spilled_sorted_block_streams_.emplace_back(spill_block_writer->get_id()); - - COUNTER_UPDATE(spilled_block_count_, 1); - COUNTER_UPDATE(spilled_original_block_size_, spill_block_writer->get_written_bytes()); - RETURN_IF_ERROR(spill_block_writer->close()); - - if (init_merge_sorted_block_) { - init_merge_sorted_block_ = false; - merge_sorted_block_ = block.clone_empty(); - } - } else { - sorted_blocks_.emplace_back(std::move(block)); - } + sorted_blocks_.emplace_back(std::move(block)); + in_mem_sorted_bocks_size_ += block.bytes(); num_rows_ += rows; return Status::OK(); } -void MergeSorterState::_build_merge_tree_not_spilled(const SortDescription& sort_description) { +Status MergeSorterState::build_merge_tree(const SortDescription& sort_description) { for (auto& block : sorted_blocks_) { cursors_.emplace_back(block, sort_description); } @@ -108,60 +86,28 @@ void MergeSorterState::_build_merge_tree_not_spilled(const SortDescription& sort priority_queue_.emplace(&cursor); } } -} -Status MergeSorterState::build_merge_tree(const SortDescription& sort_description) { - _build_merge_tree_not_spilled(sort_description); - - if (spilled_sorted_block_streams_.size() > 0) { - if (sorted_blocks_.size() > 0) { - BlockSpillWriterUPtr spill_block_writer; - RETURN_IF_ERROR(ExecEnv::GetInstance()->block_spill_mgr()->get_writer( - spill_block_batch_size_, spill_block_writer, block_spill_profile_)); - - if (sorted_blocks_.size() == 1) { - RETURN_IF_ERROR(spill_block_writer->write(sorted_blocks_[0])); - } else { - bool eos = false; - - // merge blocks in memory and write merge result to disk - while (!eos) { - merge_sorted_block_.clear_column_data(); - RETURN_IF_ERROR(_merge_sort_read_not_spilled(spill_block_batch_size_, - &merge_sorted_block_, &eos)); - RETURN_IF_ERROR(spill_block_writer->write(merge_sorted_block_)); - } - } - spilled_sorted_block_streams_.emplace_back(spill_block_writer->get_id()); - RETURN_IF_ERROR(spill_block_writer->close()); - } - RETURN_IF_ERROR(_merge_spilled_blocks(sort_description)); - } return Status::OK(); } -Status MergeSorterState::merge_sort_read(doris::RuntimeState* state, - doris::vectorized::Block* block, bool* eos) { - if (is_spilled_) { - RETURN_IF_ERROR(merger_->get_next(block, eos)); +Status MergeSorterState::merge_sort_read(doris::vectorized::Block* block, int batch_size, + bool* eos) { + if (sorted_blocks_.empty()) { + *eos = true; + } else if (sorted_blocks_.size() == 1) { + if (offset_ != 0) { + sorted_blocks_[0].skip_num_rows(offset_); + } + block->swap(sorted_blocks_[0]); + *eos = true; } else { - if (sorted_blocks_.empty()) { - *eos = true; - } else if (sorted_blocks_.size() == 1) { - if (offset_ != 0) { - sorted_blocks_[0].skip_num_rows(offset_); - } - block->swap(sorted_blocks_[0]); - *eos = true; - } else { - RETURN_IF_ERROR(_merge_sort_read_not_spilled(state->batch_size(), block, eos)); - } + RETURN_IF_ERROR(_merge_sort_read_impl(batch_size, block, eos)); } return Status::OK(); } -Status MergeSorterState::_merge_sort_read_not_spilled(int batch_size, - doris::vectorized::Block* block, bool* eos) { +Status MergeSorterState::_merge_sort_read_impl(int batch_size, doris::vectorized::Block* block, + bool* eos) { size_t num_columns = sorted_blocks_[0].columns(); MutableBlock m_block = VectorizedUtils::build_mutable_mem_reuse_block(block, sorted_blocks_[0]); @@ -198,61 +144,9 @@ Status MergeSorterState::_merge_sort_read_not_spilled(int batch_size, return Status::OK(); } -int MergeSorterState::_calc_spill_blocks_to_merge() const { - int count = external_sort_bytes_threshold_ / BLOCK_SPILL_BATCH_BYTES; - return std::max(2, count); -} - -// merge all the intermediate spilled blocks -Status MergeSorterState::_merge_spilled_blocks(const SortDescription& sort_description) { - int num_of_blocks_to_merge = _calc_spill_blocks_to_merge(); - while (true) { - // pick some spilled blocks to merge, and spill the merged result - // to disk, until all splled blocks can be merged in a run. - RETURN_IF_ERROR(_create_intermediate_merger(num_of_blocks_to_merge, sort_description)); - if (spilled_sorted_block_streams_.empty()) { - break; - } - - bool eos = false; - - BlockSpillWriterUPtr spill_block_writer; - RETURN_IF_ERROR(ExecEnv::GetInstance()->block_spill_mgr()->get_writer( - spill_block_batch_size_, spill_block_writer, block_spill_profile_)); - - while (!eos) { - merge_sorted_block_.clear_column_data(); - RETURN_IF_ERROR(merger_->get_next(&merge_sorted_block_, &eos)); - RETURN_IF_ERROR(spill_block_writer->write(merge_sorted_block_)); - } - spilled_sorted_block_streams_.emplace_back(spill_block_writer->get_id()); - RETURN_IF_ERROR(spill_block_writer->close()); - } - return Status::OK(); -} - -Status MergeSorterState::_create_intermediate_merger(int num_blocks, - const SortDescription& sort_description) { - spilled_block_readers_.clear(); - - std::vector child_block_suppliers; - merger_.reset(new VSortedRunMerger(sort_description, spill_block_batch_size_, limit_, offset_, - profile_)); - - for (int i = 0; i < num_blocks && !spilled_sorted_block_streams_.empty(); ++i) { - auto stream_id = spilled_sorted_block_streams_.front(); - BlockSpillReaderUPtr spilled_block_reader; - RETURN_IF_ERROR(ExecEnv::GetInstance()->block_spill_mgr()->get_reader( - stream_id, spilled_block_reader, block_spill_profile_)); - child_block_suppliers.emplace_back(std::bind(std::mem_fn(&BlockSpillReader::read), - spilled_block_reader.get(), - std::placeholders::_1, std::placeholders::_2)); - spilled_block_readers_.emplace_back(std::move(spilled_block_reader)); - - spilled_sorted_block_streams_.pop_front(); - } - RETURN_IF_ERROR(merger_->prepare(child_block_suppliers)); - return Status::OK(); +Status Sorter::merge_sort_read_for_spill(RuntimeState* state, doris::vectorized::Block* block, + int batch_size, bool* eos) { + return get_next(state, block, eos); } Status Sorter::partial_sort(Block& src_block, Block& dest_block) { @@ -345,7 +239,12 @@ Status FullSorter::prepare_for_read() { } Status FullSorter::get_next(RuntimeState* state, Block* block, bool* eos) { - return _state->merge_sort_read(state, block, eos); + return _state->merge_sort_read(block, state->batch_size(), eos); +} + +Status FullSorter::merge_sort_read_for_spill(RuntimeState* state, doris::vectorized::Block* block, + int batch_size, bool* eos) { + return _state->merge_sort_read(block, batch_size, eos); } Status FullSorter::_do_sort() { @@ -354,47 +253,37 @@ Status FullSorter::_do_sort() { RETURN_IF_ERROR(partial_sort(*src_block, desc_block)); // dispose TOP-N logic - if (_limit != -1 && !_state->is_spilled()) { + if (_limit != -1 && !_enable_spill) { // Here is a little opt to reduce the mem usage, we build a max heap // to order the block in _block_priority_queue. // if one block totally greater the heap top of _block_priority_queue // we can throw the block data directly. if (_state->num_rows() < _offset + _limit) { static_cast(_state->add_sorted_block(desc_block)); - // if it's spilled, sorted_block is not added into sorted block vector, - // so it's should not be added to _block_priority_queue, since - // sorted_block will be destroyed when _do_sort is finished - if (!_state->is_spilled()) { - _block_priority_queue.emplace(_pool->add( - new MergeSortCursorImpl(_state->last_sorted_block(), _sort_description))); - } + _block_priority_queue.emplace(_pool->add( + new MergeSortCursorImpl(_state->last_sorted_block(), _sort_description))); } else { auto tmp_cursor_impl = std::make_unique(desc_block, _sort_description); MergeSortBlockCursor block_cursor(tmp_cursor_impl.get()); if (!block_cursor.totally_greater(_block_priority_queue.top())) { static_cast(_state->add_sorted_block(desc_block)); - if (!_state->is_spilled()) { - _block_priority_queue.emplace(_pool->add(new MergeSortCursorImpl( - _state->last_sorted_block(), _sort_description))); - } + _block_priority_queue.emplace(_pool->add( + new MergeSortCursorImpl(_state->last_sorted_block(), _sort_description))); } } } else { // dispose normal sort logic static_cast(_state->add_sorted_block(desc_block)); } - if (_state->is_spilled()) { - std::priority_queue tmp; - _block_priority_queue.swap(tmp); - - buffered_block_size_ = SPILL_BUFFERED_BLOCK_SIZE; - buffered_block_bytes_ = SPILL_BUFFERED_BLOCK_BYTES; - } return Status::OK(); } size_t FullSorter::data_size() const { return _state->data_size(); } +void FullSorter::reset() { + _state->reset(); +} + } // namespace doris::vectorized diff --git a/be/src/vec/common/sort/sorter.h b/be/src/vec/common/sort/sorter.h index e0a2b92cee..2525ca8c0c 100644 --- a/be/src/vec/common/sort/sorter.h +++ b/be/src/vec/common/sort/sorter.h @@ -30,7 +30,6 @@ #include "util/runtime_profile.h" #include "vec/common/sort/vsort_exec_exprs.h" #include "vec/core/block.h" -#include "vec/core/block_spill_reader.h" #include "vec/core/field.h" #include "vec/core/sort_cursor.h" #include "vec/core/sort_description.h" @@ -56,17 +55,7 @@ public: // since block from child node may ignored these slots : unsorted_block_(Block::create_unique( VectorizedUtils::create_empty_block(row_desc, true /*ignore invalid slot*/))), - offset_(offset), - limit_(limit), - profile_(profile) { - external_sort_bytes_threshold_ = state->external_sort_bytes_threshold(); - if (profile != nullptr) { - block_spill_profile_ = profile->create_child("BlockSpill", true, true); - spilled_block_count_ = ADD_COUNTER(block_spill_profile_, "BlockCount", TUnit::UNIT); - spilled_original_block_size_ = - ADD_COUNTER(block_spill_profile_, "BlockBytes", TUnit::BYTES); - } - } + offset_(offset) {} ~MergeSorterState() = default; @@ -74,62 +63,39 @@ public: Status build_merge_tree(const SortDescription& sort_description); - Status merge_sort_read(doris::RuntimeState* state, doris::vectorized::Block* block, bool* eos); + Status merge_sort_read(doris::vectorized::Block* block, int batch_size, bool* eos); size_t data_size() const { - size_t size = unsorted_block_->allocated_bytes(); - for (const auto& block : sorted_blocks_) { - size += block.allocated_bytes(); - } - return size; + size_t size = unsorted_block_->bytes(); + return size + in_mem_sorted_bocks_size_; } uint64_t num_rows() const { return num_rows_; } - bool is_spilled() const { return is_spilled_; } - Block& last_sorted_block() { return sorted_blocks_.back(); } std::vector& get_sorted_block() { return sorted_blocks_; } std::priority_queue& get_priority_queue() { return priority_queue_; } std::vector& get_cursors() { return cursors_; } + void reset(); std::unique_ptr unsorted_block_; private: int _calc_spill_blocks_to_merge() const; - void _build_merge_tree_not_spilled(const SortDescription& sort_description); - - Status _merge_sort_read_not_spilled(int batch_size, doris::vectorized::Block* block, bool* eos); - - Status _merge_spilled_blocks(const SortDescription& sort_description); - - Status _create_intermediate_merger(int num_blocks, const SortDescription& sort_description); + Status _merge_sort_read_impl(int batch_size, doris::vectorized::Block* block, bool* eos); std::priority_queue priority_queue_; std::vector cursors_; std::vector sorted_blocks_; + size_t in_mem_sorted_bocks_size_ = 0; uint64_t num_rows_ = 0; int64_t offset_; - int64_t limit_; - size_t avg_row_bytes_ = 0; - int spill_block_batch_size_ = 0; - int64_t external_sort_bytes_threshold_; - - bool is_spilled_ = false; - bool init_merge_sorted_block_ = true; - std::deque spilled_sorted_block_streams_; - std::vector spilled_block_readers_; Block merge_sorted_block_; std::unique_ptr merger_; - - RuntimeProfile* profile_ = nullptr; - RuntimeProfile* block_spill_profile_ = nullptr; - RuntimeProfile::Counter* spilled_block_count_ = nullptr; - RuntimeProfile::Counter* spilled_original_block_size_ = nullptr; }; class Sorter { @@ -159,15 +125,23 @@ public: virtual size_t data_size() const = 0; - virtual bool is_spilled() const { return false; } - // for topn runtime predicate - const SortDescription& get_sort_description() { return _sort_description; } + const SortDescription& get_sort_description() const { return _sort_description; } virtual Field get_top_value() { return Field {Field::Types::Null}; } + virtual Status merge_sort_read_for_spill(RuntimeState* state, doris::vectorized::Block* block, + int batch_size, bool* eos); + virtual void reset() {} + + int64_t limit() const { return _limit; } + int64_t offset() const { return _offset; } + + void set_enable_spill(bool b) { _enable_spill = b; } + protected: Status partial_sort(Block& src_block, Block& dest_block); + bool _enable_spill = false; SortDescription _sort_description; VSortExecExprs& _vsort_exec_exprs; int _limit; @@ -201,7 +175,9 @@ public: size_t data_size() const override; - bool is_spilled() const override { return _state->is_spilled(); } + Status merge_sort_read_for_spill(RuntimeState* state, doris::vectorized::Block* block, + int batch_size, bool* eos) override; + void reset() override; private: bool _reach_limit() { diff --git a/be/src/vec/common/sort/topn_sorter.cpp b/be/src/vec/common/sort/topn_sorter.cpp index e24c0d5e9d..58c3cd2dd0 100644 --- a/be/src/vec/common/sort/topn_sorter.cpp +++ b/be/src/vec/common/sort/topn_sorter.cpp @@ -58,7 +58,7 @@ Status TopNSorter::prepare_for_read() { } Status TopNSorter::get_next(RuntimeState* state, Block* block, bool* eos) { - return _state->merge_sort_read(state, block, eos); + return _state->merge_sort_read(block, state->batch_size(), eos); } Status TopNSorter::_do_sort(Block* block) { @@ -72,29 +72,18 @@ Status TopNSorter::_do_sort(Block* block) { // if one block totally greater the heap top of _block_priority_queue // we can throw the block data directly. if (_state->num_rows() < _offset + _limit) { - static_cast(_state->add_sorted_block(sorted_block)); - // if it's spilled, sorted_block is not added into sorted block vector, - // so it's should not be added to _block_priority_queue, since - // sorted_block will be destroyed when _do_sort is finished - if (!_state->is_spilled()) { + RETURN_IF_ERROR(_state->add_sorted_block(sorted_block)); + _block_priority_queue.emplace(_pool->add( + new MergeSortCursorImpl(_state->last_sorted_block(), _sort_description))); + } else { + auto tmp_cursor_impl = + std::make_unique(sorted_block, _sort_description); + MergeSortBlockCursor block_cursor(tmp_cursor_impl.get()); + if (!block_cursor.totally_greater(_block_priority_queue.top())) { + RETURN_IF_ERROR(_state->add_sorted_block(sorted_block)); _block_priority_queue.emplace(_pool->add( new MergeSortCursorImpl(_state->last_sorted_block(), _sort_description))); } - } else { - if (!_state->is_spilled()) { - auto tmp_cursor_impl = - std::make_unique(sorted_block, _sort_description); - MergeSortBlockCursor block_cursor(tmp_cursor_impl.get()); - if (!block_cursor.totally_greater(_block_priority_queue.top())) { - static_cast(_state->add_sorted_block(sorted_block)); - if (!_state->is_spilled()) { - _block_priority_queue.emplace(_pool->add(new MergeSortCursorImpl( - _state->last_sorted_block(), _sort_description))); - } - } - } else { - static_cast(_state->add_sorted_block(sorted_block)); - } } } else { return Status::InternalError("Should not reach TopN sorter for full sort query"); diff --git a/be/src/vec/common/sort/topn_sorter.h b/be/src/vec/common/sort/topn_sorter.h index 4fa7399329..39d6ef8a37 100644 --- a/be/src/vec/common/sort/topn_sorter.h +++ b/be/src/vec/common/sort/topn_sorter.h @@ -57,8 +57,6 @@ public: size_t data_size() const override; - bool is_spilled() const override { return _state->is_spilled(); } - static constexpr size_t TOPN_SORT_THRESHOLD = 256; private: diff --git a/be/src/vec/exec/vsort_node.cpp b/be/src/vec/exec/vsort_node.cpp index f49fbe99c6..b142f01169 100644 --- a/be/src/vec/exec/vsort_node.cpp +++ b/be/src/vec/exec/vsort_node.cpp @@ -201,9 +201,6 @@ Status VSortNode::pull(doris::RuntimeState* state, vectorized::Block* output_blo SCOPED_TIMER(_get_next_timer); RETURN_IF_ERROR_OR_CATCH_EXCEPTION(_sorter->get_next(state, output_block, eos)); reached_limit(output_block, eos); - if (*eos) { - _runtime_profile->add_info_string("Spilled", _sorter->is_spilled() ? "true" : "false"); - } return Status::OK(); } diff --git a/be/src/vec/spill/spill_reader.cpp b/be/src/vec/spill/spill_reader.cpp new file mode 100644 index 0000000000..54fb5535b9 --- /dev/null +++ b/be/src/vec/spill/spill_reader.cpp @@ -0,0 +1,147 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/spill/spill_reader.h" + +#include + +#include "common/exception.h" +#include "io/file_factory.h" +#include "io/fs/file_reader.h" +#include "io/fs/local_file_system.h" +#include "runtime/exec_env.h" +#include "util/slice.h" +#include "vec/core/block.h" +namespace doris { +namespace io { +class FileSystem; +} // namespace io + +namespace vectorized { +Status SpillReader::open() { + if (file_reader_) { + return Status::OK(); + } + std::shared_ptr file_system; + io::FileSystemProperties system_properties; + system_properties.system_type = TFileType::FILE_LOCAL; + + io::FileDescription file_description; + file_description.path = file_path_; + + SCOPED_TIMER(read_timer_); + RETURN_IF_ERROR(FileFactory::create_file_reader(system_properties, file_description, + io::FileReaderOptions::DEFAULT, &file_system, + &file_reader_)); + + size_t file_size = file_reader_->size(); + DCHECK(file_size >= 16); // max_sub_block_size, block count + + Slice result((char*)&block_count_, sizeof(size_t)); + + // read block count + size_t bytes_read = 0; + RETURN_IF_ERROR(file_reader_->read_at(file_size - sizeof(size_t), result, &bytes_read)); + DCHECK(bytes_read == 8); // max_sub_block_size, block count + + // read max sub block size + bytes_read = 0; + result.data = (char*)&max_sub_block_size_; + RETURN_IF_ERROR(file_reader_->read_at(file_size - sizeof(size_t) * 2, result, &bytes_read)); + DCHECK(bytes_read == 8); // max_sub_block_size, block count + + size_t buff_size = std::max(block_count_ * sizeof(size_t), max_sub_block_size_); + try { + read_buff_.reset(new char[buff_size]); + } catch (const std::bad_alloc&) { + LOG(INFO) << "spill max block size: " << max_sub_block_size_ + << ", block count: " << block_count_ << ", buff size: " << buff_size; + return Status::InternalError("bad alloc"); + } + + // read block start offsets + size_t read_offset = file_size - (block_count_ + 2) * sizeof(size_t); + result.data = read_buff_.get(); + result.size = block_count_ * sizeof(size_t); + + RETURN_IF_ERROR(file_reader_->read_at(read_offset, result, &bytes_read)); + DCHECK(bytes_read == block_count_ * sizeof(size_t)); + + block_start_offsets_.resize(block_count_ + 1); + for (size_t i = 0; i < block_count_; ++i) { + block_start_offsets_[i] = *(size_t*)(result.data + i * sizeof(size_t)); + } + block_start_offsets_[block_count_] = file_size - (block_count_ + 2) * sizeof(size_t); + + return Status::OK(); +} + +Status SpillReader::read(Block* block, bool* eos) { + DCHECK(file_reader_); + block->clear_column_data(); + + if (read_block_index_ >= block_count_) { + *eos = true; + return Status::OK(); + } + + size_t bytes_to_read = + block_start_offsets_[read_block_index_ + 1] - block_start_offsets_[read_block_index_]; + + if (bytes_to_read == 0) { + ++read_block_index_; + return Status::OK(); + } + + Slice result(read_buff_.get(), bytes_to_read); + size_t bytes_read = 0; + { + SCOPED_TIMER(read_timer_); + RETURN_IF_ERROR(file_reader_->read_at(block_start_offsets_[read_block_index_], result, + &bytes_read)); + } + DCHECK(bytes_read == bytes_to_read); + COUNTER_UPDATE(read_bytes_, bytes_read); + + if (bytes_read > 0) { + { + SCOPED_TIMER(deserialize_timer_); + if (!pb_block_.ParseFromArray(result.data, result.size)) { + return Status::InternalError("Failed to read spilled block"); + } + RETURN_IF_ERROR(block->deserialize(pb_block_)); + } + } else { + block->clear_column_data(); + } + + ++read_block_index_; + + return Status::OK(); +} + +Status SpillReader::close() { + if (!file_reader_) { + return Status::OK(); + } + (void)file_reader_->close(); + file_reader_.reset(); + return Status::OK(); +} + +} // namespace vectorized +} // namespace doris \ No newline at end of file diff --git a/be/src/vec/spill/spill_reader.h b/be/src/vec/spill/spill_reader.h new file mode 100644 index 0000000000..6694bf9157 --- /dev/null +++ b/be/src/vec/spill/spill_reader.h @@ -0,0 +1,81 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include +#include +#include + +#include "common/status.h" +#include "io/fs/file_reader_writer_fwd.h" +#include "util/runtime_profile.h" + +namespace doris::vectorized { +class Block; +class SpillReader { +public: + SpillReader(int64_t stream_id, std::string file_path) + : stream_id_(stream_id), file_path_(std::move(file_path)) {} + + ~SpillReader() { (void)close(); } + + Status open(); + + Status close(); + + Status read(Block* block, bool* eos); + + void seek(size_t block_index); + + int64_t get_id() const { return stream_id_; } + + std::string get_path() const { return file_path_; } + + size_t block_count() const { return block_count_; } + + void set_counters(RuntimeProfile::Counter* read_timer, + RuntimeProfile::Counter* deserialize_timer, + RuntimeProfile::Counter* read_bytes) { + read_timer_ = read_timer; + deserialize_timer_ = deserialize_timer; + read_bytes_ = read_bytes; + } + +private: + int64_t stream_id_; + std::string file_path_; + io::FileReaderSPtr file_reader_; + + size_t block_count_ = 0; + size_t read_block_index_ = 0; + size_t max_sub_block_size_ = 0; + std::unique_ptr read_buff_; + std::vector block_start_offsets_; + + PBlock pb_block_; + + RuntimeProfile::Counter* read_timer_; + RuntimeProfile::Counter* deserialize_timer_; + RuntimeProfile::Counter* read_bytes_; +}; + +using SpillReaderUPtr = std::unique_ptr; + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/spill/spill_stream.cpp b/be/src/vec/spill/spill_stream.cpp new file mode 100644 index 0000000000..d770e44a14 --- /dev/null +++ b/be/src/vec/spill/spill_stream.cpp @@ -0,0 +1,137 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/spill/spill_stream.h" + +#include + +#include +#include +#include + +#include "io/fs/local_file_system.h" +#include "runtime/exec_env.h" +#include "runtime/thread_context.h" +#include "vec/core/block.h" +#include "vec/spill/spill_reader.h" +#include "vec/spill/spill_stream_manager.h" +#include "vec/spill/spill_writer.h" + +namespace doris::vectorized { +SpillStream::SpillStream(RuntimeState* state, int64_t stream_id, doris::DataDir* data_dir, + std::string spill_dir, size_t batch_rows, size_t batch_bytes, + RuntimeProfile* profile) + : state_(state), + stream_id_(stream_id), + data_dir_(data_dir), + spill_dir_(std::move(spill_dir)), + batch_rows_(batch_rows), + batch_bytes_(batch_bytes), + profile_(profile) { + io_thread_pool_ = + ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(data_dir->path()); +} + +Status SpillStream::prepare() { + writer_ = std::make_unique(stream_id_, batch_rows_, data_dir_, spill_dir_); + + reader_ = std::make_unique(stream_id_, writer_->get_file_path()); + return Status::OK(); +} + +void SpillStream::close() { + if (closed_) { + return; + } + VLOG_ROW << "closing: " << stream_id_; + closed_ = true; + if (spill_promise_) { + spill_future_.wait(); + spill_promise_.reset(); + } + if (read_promise_) { + read_future_.wait(); + read_promise_.reset(); + } + + (void)writer_->close(); + (void)reader_->close(); +} + +Status SpillStream::prepare_spill() { + DCHECK(!spill_promise_); + RETURN_IF_ERROR(writer_->open()); + + spill_promise_ = std::make_unique>(); + spill_future_ = spill_promise_->get_future(); + return Status::OK(); +} +void SpillStream::end_spill(const Status& status) { + spill_promise_->set_value(status); +} + +Status SpillStream::wait_spill() { + if (spill_promise_) { + auto status = spill_future_.get(); + spill_promise_.reset(); + return status; + } + return Status::OK(); +} + +Status SpillStream::spill_block(const Block& block, bool eof) { + size_t written_bytes = 0; + RETURN_IF_ERROR(writer_->write(block, written_bytes)); + if (eof) { + return writer_->close(); + } + return Status::OK(); +} + +Status SpillStream::spill_eof() { + return writer_->close(); +} + +Status SpillStream::read_next_block_sync(Block* block, bool* eos) { + DCHECK(!read_promise_); + DCHECK(reader_ != nullptr); + Status status; + read_promise_ = std::make_unique>(); + read_future_ = read_promise_->get_future(); + // use thread pool to limit concurrent io tasks + status = io_thread_pool_->submit_func([this, block, eos] { + SCOPED_ATTACH_TASK(state_); + Status st; + Defer defer {[&]() { read_promise_->set_value(st); }}; + st = reader_->open(); + if (!st.ok()) { + return; + } + st = reader_->read(block, eos); + }); + if (!status.ok()) { + LOG(WARNING) << "read spill data failed: " << status; + read_promise_.reset(); + return status; + } + + status = read_future_.get(); + read_promise_.reset(); + return status; +} + +} // namespace doris::vectorized diff --git a/be/src/vec/spill/spill_stream.h b/be/src/vec/spill/spill_stream.h new file mode 100644 index 0000000000..579449e503 --- /dev/null +++ b/be/src/vec/spill/spill_stream.h @@ -0,0 +1,105 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once +#include +#include +#include + +#include "olap/data_dir.h" +#include "vec/spill/spill_reader.h" +#include "vec/spill/spill_writer.h" + +namespace doris { +class RuntimeProfile; +class DataDir; +class ThreadPool; + +namespace vectorized { + +class Block; + +class SpillStream { +public: + SpillStream(RuntimeState* state, int64_t stream_id, doris::DataDir* data_dir, + std::string spill_dir, size_t batch_rows, size_t batch_bytes, + RuntimeProfile* profile); + + int64_t id() const { return stream_id_; } + + DataDir* get_data_dir() const { return data_dir_; } + const std::string& get_spill_root_dir() const { return data_dir_->path(); } + + const std::string& get_spill_dir() const { return spill_dir_; } + + size_t get_written_bytes() const { return writer_->get_written_bytes(); } + + Status prepare_spill(); + + Status spill_block(const Block& block, bool eof); + + void end_spill(const Status& status); + + Status spill_eof(); + + Status wait_spill(); + + Status read_next_block_sync(Block* block, bool* eos); + + void set_write_counters(RuntimeProfile::Counter* serialize_timer, + RuntimeProfile::Counter* write_block_counter, + RuntimeProfile::Counter* write_bytes_counter, + RuntimeProfile::Counter* write_timer) { + writer_->set_counters(serialize_timer, write_block_counter, write_bytes_counter, + write_timer); + } + + void set_read_counters(RuntimeProfile::Counter* read_timer, + RuntimeProfile::Counter* deserialize_timer, + RuntimeProfile::Counter* read_bytes) { + reader_->set_counters(read_timer, deserialize_timer, read_bytes); + } + +private: + friend class SpillStreamManager; + + Status prepare(); + + void close(); + + RuntimeState* state_ = nullptr; + ThreadPool* io_thread_pool_; + int64_t stream_id_; + std::atomic_bool closed_ = false; + doris::DataDir* data_dir_ = nullptr; + std::string spill_dir_; + size_t batch_rows_; + size_t batch_bytes_; + + std::unique_ptr> spill_promise_; + std::future spill_future_; + std::unique_ptr> read_promise_; + std::future read_future_; + + SpillWriterUPtr writer_; + SpillReaderUPtr reader_; + + RuntimeProfile* profile_ = nullptr; +}; +using SpillStreamSPtr = std::shared_ptr; +} // namespace vectorized +} // namespace doris \ No newline at end of file diff --git a/be/src/vec/spill/spill_stream_manager.cpp b/be/src/vec/spill/spill_stream_manager.cpp new file mode 100644 index 0000000000..4dfd847482 --- /dev/null +++ b/be/src/vec/spill/spill_stream_manager.cpp @@ -0,0 +1,262 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/spill/spill_stream_manager.h" + +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "io/fs/file_system.h" +#include "io/fs/local_file_system.h" +#include "olap/data_dir.h" +#include "olap/olap_define.h" +#include "olap/storage_engine.h" +#include "util/runtime_profile.h" +#include "util/time.h" +#include "vec/spill/spill_stream.h" + +namespace doris::vectorized { + +SpillStreamManager::SpillStreamManager(const std::vector& paths) + : _spill_store_paths(paths), _stop_background_threads_latch(1) {} + +Status SpillStreamManager::init() { + LOG(INFO) << "init spill stream manager"; + RETURN_NOT_OK_STATUS_WITH_WARN(_init_spill_store_map(), "_init_spill_store_map failed"); + + int spill_io_thread_count = config::spill_io_thread_pool_per_disk_thread_num; + if (spill_io_thread_count <= 0) { + spill_io_thread_count = 2; + } + int pool_idx = 0; + for (const auto& path : _spill_store_paths) { + auto gc_dir_root_dir = fmt::format("{}/{}", path.path, SPILL_GC_DIR_PREFIX); + bool exists = true; + RETURN_IF_ERROR(io::global_local_filesystem()->exists(gc_dir_root_dir, &exists)); + if (!exists) { + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(gc_dir_root_dir)); + } + + auto spill_dir = fmt::format("{}/{}", path.path, SPILL_DIR_PREFIX); + RETURN_IF_ERROR(io::global_local_filesystem()->exists(spill_dir, &exists)); + if (!exists) { + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(spill_dir)); + } else { + auto suffix = ToStringFromUnixMillis(UnixMillis()); + auto gc_dir = fmt::format("{}/{}/{}", path.path, SPILL_GC_DIR_PREFIX, suffix); + if (std::filesystem::exists(gc_dir)) { + LOG(WARNING) << "gc dir already exists: " << gc_dir; + } + (void)io::global_local_filesystem()->rename(spill_dir, gc_dir); + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(spill_dir)); + } + + size_t spill_data_size = 0; + for (auto const& dir_entry : + std::filesystem::recursive_directory_iterator {gc_dir_root_dir}) { + if (dir_entry.is_regular_file()) { + spill_data_size += dir_entry.file_size(); + } + } + path_to_spill_data_size_[path.path] = spill_data_size; + + std::unique_ptr io_pool; + static_cast(ThreadPoolBuilder(fmt::format("SpillIOThreadPool-{}", pool_idx++)) + .set_min_threads(spill_io_thread_count) + .set_max_threads(spill_io_thread_count) + .set_max_queue_size(config::spill_io_thread_pool_queue_size) + .build(&io_pool)); + path_to_io_thread_pool_[path.path] = std::move(io_pool); + } + static_cast(ThreadPoolBuilder("SpillAsyncTaskThreadPool") + .set_min_threads(config::spill_async_task_thread_pool_thread_num) + .set_max_threads(config::spill_async_task_thread_pool_thread_num) + .set_max_queue_size(config::spill_async_task_thread_pool_queue_size) + .build(&async_task_thread_pool_)); + + RETURN_IF_ERROR(Thread::create( + "Spill", "spill_gc_thread", [this]() { this->_spill_gc_thread_callback(); }, + &_spill_gc_thread)); + LOG(INFO) << "spill gc thread started"; + return Status::OK(); +} + +// clean up stale spilled files +void SpillStreamManager::_spill_gc_thread_callback() { + while (!_stop_background_threads_latch.wait_for( + std::chrono::milliseconds(config::spill_gc_interval_ms))) { + gc(2000); + } +} + +Status SpillStreamManager::_init_spill_store_map() { + for (const auto& path : _spill_store_paths) { + auto store = std::make_unique(ExecEnv::GetInstance()->storage_engine().to_local(), + path.path, path.capacity_bytes, path.storage_medium); + auto st = store->init(false); + if (!st.ok()) { + LOG(WARNING) << "Store load failed, status=" << st.to_string() + << ", path=" << store->path(); + return st; + } + _spill_store_map.emplace(store->path(), std::move(store)); + } + + return Status::OK(); +} + +std::vector SpillStreamManager::_get_stores_for_spill( + TStorageMedium::type storage_medium) { + std::vector stores; + for (auto&& [_, store] : _spill_store_map) { + if (store->storage_medium() == storage_medium && !store->reach_capacity_limit(0)) { + stores.push_back(store.get()); + } + } + + std::sort(stores.begin(), stores.end(), + [](DataDir* a, DataDir* b) { return a->get_usage(0) < b->get_usage(0); }); + + size_t seventy_percent_index = stores.size(); + size_t eighty_five_percent_index = stores.size(); + for (size_t index = 0; index < stores.size(); index++) { + // If the usage of the store is less than 70%, we choose disk randomly. + if (stores[index]->get_usage(0) > 0.7 && seventy_percent_index == stores.size()) { + seventy_percent_index = index; + } + if (stores[index]->get_usage(0) > 0.85 && eighty_five_percent_index == stores.size()) { + eighty_five_percent_index = index; + break; + } + } + + std::random_device rd; + std::mt19937 g(rd()); + std::shuffle(stores.begin(), stores.begin() + seventy_percent_index, g); + std::shuffle(stores.begin() + seventy_percent_index, stores.begin() + eighty_five_percent_index, + g); + std::shuffle(stores.begin() + eighty_five_percent_index, stores.end(), g); + + return stores; +} + +Status SpillStreamManager::register_spill_stream(RuntimeState* state, SpillStreamSPtr& spill_stream, + std::string query_id, std::string operator_name, + int32_t node_id, int32_t batch_rows, + size_t batch_bytes, RuntimeProfile* profile) { + auto data_dirs = _get_stores_for_spill(TStorageMedium::type::SSD); + if (data_dirs.empty()) { + data_dirs = _get_stores_for_spill(TStorageMedium::type::HDD); + } + if (data_dirs.empty()) { + return Status::Error( + "no available disk can be used for spill."); + } + + int64_t id = id_++; + std::string spill_dir; + doris::DataDir* data_dir = nullptr; + for (auto& dir : data_dirs) { + data_dir = dir; + std::string spill_root_dir = fmt::format("{}/{}", data_dir->path(), SPILL_DIR_PREFIX); + spill_dir = fmt::format("{}/{}-{}-{}-{}-{}", spill_root_dir, query_id, operator_name, + node_id, state->task_id(), id); + auto st = io::global_local_filesystem()->create_directory(spill_dir); + if (!st.ok()) { + continue; + } + break; + } + if (!data_dir) { + return Status::Error( + "there is no available disk that can be used to spill."); + } + spill_stream = std::make_shared(state, id, data_dir, spill_dir, batch_rows, + batch_bytes, profile); + RETURN_IF_ERROR(spill_stream->prepare()); + return Status::OK(); +} + +void SpillStreamManager::delete_spill_stream(SpillStreamSPtr stream) { + stream->close(); + + auto gc_dir = fmt::format("{}/{}/{}", stream->get_data_dir()->path(), SPILL_GC_DIR_PREFIX, + std::filesystem::path(stream->get_spill_dir()).filename().string()); + (void)io::global_local_filesystem()->rename(stream->get_spill_dir(), gc_dir); +} + +void SpillStreamManager::gc(int64_t max_file_count) { + if (max_file_count < 1) { + return; + } + + bool exists = true; + int64_t count = 0; + for (const auto& path : _spill_store_paths) { + std::string gc_root_dir = fmt::format("{}/{}", path.path, SPILL_GC_DIR_PREFIX); + + std::error_code ec; + exists = std::filesystem::exists(gc_root_dir, ec); + if (ec || !exists) { + continue; + } + std::vector dirs; + auto st = io::global_local_filesystem()->list(gc_root_dir, false, &dirs, &exists); + if (!st.ok()) { + continue; + } + + for (const auto& dir : dirs) { + if (dir.is_file) { + continue; + } + std::string abs_dir = fmt::format("{}/{}", gc_root_dir, dir.file_name); + std::vector files; + st = io::global_local_filesystem()->list(abs_dir, true, &files, &exists); + if (!st.ok()) { + continue; + } + if (files.empty()) { + static_cast(io::global_local_filesystem()->delete_directory(abs_dir)); + if (count++ == max_file_count) { + return; + } + continue; + } + + int64_t data_size = 0; + Defer defer {[&]() { update_usage(path.path, -data_size); }}; + + for (const auto& file : files) { + auto abs_file_path = fmt::format("{}/{}", abs_dir, file.file_name); + data_size += file.file_size; + static_cast(io::global_local_filesystem()->delete_file(abs_file_path)); + if (count++ == max_file_count) { + return; + } + } + } + } +} +} // namespace doris::vectorized diff --git a/be/src/vec/spill/spill_stream_manager.h b/be/src/vec/spill/spill_stream_manager.h new file mode 100644 index 0000000000..2b41207203 --- /dev/null +++ b/be/src/vec/spill/spill_stream_manager.h @@ -0,0 +1,91 @@ + +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once +#include +#include +#include +#include +#include + +#include "olap/options.h" +#include "util/threadpool.h" +#include "vec/spill/spill_stream.h" +namespace doris { +class RuntimeProfile; + +namespace vectorized { + +class SpillStreamManager { +public: + SpillStreamManager(const std::vector& paths); + + Status init(); + + void stop() { + _stop_background_threads_latch.count_down(); + if (_spill_gc_thread) { + _spill_gc_thread->join(); + } + } + + // 创建SpillStream并登记 + Status register_spill_stream(RuntimeState* state, SpillStreamSPtr& spill_stream, + std::string query_id, std::string operator_name, int32_t node_id, + int32_t batch_rows, size_t batch_bytes, RuntimeProfile* profile); + + // 标记SpillStream需要被删除,在GC线程中异步删除落盘文件 + void delete_spill_stream(SpillStreamSPtr spill_stream); + + void gc(int64_t max_file_count); + + void update_usage(const std::string& path, int64_t incoming_data_size) { + path_to_spill_data_size_[path] += incoming_data_size; + } + + static bool reach_capacity_limit(size_t size, size_t incoming_data_size) { + return size + incoming_data_size > config::spill_storage_limit; + } + + int64_t spilled_data_size(const std::string& path) { return path_to_spill_data_size_[path]; } + + ThreadPool* get_spill_io_thread_pool(const std::string& path) const { + const auto it = path_to_io_thread_pool_.find(path); + DCHECK(it != path_to_io_thread_pool_.end()); + return it->second.get(); + } + ThreadPool* get_async_task_thread_pool() const { return async_task_thread_pool_.get(); } + +private: + Status _init_spill_store_map(); + void _spill_gc_thread_callback(); + std::vector _get_stores_for_spill(TStorageMedium::type storage_medium); + + std::vector _spill_store_paths; + std::unordered_map> _spill_store_map; + + CountDownLatch _stop_background_threads_latch; + std::unique_ptr async_task_thread_pool_; + std::unordered_map> path_to_io_thread_pool_; + std::unordered_map path_to_spill_data_size_; + scoped_refptr _spill_gc_thread; + + std::atomic_uint64_t id_ = 0; +}; +} // namespace vectorized +} // namespace doris \ No newline at end of file diff --git a/be/src/vec/spill/spill_writer.cpp b/be/src/vec/spill/spill_writer.cpp new file mode 100644 index 0000000000..6e048ecd0d --- /dev/null +++ b/be/src/vec/spill/spill_writer.cpp @@ -0,0 +1,160 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "vec/spill/spill_writer.h" + +#include "agent/be_exec_version_manager.h" +#include "io/file_factory.h" +#include "io/fs/local_file_system.h" +#include "io/fs/local_file_writer.h" +#include "olap/data_dir.h" +#include "runtime/exec_env.h" +#include "runtime/thread_context.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris::vectorized { +Status SpillWriter::open() { + if (file_writer_) { + return Status::OK(); + } + RETURN_IF_ERROR(FileFactory::create_file_writer(TFileType::FILE_LOCAL, ExecEnv::GetInstance(), + {}, {}, file_path_, 0, file_writer_)); + return Status::OK(); +} + +Status SpillWriter::close() { + if (closed_ || !file_writer_) { + return Status::OK(); + } + closed_ = true; + + tmp_block_.clear_column_data(); + + meta_.append((const char*)&max_sub_block_size_, sizeof(max_sub_block_size_)); + meta_.append((const char*)&written_blocks_, sizeof(written_blocks_)); + + // meta: block1 offset, block2 offset, ..., blockn offset, max_sub_block_size, n + { + SCOPED_TIMER(write_timer_); + RETURN_IF_ERROR(file_writer_->append(meta_)); + } + + total_written_bytes_ += meta_.size(); + COUNTER_UPDATE(write_bytes_counter_, meta_.size()); + + ExecEnv::GetInstance()->spill_stream_mgr()->update_usage(data_dir_->path(), meta_.size()); + + RETURN_IF_ERROR(file_writer_->close()); + + file_writer_.reset(); + return Status::OK(); +} + +Status SpillWriter::write(const Block& block, size_t& written_bytes) { + written_bytes = 0; + DCHECK(file_writer_); + auto rows = block.rows(); + // file format: block1, block2, ..., blockn, meta + if (rows <= batch_size_) { + return _write_internal(block, written_bytes); + } else { + if (is_first_write_) { + is_first_write_ = false; + tmp_block_ = block.clone_empty(); + } + + const auto& src_data = block.get_columns_with_type_and_name(); + + for (size_t row_idx = 0; row_idx < rows;) { + tmp_block_.clear_column_data(); + + auto& dst_data = tmp_block_.get_columns_with_type_and_name(); + + size_t block_rows = std::min(rows - row_idx, batch_size_); + RETURN_IF_CATCH_EXCEPTION({ + for (size_t col_idx = 0; col_idx < block.columns(); ++col_idx) { + dst_data[col_idx].column->assume_mutable()->insert_range_from( + *src_data[col_idx].column, row_idx, block_rows); + } + }); + + RETURN_IF_ERROR(_write_internal(tmp_block_, written_bytes)); + + row_idx += block_rows; + } + return Status::OK(); + } +} + +Status SpillWriter::_write_internal(const Block& block, size_t& written_bytes) { + size_t uncompressed_bytes = 0, compressed_bytes = 0; + + Status status; + std::string buff; + + if (block.rows() > 0) { + PBlock pblock; + { + SCOPED_TIMER(serialize_timer_); + status = block.serialize(BeExecVersionManager::get_newest_version(), &pblock, + &uncompressed_bytes, &compressed_bytes, + segment_v2::CompressionTypePB::LZ4); + RETURN_IF_ERROR(status); + if (!pblock.SerializeToString(&buff)) { + return Status::Error( + "serialize spill data error. [path={}]", file_path_); + } + } + auto* spill_stream_mgr = ExecEnv::GetInstance()->spill_stream_mgr(); + auto splled_data_size = spill_stream_mgr->spilled_data_size(data_dir_->path()); + if (spill_stream_mgr->reach_capacity_limit(splled_data_size, buff.size())) { + return Status::Error( + "spill data total size exceed limit, path: {}, size limit: {}, spill data " + "size: {}", + data_dir_->path(), PrettyPrinter::print_bytes(config::spill_storage_limit), + PrettyPrinter::print_bytes( + spill_stream_mgr->spilled_data_size(data_dir_->path()))); + } + + { + Defer defer {[&]() { + if (status.ok()) { + spill_stream_mgr->update_usage(data_dir_->path(), buff.size()); + } + }}; + { + SCOPED_TIMER(write_timer_); + status = file_writer_->append(buff); + RETURN_IF_ERROR(status); + } + } + } + + auto buff_size = buff.size(); + written_bytes += buff_size; + max_sub_block_size_ = std::max(max_sub_block_size_, buff_size); + + meta_.append((const char*)&total_written_bytes_, sizeof(size_t)); + COUNTER_UPDATE(write_bytes_counter_, buff_size); + COUNTER_UPDATE(write_block_counter_, 1); + total_written_bytes_ += buff_size; + ++written_blocks_; + + return Status::OK(); +} + +} // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/spill/spill_writer.h b/be/src/vec/spill/spill_writer.h new file mode 100644 index 0000000000..1ecda1aff9 --- /dev/null +++ b/be/src/vec/spill/spill_writer.h @@ -0,0 +1,92 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include +#include +#include + +#include "io/fs/file_writer.h" +#include "util/runtime_profile.h" +#include "vec/core/block.h" +namespace doris { +class DataDir; + +namespace vectorized { +class SpillWriter { +public: + SpillWriter(int64_t id, size_t batch_size, doris::DataDir* data_dir, const std::string& dir) + : data_dir_(data_dir), stream_id_(id), batch_size_(batch_size) { + file_path_ = dir + "/" + std::to_string(file_index_); + } + + ~SpillWriter() { (void)close(); } + + Status open(); + + Status close(); + + Status write(const Block& block, size_t& written_bytes); + + int64_t get_id() const { return stream_id_; } + + size_t get_written_bytes() const { return total_written_bytes_; } + + const std::string& get_file_path() const { return file_path_; } + + void set_counters(RuntimeProfile::Counter* serialize_timer, + RuntimeProfile::Counter* write_block_counter, + RuntimeProfile::Counter* write_bytes_counter, + RuntimeProfile::Counter* write_timer) { + serialize_timer_ = serialize_timer; + write_block_counter_ = write_block_counter; + write_bytes_counter_ = write_bytes_counter; + write_timer_ = write_timer; + } + +private: + void _init_profile(); + + Status _write_internal(const Block& block, size_t& written_bytes); + + // not owned, point to the data dir of this rowset + // for checking disk capacity when write data to disk. + doris::DataDir* data_dir_ = nullptr; + std::atomic_bool closed_ = false; + int64_t stream_id_; + size_t batch_size_; + size_t max_sub_block_size_ = 0; + int file_index_ = 0; + std::string file_path_; + std::unique_ptr file_writer_; + + size_t written_blocks_ = 0; + size_t total_written_bytes_ = 0; + std::string meta_; + + bool is_first_write_ = true; + Block tmp_block_; + + RuntimeProfile::Counter* write_bytes_counter_; + RuntimeProfile::Counter* serialize_timer_; + RuntimeProfile::Counter* write_timer_; + RuntimeProfile::Counter* write_block_counter_; +}; +using SpillWriterUPtr = std::unique_ptr; +} // namespace vectorized +} // namespace doris diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 634904202a..202dcfa167 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -364,10 +364,6 @@ public class SessionVariable implements Serializable, Writable { public static final String GROUP_CONCAT_MAX_LEN = "group_concat_max_len"; - public static final String EXTERNAL_SORT_BYTES_THRESHOLD = "external_sort_bytes_threshold"; - public static final String EXTERNAL_AGG_BYTES_THRESHOLD = "external_agg_bytes_threshold"; - public static final String EXTERNAL_AGG_PARTITION_BITS = "external_agg_partition_bits"; - public static final String ENABLE_TWO_PHASE_READ_OPT = "enable_two_phase_read_opt"; public static final String TOPN_OPT_LIMIT_THRESHOLD = "topn_opt_limit_threshold"; @@ -481,6 +477,15 @@ public class SessionVariable implements Serializable, Writable { public static final String HUGE_TABLE_DEFAULT_SAMPLE_ROWS = "huge_table_default_sample_rows"; public static final String HUGE_TABLE_LOWER_BOUND_SIZE_IN_BYTES = "huge_table_lower_bound_size_in_bytes"; + // for spill to disk + public static final String EXTERNAL_SORT_BYTES_THRESHOLD = "external_sort_bytes_threshold"; + public static final String EXTERNAL_AGG_BYTES_THRESHOLD = "external_agg_bytes_threshold"; + public static final String EXTERNAL_AGG_PARTITION_BITS = "external_agg_partition_bits"; + public static final String MIN_REVOCABLE_MEM = "min_revocable_mem"; + public static final String ENABLE_JOIN_SPILL = "enable_join_spill"; + public static final String ENABLE_SORT_SPILL = "enable_sort_spill"; + public static final String ENABLE_AGG_SPILL = "enable_agg_spill"; + public static final String GENERATE_STATS_FACTOR = "generate_stats_factor"; public static final String HUGE_TABLE_AUTO_ANALYZE_INTERVAL_IN_MILLIS @@ -1270,25 +1275,6 @@ public class SessionVariable implements Serializable, Writable { @VariableMgr.VarAttr(name = GROUP_CONCAT_MAX_LEN) public long groupConcatMaxLen = 2147483646; - // If the memory consumption of sort node exceed this limit, will trigger spill to disk; - // Set to 0 to disable; min: 128M - public static final long MIN_EXTERNAL_SORT_BYTES_THRESHOLD = 134217728; - @VariableMgr.VarAttr(name = EXTERNAL_SORT_BYTES_THRESHOLD, - checker = "checkExternalSortBytesThreshold", fuzzy = true) - public long externalSortBytesThreshold = 0; - - // Set to 0 to disable; min: 128M - public static final long MIN_EXTERNAL_AGG_BYTES_THRESHOLD = 134217728; - @VariableMgr.VarAttr(name = EXTERNAL_AGG_BYTES_THRESHOLD, - checker = "checkExternalAggBytesThreshold", fuzzy = true) - public long externalAggBytesThreshold = 0; - - public static final int MIN_EXTERNAL_AGG_PARTITION_BITS = 4; - public static final int MAX_EXTERNAL_AGG_PARTITION_BITS = 8; - @VariableMgr.VarAttr(name = EXTERNAL_AGG_PARTITION_BITS, - checker = "checkExternalAggPartitionBits", fuzzy = true) - public int externalAggPartitionBits = 8; // means that the hash table will be partitioned into 256 blocks. - // Whether enable two phase read optimization // 1. read related rowids along with necessary column data // 2. spawn fetch RPC to other nodes to get related data by sorted rowids @@ -1664,6 +1650,65 @@ public class SessionVariable implements Serializable, Writable { public boolean disableEmptyPartitionPrune = false; // CLOUD_VARIABLES_END + // for spill to disk + @VariableMgr.VarAttr(name = MIN_REVOCABLE_MEM, fuzzy = true) + public long minRevocableMem = 32 * 1024 * 1024; + + @VariableMgr.VarAttr( + name = ENABLE_JOIN_SPILL, + description = {"控制是否启用join算子落盘。默认为 false。", + "Controls whether to enable spill to disk of join operation. " + + "The default value is false."}, + needForward = true) + public boolean enableJoinSpill = false; + + @VariableMgr.VarAttr( + name = ENABLE_SORT_SPILL, + description = {"控制是否启用排序算子落盘。默认为 false。", + "Controls whether to enable spill to disk of sort operation. " + + "The default value is false."}, + needForward = true) + public boolean enableSortSpill = false; + + @VariableMgr.VarAttr( + name = ENABLE_AGG_SPILL, + description = {"控制是否启用聚合算子落盘。默认为 false。", + "Controls whether to enable spill to disk of aggregation operation. " + + "The default value is false."}, + needForward = true) + public boolean enableAggSpill = false; + + // If the memory consumption of sort node exceed this limit, will trigger spill to disk; + // Set to 0 to disable; min: 128M + public static final long MIN_EXTERNAL_SORT_BYTES_THRESHOLD = 2097152; + @VariableMgr.VarAttr(name = EXTERNAL_SORT_BYTES_THRESHOLD, + checker = "checkExternalSortBytesThreshold", fuzzy = true) + public long externalSortBytesThreshold = 0; + + // Set to 0 to disable; min: 128M + public static final long MIN_EXTERNAL_AGG_BYTES_THRESHOLD = 134217728; + @VariableMgr.VarAttr(name = EXTERNAL_AGG_BYTES_THRESHOLD, + checker = "checkExternalAggBytesThreshold", fuzzy = true) + public long externalAggBytesThreshold = 0; + + public static final int MIN_EXTERNAL_AGG_PARTITION_BITS = 4; + public static final int MAX_EXTERNAL_AGG_PARTITION_BITS = 20; + @VariableMgr.VarAttr(name = EXTERNAL_AGG_PARTITION_BITS, + checker = "checkExternalAggPartitionBits", fuzzy = true) + public int externalAggPartitionBits = 8; // means that the hash table will be partitioned into 256 blocks. + + public boolean isEnableJoinSpill() { + return enableJoinSpill; + } + + public void setEnableJoinSpill(boolean enableJoinSpill) { + this.enableJoinSpill = enableJoinSpill; + } + + public boolean isEnableSortSpill() { + return enableSortSpill; + } + // If this fe is in fuzzy mode, then will use initFuzzyModeVariables to generate some variables, // not the default value set in the code. @SuppressWarnings("checkstyle:Indentation") @@ -2939,7 +2984,7 @@ public class SessionVariable implements Serializable, Writable { tResult.setRepeatMaxNum(repeatMaxNum); - tResult.setExternalSortBytesThreshold(0); // disable for now + tResult.setExternalSortBytesThreshold(externalSortBytesThreshold); tResult.setExternalAggBytesThreshold(0); // disable for now @@ -2979,7 +3024,10 @@ public class SessionVariable implements Serializable, Writable { tResult.setParallelScanMaxScannersCount(parallelScanMaxScannersCount); tResult.setParallelScanMinRowsPerScanner(parallelScanMinRowsPerScanner); tResult.setSkipBadTablet(skipBadTablet); - + tResult.setEnableJoinSpill(enableJoinSpill); + tResult.setEnableSortSpill(enableSortSpill); + tResult.setEnableAggSpill(enableAggSpill); + tResult.setMinRevocableMem(minRevocableMem); return tResult; } diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index e846b32bff..7004fdc05d 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -276,6 +276,14 @@ struct TQueryOptions { 100: optional bool enable_distinct_streaming_aggregation = true; + 101: optional bool enable_join_spill = false + + 102: optional bool enable_sort_spill = false + + 103: optional bool enable_agg_spill = false + + 104: optional i64 min_revocable_mem = 0 + // For cloud, to control if the content would be written into file cache 1000: optional bool disable_file_cache = false }