diff --git a/be/benchmark/benchmark_main.cpp b/be/benchmark/benchmark_main.cpp index ed77fa6e79e419..32add5a16621b9 100644 --- a/be/benchmark/benchmark_main.cpp +++ b/be/benchmark/benchmark_main.cpp @@ -21,8 +21,8 @@ #include "benchmark_fastunion.hpp" #include "benchmark_hll_merge.hpp" #include "binary_cast_benchmark.hpp" +#include "core/block/block.h" #include "vec/columns/column_string.h" -#include "vec/core/block.h" #include "vec/data_types/data_type.h" #include "vec/data_types/data_type_string.h" diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 6865b28a4eb485..51b7c44a8540ea 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1434,9 +1434,10 @@ DEFINE_mDouble(high_disk_avail_level_diff_usages, "0.15"); DEFINE_Int32(partition_disk_index_lru_size, "10000"); // limit the storage space that query spill files can use DEFINE_String(spill_storage_root_path, ""); -DEFINE_String(spill_storage_limit, "20%"); // 20% -DEFINE_mInt32(spill_gc_interval_ms, "2000"); // 2s -DEFINE_mInt32(spill_gc_work_time_ms, "2000"); // 2s +DEFINE_String(spill_storage_limit, "20%"); // 20% +DEFINE_mInt32(spill_gc_interval_ms, "2000"); // 2s +DEFINE_mInt32(spill_gc_work_time_ms, "2000"); // 2s +DEFINE_mInt64(spill_file_part_size_bytes, "1073741824"); // 1GB // paused query in queue timeout(ms) will be resumed or canceled DEFINE_Int64(spill_in_paused_queue_timeout_ms, "60000"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 60a3cdaca325cd..42ddcdd3ff99c1 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1538,6 +1538,8 @@ DECLARE_String(spill_storage_root_path); DECLARE_String(spill_storage_limit); DECLARE_mInt32(spill_gc_interval_ms); DECLARE_mInt32(spill_gc_work_time_ms); +// Maximum size of each spill part file before rotation (bytes). Default 1GB. +DECLARE_mInt64(spill_file_part_size_bytes); DECLARE_Int64(spill_in_paused_queue_timeout_ms); DECLARE_Int64(wait_cancel_release_memory_ms); diff --git a/be/src/exec/operator/aggregation_sink_operator.cpp b/be/src/exec/operator/aggregation_sink_operator.cpp index 35f1c49ef04e6a..39414e37aa0f2e 100644 --- a/be/src/exec/operator/aggregation_sink_operator.cpp +++ b/be/src/exec/operator/aggregation_sink_operator.cpp @@ -202,7 +202,7 @@ Status AggSinkLocalState::_merge_with_serialized_key(Block* block) { } size_t AggSinkLocalState::_memory_usage() const { - if (0 == _get_hash_table_size()) { + if (0 == get_hash_table_size()) { return 0; } size_t usage = 0; @@ -380,7 +380,7 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(Block* block) { } if (!limit && _should_limit_output) { - const size_t hash_table_size = _get_hash_table_size(); + const size_t hash_table_size = get_hash_table_size(); _shared_state->reach_limit = hash_table_size >= Base::_parent->template cast()._limit; if (_shared_state->do_sort_limit && _shared_state->reach_limit) { @@ -499,7 +499,7 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(Block* block) { RETURN_IF_ERROR(do_aggregate_evaluators()); if (_should_limit_output && !Base::_shared_state->enable_spill) { - const size_t hash_table_size = _get_hash_table_size(); + const size_t hash_table_size = get_hash_table_size(); _shared_state->reach_limit = hash_table_size >= @@ -516,7 +516,7 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(Block* block) { return Status::OK(); } -size_t AggSinkLocalState::_get_hash_table_size() const { +size_t AggSinkLocalState::get_hash_table_size() const { return std::visit(Overload {[&](std::monostate& arg) -> size_t { return 0; }, [&](auto& agg_method) { return agg_method.hash_table->size(); }}, _agg_data->method_variant); @@ -872,7 +872,7 @@ Status AggSinkOperatorX::sink(doris::RuntimeState* state, Block* in_block, bool RETURN_IF_ERROR(local_state._executor->execute(&local_state, in_block)); local_state._executor->update_memusage(&local_state); COUNTER_SET(local_state._hash_table_size_counter, - (int64_t)local_state._get_hash_table_size()); + (int64_t)local_state.get_hash_table_size()); } if (eos) { local_state._dependency->set_ready_to_read(); @@ -899,6 +899,11 @@ size_t AggSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { return local_state.get_reserve_mem_size(state, eos); } +size_t AggSinkOperatorX::get_hash_table_size(RuntimeState* state) const { + auto& local_state = get_local_state(state); + return local_state.get_hash_table_size(); +} + Status AggSinkLocalState::close(RuntimeState* state, Status exec_status) { SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_close_timer); diff --git a/be/src/exec/operator/aggregation_sink_operator.h b/be/src/exec/operator/aggregation_sink_operator.h index f90671cb465e2d..9774d2b95e512a 100644 --- a/be/src/exec/operator/aggregation_sink_operator.h +++ b/be/src/exec/operator/aggregation_sink_operator.h @@ -38,6 +38,7 @@ class AggSinkLocalState : public PipelineXSinkLocalState { Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; bool is_blockable() const override; + size_t get_hash_table_size() const; protected: friend class AggSinkOperatorX; @@ -81,6 +82,7 @@ class AggSinkLocalState : public PipelineXSinkLocalState { Status _merge_with_serialized_key(Block* block); void _update_memusage_with_serialized_key(); template + Status _execute_with_serialized_key_helper(Block* block); void _find_in_hash_table(AggregateDataPtr* places, ColumnRawPtrs& key_columns, uint32_t num_rows); @@ -89,7 +91,6 @@ class AggSinkLocalState : public PipelineXSinkLocalState { bool _emplace_into_hash_table_limit(AggregateDataPtr* places, Block* block, const std::vector& key_locs, ColumnRawPtrs& key_columns, uint32_t num_rows); - size_t _get_hash_table_size() const; template Status _merge_with_serialized_key_helper(Block* block); @@ -119,7 +120,7 @@ class AggSinkLocalState : public PipelineXSinkLocalState { PODArray _places; std::vector _deserialize_buffer; - Block _preagg_block = Block(); + Block _preagg_block; AggregatedDataVariants* _agg_data = nullptr; @@ -178,6 +179,8 @@ class AggSinkOperatorX MOCK_REMOVE(final) : public DataSinkOperatorX::node_id; using DataSinkOperatorX::operator_id; using DataSinkOperatorX::get_local_state; diff --git a/be/src/exec/operator/aggregation_source_operator.cpp b/be/src/exec/operator/aggregation_source_operator.cpp index 1042e5cc732091..a00a48a030089a 100644 --- a/be/src/exec/operator/aggregation_source_operator.cpp +++ b/be/src/exec/operator/aggregation_source_operator.cpp @@ -541,6 +541,12 @@ Status AggSourceOperatorX::reset_hash_table(RuntimeState* state) { return Status::OK(); } +Status AggSourceOperatorX::get_serialized_block(RuntimeState* state, Block* block, bool* eos) { + auto& local_state = get_local_state(state); + // Always use the serialized intermediate output path, regardless of _needs_finalize. + return local_state._get_results_with_serialized_key(state, block, eos); +} + void AggLocalState::_emplace_into_hash_table(AggregateDataPtr* places, ColumnRawPtrs& key_columns, uint32_t num_rows) { std::visit( diff --git a/be/src/exec/operator/aggregation_source_operator.h b/be/src/exec/operator/aggregation_source_operator.h index e2306a2a081f47..e6443b2a77e3fd 100644 --- a/be/src/exec/operator/aggregation_source_operator.h +++ b/be/src/exec/operator/aggregation_source_operator.h @@ -113,6 +113,13 @@ class AggSourceOperatorX : public OperatorX { Status reset_hash_table(RuntimeState* state); + /// Get a block of serialized intermediate aggregate states from the hash table. + /// Unlike get_block() which may finalize, this always outputs the serialized + /// intermediate format (key columns + serialized agg state columns), which is + /// the same format as the spill block. This is needed for repartitioning during + /// multi-level spill recovery: the data must be re-mergeable after repartitioning. + Status get_serialized_block(RuntimeState* state, Block* block, bool* eos); + private: friend class AggLocalState; diff --git a/be/src/exec/operator/multi_cast_data_stream_sink.cpp b/be/src/exec/operator/multi_cast_data_stream_sink.cpp index fbfb66482bc6cf..437416b1b23a67 100644 --- a/be/src/exec/operator/multi_cast_data_stream_sink.cpp +++ b/be/src/exec/operator/multi_cast_data_stream_sink.cpp @@ -52,7 +52,6 @@ std::shared_ptr MultiCastDataStreamSinkOperatorX::create_share Status MultiCastDataStreamSinkLocalState::open(RuntimeState* state) { RETURN_IF_ERROR(Base::open(state)); _shared_state->multi_cast_data_streamer->set_sink_profile(operator_profile()); - _shared_state->setup_shared_profile(custom_profile()); _shared_state->multi_cast_data_streamer->set_write_dependency(_dependency); return Status::OK(); } diff --git a/be/src/exec/operator/multi_cast_data_streamer.cpp b/be/src/exec/operator/multi_cast_data_streamer.cpp index 4511f42ae3c57d..142f78003384d1 100644 --- a/be/src/exec/operator/multi_cast_data_streamer.cpp +++ b/be/src/exec/operator/multi_cast_data_streamer.cpp @@ -32,7 +32,9 @@ #include "exec/operator/multi_cast_data_stream_source.h" #include "exec/operator/spill_utils.h" #include "exec/pipeline/dependency.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" #include "runtime/exec_env.h" #include "runtime/runtime_state.h" #include "util/pretty_printer.h" @@ -77,14 +79,14 @@ Status MultiCastDataStreamer::pull(RuntimeState* state, int sender_idx, Block* b if (!_spill_readers[sender_idx].empty()) { auto reader_item = _spill_readers[sender_idx].front(); - if (!reader_item->stream->ready_for_reading()) { + if (!reader_item->spill_file->ready_for_reading()) { return Status::OK(); } auto& reader = reader_item->reader; RETURN_IF_ERROR(reader->open()); if (reader_item->block_offset != 0) { - reader->seek(reader_item->block_offset); + RETURN_IF_ERROR(reader->seek(reader_item->block_offset)); reader_item->block_offset = 0; } @@ -117,9 +119,9 @@ Status MultiCastDataStreamer::pull(RuntimeState* state, int sender_idx, Block* b }; l.unlock(); - SpillRecoverRunnable spill_runnable(state, _source_operator_profiles[sender_idx], - catch_exception_func); - return spill_runnable.run(); + // spill is synchronous; the profile passed to the runnable was only + // used for counters that are now tracked externally, so call helper + return run_spill_task(state, catch_exception_func); } auto& pos_to_pull = _sender_pos_to_read[sender_idx]; @@ -179,7 +181,7 @@ Status MultiCastDataStreamer::_trigger_spill_if_need(RuntimeState* state, bool* return Status::OK(); } - SpillStreamSPtr spill_stream; + SpillFileSPtr spill_file; *triggered = false; if (_cumulative_mem_size.load() >= config::exchg_node_buffer_size_bytes && _multi_cast_blocks.size() >= 4) { @@ -205,25 +207,32 @@ Status MultiCastDataStreamer::_trigger_spill_if_need(RuntimeState* state, bool* } if (has_reached_end) { - RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - state, spill_stream, print_id(state->query_id()), "MultiCastSender", _node_id, - std::numeric_limits::max(), std::numeric_limits::max(), - _sink_operator_profile)); + auto relative_path = fmt::format("{}/{}-{}-{}-{}", print_id(state->query_id()), + "MultiCastSender", _node_id, state->task_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + relative_path, spill_file)); + + // Block all senders while spilling. + for (int i = 0; i < _sender_pos_to_read.size(); ++i) { + _block_reading(i); + } + + // Write blocks to disk. _start_spill_task updates spill_file->_part_count. + RETURN_IF_ERROR(_start_spill_task(state, spill_file)); + DCHECK_EQ(_multi_cast_blocks.size(), 0); + + // Create readers AFTER writing so that _part_count is valid. for (int i = 0; i < _sender_pos_to_read.size(); ++i) { if (distances[i] < total_count) { - auto reader = spill_stream->create_separate_reader(); - reader->set_counters(_source_operator_profiles[i]); + auto reader = spill_file->create_reader(state, _source_operator_profiles[i]); auto reader_item = std::make_shared( - std::move(reader), spill_stream, distances[i], false); + std::move(reader), spill_file, distances[i], false); _spill_readers[i].emplace_back(std::move(reader_item)); } - - _block_reading(i); + _set_ready_for_read(i); } - RETURN_IF_ERROR(_start_spill_task(state, spill_stream)); - DCHECK_EQ(_multi_cast_blocks.size(), 0); - for (auto& pos : _sender_pos_to_read) { pos = _multi_cast_blocks.end(); } @@ -235,7 +244,7 @@ Status MultiCastDataStreamer::_trigger_spill_if_need(RuntimeState* state, bool* return Status::OK(); } -Status MultiCastDataStreamer::_start_spill_task(RuntimeState* state, SpillStreamSPtr spill_stream) { +Status MultiCastDataStreamer::_start_spill_task(RuntimeState* state, SpillFileSPtr spill_file) { std::vector blocks; for (auto& block : _multi_cast_blocks) { DCHECK_GT(block._block->rows(), 0); @@ -244,18 +253,20 @@ Status MultiCastDataStreamer::_start_spill_task(RuntimeState* state, SpillStream _multi_cast_blocks.clear(); - auto spill_func = [state, blocks = std::move(blocks), - spill_stream = std::move(spill_stream)]() mutable { + auto* sink_profile = _sink_operator_profile; + auto spill_func = [state, blocks = std::move(blocks), spill_file = std::move(spill_file), + sink_profile]() mutable { const auto blocks_count = blocks.size(); - while (!blocks.empty() && !state->is_cancelled()) { - auto block = std::move(blocks.front()); - blocks.erase(blocks.begin()); - - RETURN_IF_ERROR(spill_stream->spill_block(state, block, false)); + SpillFileWriterSPtr writer; + RETURN_IF_ERROR(spill_file->create_writer(state, sink_profile, writer)); + for (auto& block : blocks) { + if (state->is_cancelled()) break; + RETURN_IF_ERROR(writer->write_block(state, block)); } + RETURN_IF_ERROR(writer->close()); VLOG_DEBUG << "Query: " << print_id(state->query_id()) << " multi cast write " << blocks_count << " blocks"; - return spill_stream->spill_eof(); + return Status::OK(); }; auto exception_catch_func = [spill_func = std::move(spill_func), @@ -266,15 +277,13 @@ Status MultiCastDataStreamer::_start_spill_task(RuntimeState* state, SpillStream if (!status.ok()) { LOG(WARNING) << "Query: " << query_id << " multi cast write failed: " << status.to_string(); - } else { - for (int i = 0; i < _sender_pos_to_read.size(); ++i) { - _set_ready_for_read(i); - } } + // _set_ready_for_read is called by _trigger_spill_if_need after readers + // are created with the correct part_count. return status; }; - return SpillSinkRunnable(state, nullptr, _sink_operator_profile, exception_catch_func).run(); + return run_spill_task(state, exception_catch_func); } Status MultiCastDataStreamer::push(RuntimeState* state, doris::Block* block, bool eos) { diff --git a/be/src/exec/operator/multi_cast_data_streamer.h b/be/src/exec/operator/multi_cast_data_streamer.h index b9dc9112cfba07..773576fb2b932e 100644 --- a/be/src/exec/operator/multi_cast_data_streamer.h +++ b/be/src/exec/operator/multi_cast_data_streamer.h @@ -25,7 +25,7 @@ #include "core/block/block.h" #include "exec/exchange/vdata_stream_sender.h" #include "exec/pipeline/dependency.h" -#include "exec/spill/spill_stream.h" +#include "exec/spill/spill_file.h" #include "runtime/runtime_profile.h" namespace doris { @@ -45,8 +45,9 @@ struct MultiCastBlock { }; struct SpillingReader { - SpillReaderUPtr reader; - SpillStreamSPtr stream; + SpillFileReaderSPtr reader; + SpillFileSPtr spill_file; + int64_t block_offset {0}; bool all_data_read {false}; }; @@ -99,7 +100,7 @@ class MultiCastDataStreamer { Status _copy_block(RuntimeState* state, int32_t sender_idx, Block* block, MultiCastBlock& multi_cast_block); - Status _start_spill_task(RuntimeState* state, SpillStreamSPtr spill_stream); + Status _start_spill_task(RuntimeState* state, SpillFileSPtr spill_file); Status _trigger_spill_if_need(RuntimeState* state, bool* triggered); diff --git a/be/src/exec/operator/operator.h b/be/src/exec/operator/operator.h index f8226eca714dcc..e354aa26a32b70 100644 --- a/be/src/exec/operator/operator.h +++ b/be/src/exec/operator/operator.h @@ -130,10 +130,7 @@ class OperatorBase { virtual size_t revocable_mem_size(RuntimeState* state) const { return 0; } - virtual Status revoke_memory(RuntimeState* state, - const std::shared_ptr& spill_context) { - return Status::OK(); - } + virtual Status revoke_memory(RuntimeState* state) { return Status::OK(); } virtual bool is_hash_join_probe() const { return false; } @@ -375,15 +372,6 @@ class PipelineXSpillLocalState : public PipelineXLocalState { } void init_spill_write_counters() { - _spill_write_timer = ADD_TIMER_WITH_LEVEL(Base::custom_profile(), "SpillWriteTime", 1); - - _spill_write_wait_in_queue_task_count = ADD_COUNTER_WITH_LEVEL( - Base::custom_profile(), "SpillWriteTaskWaitInQueueCount", TUnit::UNIT, 1); - _spill_writing_task_count = ADD_COUNTER_WITH_LEVEL(Base::custom_profile(), - "SpillWriteTaskCount", TUnit::UNIT, 1); - _spill_write_wait_in_queue_timer = - ADD_TIMER_WITH_LEVEL(Base::custom_profile(), "SpillWriteTaskWaitInQueueTime", 1); - _spill_write_file_timer = ADD_TIMER_WITH_LEVEL(Base::custom_profile(), "SpillWriteFileTime", 1); @@ -397,23 +385,12 @@ class PipelineXSpillLocalState : public PipelineXLocalState { Base::custom_profile(), "SpillWriteFileBytes", TUnit::BYTES, 1); _spill_write_rows_count = ADD_COUNTER_WITH_LEVEL(Base::custom_profile(), "SpillWriteRows", TUnit::UNIT, 1); - _spill_file_total_count = ADD_COUNTER_WITH_LEVEL( + _spill_write_file_total_count = ADD_COUNTER_WITH_LEVEL( Base::custom_profile(), "SpillWriteFileTotalCount", TUnit::UNIT, 1); } void init_spill_read_counters() { - _spill_total_timer = ADD_TIMER_WITH_LEVEL(Base::custom_profile(), "SpillTotalTime", 1); - // Spill read counters - _spill_recover_time = ADD_TIMER_WITH_LEVEL(Base::custom_profile(), "SpillRecoverTime", 1); - - _spill_read_wait_in_queue_task_count = ADD_COUNTER_WITH_LEVEL( - Base::custom_profile(), "SpillReadTaskWaitInQueueCount", TUnit::UNIT, 1); - _spill_reading_task_count = ADD_COUNTER_WITH_LEVEL(Base::custom_profile(), - "SpillReadTaskCount", TUnit::UNIT, 1); - _spill_read_wait_in_queue_timer = - ADD_TIMER_WITH_LEVEL(Base::custom_profile(), "SpillReadTaskWaitInQueueTime", 1); - _spill_read_file_time = ADD_TIMER_WITH_LEVEL(Base::custom_profile(), "SpillReadFileTime", 1); _spill_read_deserialize_block_timer = @@ -436,35 +413,7 @@ class PipelineXSpillLocalState : public PipelineXLocalState { Base::custom_profile(), "SpillWriteFileCurrentCount", TUnit::UNIT, 1); } - // These two counters are shared to spill source operators as the initial value - // Initialize values of counters 'SpillWriteFileCurrentBytes' and 'SpillWriteFileCurrentCount' - // from spill sink operators' "SpillWriteFileTotalCount" and "SpillWriteFileBytes" - void copy_shared_spill_profile() { - if (_copy_shared_spill_profile) { - _copy_shared_spill_profile = false; - const auto* spill_shared_state = (const BasicSpillSharedState*)Base::_shared_state; - COUNTER_UPDATE(_spill_file_current_size, - spill_shared_state->_spill_write_file_total_size->value()); - COUNTER_UPDATE(_spill_file_current_count, - spill_shared_state->_spill_file_total_count->value()); - Base::_shared_state->update_spill_stream_profiles(Base::custom_profile()); - } - } - - // Total time of spill, including spill task scheduling time, - // serialize block time, write disk file time, - // and read disk file time, deserialize block time etc. - RuntimeProfile::Counter* _spill_total_timer = nullptr; - // Spill write counters - // Total time of spill write, including serialize block time, write disk file, - // and wait in queue time, etc. - RuntimeProfile::Counter* _spill_write_timer = nullptr; - - RuntimeProfile::Counter* _spill_write_wait_in_queue_task_count = nullptr; - RuntimeProfile::Counter* _spill_writing_task_count = nullptr; - RuntimeProfile::Counter* _spill_write_wait_in_queue_timer = nullptr; - // Total time of writing file RuntimeProfile::Counter* _spill_write_file_timer = nullptr; RuntimeProfile::Counter* _spill_write_serialize_block_timer = nullptr; @@ -476,21 +425,13 @@ class PipelineXSpillLocalState : public PipelineXLocalState { // Total bytes of spill data written to disk file(after serialized) RuntimeProfile::Counter* _spill_write_file_total_size = nullptr; RuntimeProfile::Counter* _spill_write_rows_count = nullptr; - RuntimeProfile::Counter* _spill_file_total_count = nullptr; + RuntimeProfile::Counter* _spill_write_file_total_count = nullptr; RuntimeProfile::Counter* _spill_file_current_count = nullptr; // Spilled file total size RuntimeProfile::Counter* _spill_file_total_size = nullptr; // Current spilled file size RuntimeProfile::Counter* _spill_file_current_size = nullptr; - // Spill read counters - // Total time of recovring spilled data, including read file time, deserialize time, etc. - RuntimeProfile::Counter* _spill_recover_time = nullptr; - - RuntimeProfile::Counter* _spill_read_wait_in_queue_task_count = nullptr; - RuntimeProfile::Counter* _spill_reading_task_count = nullptr; - RuntimeProfile::Counter* _spill_read_wait_in_queue_timer = nullptr; - RuntimeProfile::Counter* _spill_read_file_time = nullptr; RuntimeProfile::Counter* _spill_read_deserialize_block_timer = nullptr; RuntimeProfile::Counter* _spill_read_block_count = nullptr; @@ -500,8 +441,6 @@ class PipelineXSpillLocalState : public PipelineXLocalState { RuntimeProfile::Counter* _spill_read_file_size = nullptr; RuntimeProfile::Counter* _spill_read_rows_count = nullptr; RuntimeProfile::Counter* _spill_read_file_count = nullptr; - - bool _copy_shared_spill_profile = true; }; class DataSinkOperatorXBase; @@ -666,6 +605,8 @@ class DataSinkOperatorXBase : public OperatorBase { [[nodiscard]] virtual Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) = 0; + // Returns the memory this sink operator expects to allocate in the next + // execution round (sink only — pipeline task sums all operators + sink). [[nodiscard]] virtual size_t get_reserve_mem_size(RuntimeState* state, bool eos) { return state->minimum_operator_memory_required_bytes(); } @@ -779,17 +720,10 @@ class PipelineXSpillSinkLocalState : public PipelineXSinkLocalState _rows_in_partitions; - - // Total time of spill, including spill task scheduling time, - // serialize block time, write disk file time, - // and read disk file time, deserialize block time etc. - RuntimeProfile::Counter* _spill_total_timer = nullptr; - // Spill write counters - // Total time of spill write, including serialize block time, write disk file, - // and wait in queue time, etc. - RuntimeProfile::Counter* _spill_write_timer = nullptr; - - RuntimeProfile::Counter* _spill_write_wait_in_queue_task_count = nullptr; - RuntimeProfile::Counter* _spill_writing_task_count = nullptr; - RuntimeProfile::Counter* _spill_write_wait_in_queue_timer = nullptr; - // Total time of writing file + RuntimeProfile::Counter* _spill_write_file_total_size = nullptr; + RuntimeProfile::Counter* _spill_write_file_total_count = nullptr; RuntimeProfile::Counter* _spill_write_file_timer = nullptr; RuntimeProfile::Counter* _spill_write_serialize_block_timer = nullptr; // Original count of spilled Blocks @@ -944,11 +866,13 @@ class OperatorXBase : public OperatorBase { } } - size_t revocable_mem_size(RuntimeState* state) const override { - return (_child and !is_source()) ? _child->revocable_mem_size(state) : 0; - } - - // If this method is not overwrite by child, its default value is 1MB + // Returns the memory this single operator expects to allocate in the next + // execution round. Each operator reports only its OWN requirement — the + // pipeline task is responsible for summing all operators + sink. + // After the value is consumed the caller should invoke + // reset_reserve_mem_size() so the next round starts from zero. + // If this method is not overridden by a subclass, its default value is the + // minimum operator memory (typically 1 MB). [[nodiscard]] virtual size_t get_reserve_mem_size(RuntimeState* state) { return state->minimum_operator_memory_required_bytes(); } @@ -1007,6 +931,9 @@ class OperatorXBase : public OperatorBase { // To keep compatibility with older FE void set_serial_operator() { _is_serial_operator = true; } + // Resets this operator's estimated memory usage to zero so that the next + // call to get_reserve_mem_size() starts fresh. The pipeline task calls + // this after consuming the reserve size for all operators in a round. virtual void reset_reserve_mem_size(RuntimeState* state) {} protected: @@ -1074,27 +1001,21 @@ class OperatorX : public OperatorXBase { return state->get_local_state(operator_id())->template cast(); } + // Returns memory this single operator expects to allocate in the next round. + // Does NOT include child operators — the pipeline task iterates all + // operators itself. size_t get_reserve_mem_size(RuntimeState* state) override { auto& local_state = get_local_state(state); auto estimated_size = local_state.estimate_memory_usage(); if (estimated_size < state->minimum_operator_memory_required_bytes()) { estimated_size = state->minimum_operator_memory_required_bytes(); } - if (!is_source() && _child) { - auto child_reserve_size = _child->get_reserve_mem_size(state); - estimated_size += - std::max(state->minimum_operator_memory_required_bytes(), child_reserve_size); - } return estimated_size; } void reset_reserve_mem_size(RuntimeState* state) override { auto& local_state = get_local_state(state); local_state.reset_estimate_memory_usage(); - - if (!is_source() && _child) { - _child->reset_reserve_mem_size(state); - } } }; @@ -1229,6 +1150,10 @@ class DummyOperator final : public OperatorX { ? 0 : OperatorX::get_reserve_mem_size(state); } + Status revoke_memory(RuntimeState* state) override { + _revoke_called = true; + return Status::OK(); + } private: friend class AssertNumRowsLocalState; @@ -1237,6 +1162,7 @@ class DummyOperator final : public OperatorX { bool _terminated = false; size_t _revocable_mem_size = 0; bool _disable_reserve_mem = false; + bool _revoke_called = false; }; class DummySinkLocalState final : public PipelineXSinkLocalState { @@ -1279,6 +1205,10 @@ class DummySinkOperatorX final : public DataSinkOperatorX { ? 0 : DataSinkOperatorX::get_reserve_mem_size(state, eos); } + Status revoke_memory(RuntimeState* state) override { + _revoke_called = true; + return Status::OK(); + } private: bool _low_memory_mode = false; @@ -1286,6 +1216,7 @@ class DummySinkOperatorX final : public DataSinkOperatorX { std::atomic_bool _return_eof = false; size_t _revocable_mem_size = 0; bool _disable_reserve_mem = false; + bool _revoke_called = false; }; #endif diff --git a/be/src/exec/operator/partitioned_aggregation_sink_operator.cpp b/be/src/exec/operator/partitioned_aggregation_sink_operator.cpp index 004c010e4b37b4..82d29ca4bb84a2 100644 --- a/be/src/exec/operator/partitioned_aggregation_sink_operator.cpp +++ b/be/src/exec/operator/partitioned_aggregation_sink_operator.cpp @@ -20,7 +20,6 @@ #include #include -#include #include #include "common/status.h" @@ -28,8 +27,8 @@ #include "exec/operator/spill_utils.h" #include "exec/pipeline/dependency.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" #include "util/pretty_printer.h" @@ -50,27 +49,24 @@ Status PartitionedAggSinkLocalState::init(doris::RuntimeState* state, _init_counters(); auto& parent = Base::_parent->template cast(); - Base::_shared_state->init_spill_params(parent._spill_partition_count); + _spill_writers.resize(parent._partition_count); + RETURN_IF_ERROR(_setup_in_memory_agg_op(state)); - 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& 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()); + 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()); } - - _rows_in_partitions.assign(Base::_shared_state->partition_count, 0); + _rows_in_partitions.assign(parent._partition_count, 0); return Status::OK(); } Status PartitionedAggSinkLocalState::open(RuntimeState* state) { SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_open_timer); - _shared_state->setup_shared_profile(custom_profile()); return Base::open(state); } @@ -80,7 +76,24 @@ Status PartitionedAggSinkLocalState::close(RuntimeState* state, Status exec_stat if (Base::_closed) { return Status::OK(); } - return Base::close(state, exec_status); + + Status first_error; + for (auto& writer : _spill_writers) { + if (writer) { + auto st = writer->close(); + if (!st.ok() && first_error.ok()) { + first_error = st; + } + writer.reset(); + } + } + _spill_writers.clear(); + + auto st = Base::close(state, exec_status); + if (!first_error.ok()) { + return first_error; + } + return st; } void PartitionedAggSinkLocalState::_init_counters() { @@ -96,7 +109,7 @@ void PartitionedAggSinkLocalState::_init_counters() { update_profile_from_inner_profile(name, custom_profile(), child_profile) template -void PartitionedAggSinkLocalState::update_profile(RuntimeProfile* child_profile) { +void PartitionedAggSinkLocalState::_update_profile(RuntimeProfile* child_profile) { UPDATE_PROFILE("MemoryUsageHashTable"); UPDATE_PROFILE("MemoryUsageSerializeKeyArena"); UPDATE_PROFILE("BuildTime"); @@ -124,7 +137,12 @@ PartitionedAggSinkOperatorX::PartitionedAggSinkOperatorX(ObjectPool* pool, int o Status PartitionedAggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(DataSinkOperatorX::init(tnode, state)); _name = "PARTITIONED_AGGREGATION_SINK_OPERATOR"; - _spill_partition_count = state->spill_aggregation_partition_count(); + _partition_count = state->spill_aggregation_partition_count(); + if (_partition_count < 2 || _partition_count > 32) { + return Status::InvalidArgument(fmt::format( + "Invalid partition count {} for PartitionedAggSinkOperatorX, should be in [2, 32]", + _partition_count)); + } return _agg_sink_operator->init(tnode, state); } @@ -136,65 +154,67 @@ Status PartitionedAggSinkOperatorX::sink(doris::RuntimeState* state, Block* in_b 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()); - local_state._eos = eos; + auto* runtime_state = local_state._runtime_state.get(); DBUG_EXECUTE_IF("fault_inject::partitioned_agg_sink::sink", { return Status::Error("fault_inject partitioned_agg_sink sink failed"); }); RETURN_IF_ERROR(_agg_sink_operator->sink(runtime_state, in_block, false)); - size_t revocable_size = 0; - int64_t query_mem_limit = 0; + // handle spill condition first, independent of eos + if (local_state._shared_state->_is_spilled) { + if (revocable_mem_size(state) >= state->spill_aggregation_sink_mem_limit_bytes()) { + RETURN_IF_ERROR(revoke_memory(state)); + DCHECK(local_state._shared_state->_in_mem_shared_state->aggregate_data_container + ->total_count() == 0); + } + } else { + auto* sink_local_state = local_state._runtime_state->get_sink_local_state(); + local_state._update_profile(sink_local_state->custom_profile()); + } + + // finally perform EOS bookkeeping if (eos) { - revocable_size = revocable_mem_size(state); - query_mem_limit = state->get_query_ctx()->resource_ctx()->memory_context()->mem_limit(); - LOG(INFO) << fmt::format( - "Query:{}, agg sink:{}, task:{}, eos, need spill:{}, query mem limit:{}, " - "revocable memory:{}", - print_id(state->query_id()), node_id(), state->task_id(), - local_state._shared_state->is_spilled, PrettyPrinter::print_bytes(query_mem_limit), - PrettyPrinter::print_bytes(revocable_size)); - - if (local_state._shared_state->is_spilled) { - if (revocable_mem_size(state) > 0) { - RETURN_IF_ERROR(revoke_memory(state, nullptr)); - } else { - for (auto& partition : local_state._shared_state->spill_partitions) { - RETURN_IF_ERROR(partition->finish_current_spilling(eos)); + if (local_state._shared_state->_is_spilled) { + // If there are still memory aggregation data, revoke memory, it is a flush operation. + if (_agg_sink_operator->get_hash_table_size(runtime_state) > 0) { + RETURN_IF_ERROR(revoke_memory(state)); + DCHECK(local_state._shared_state->_in_mem_shared_state->aggregate_data_container + ->total_count() == 0); + } + // Close all writers (finalizes SpillFile metadata) + for (auto& writer : local_state._spill_writers) { + if (writer) { + RETURN_IF_ERROR(writer->close()); } - local_state._dependency->set_ready_to_read(); } - } else { - local_state._dependency->set_ready_to_read(); + local_state._clear_tmp_data(); } - } else if (local_state._shared_state->is_spilled) { - if (revocable_mem_size(state) >= SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { - return revoke_memory(state, nullptr); - } - } - - if (!local_state._shared_state->is_spilled) { - auto* sink_local_state = local_state._runtime_state->get_sink_local_state(); - local_state.update_profile(sink_local_state->custom_profile()); + // Should set here, not at the beginning, because revoke memory will check eos flag. + local_state._eos = eos; + local_state._dependency->set_ready_to_read(); } return Status::OK(); } -Status PartitionedAggSinkOperatorX::revoke_memory( - RuntimeState* state, const std::shared_ptr& spill_context) { +Status PartitionedAggSinkOperatorX::revoke_memory(RuntimeState* state) { auto& local_state = get_local_state(state); - return local_state.revoke_memory(state, spill_context); + return local_state._revoke_memory(state); } size_t PartitionedAggSinkOperatorX::revocable_mem_size(RuntimeState* state) const { auto& local_state = get_local_state(state); + // If the agg sink already has all data, then not able to spill. + if (local_state._eos) { + return 0; + } auto* runtime_state = local_state._runtime_state.get(); auto size = _agg_sink_operator->get_revocable_mem_size(runtime_state); - return size; + return size > state->spill_min_revocable_mem() ? size : 0; } -Status PartitionedAggSinkLocalState::setup_in_memory_agg_op(RuntimeState* state) { +Status PartitionedAggSinkLocalState::_setup_in_memory_agg_op(RuntimeState* state) { _runtime_state = RuntimeState::create_unique( state->fragment_instance_id(), state->query_id(), state->fragment_id(), state->query_options(), TQueryGlobals {}, state->exec_env(), state->get_query_ctx()); @@ -206,16 +226,16 @@ Status PartitionedAggSinkLocalState::setup_in_memory_agg_op(RuntimeState* state) _runtime_state->set_task_id(state->task_id()); auto& parent = Base::_parent->template cast(); - Base::_shared_state->in_mem_shared_state_sptr = + 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; + 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 {.task_idx = 0, .parent_profile = _internal_runtime_profile.get(), .sender_id = -1, - .shared_state = Base::_shared_state->in_mem_shared_state_sptr.get(), + .shared_state = Base::_shared_state->_in_mem_shared_state_sptr.get(), .shared_state_map = {}, .tsink = {}}; RETURN_IF_ERROR(parent._agg_sink_operator->setup_local_state(_runtime_state.get(), info)); @@ -234,71 +254,67 @@ size_t PartitionedAggSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bo } template -Status PartitionedAggSinkLocalState::to_block(HashTableCtxType& context, std::vector& keys, - std::vector& values, - const AggregateDataPtr null_key_data) { +Status PartitionedAggSinkLocalState::_to_block(HashTableCtxType& context, + std::vector& keys, + std::vector& values, + const AggregateDataPtr null_key_data) { SCOPED_TIMER(_spill_serialize_hash_table_timer); - context.insert_keys_into_columns(keys, key_columns_, (uint32_t)keys.size()); + context.insert_keys_into_columns(keys, _key_columns, (uint32_t)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); + 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(); + 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] + 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()); + Base::_shared_state->_in_mem_shared_state->offsets_of_aggregate_states[i], + _value_columns[i], values.size()); } ColumnsWithTypeAndName key_columns_with_schema; - for (int i = 0; i < key_columns_.size(); ++i) { + 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()); + 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; + _key_block = key_columns_with_schema; ColumnsWithTypeAndName value_columns_with_schema; - for (int i = 0; i < value_columns_.size(); ++i) { + 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] + 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; + _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 : _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); + for (const auto& column : _value_block.get_columns_with_type_and_name()) { + _block.insert(column); } return Status::OK(); } template Status PartitionedAggSinkLocalState::_spill_partition( - RuntimeState* state, HashTableCtxType& context, AggSpillPartitionSPtr& spill_partition, + RuntimeState* state, HashTableCtxType& context, size_t partition_idx, std::vector& keys, std::vector& values, const AggregateDataPtr null_key_data, bool is_last) { - SpillStreamSPtr spill_stream; - auto status = spill_partition->get_spill_stream(state, Base::_parent->node_id(), - Base::operator_profile(), spill_stream); - RETURN_IF_ERROR(status); - - status = to_block(context, keys, values, null_key_data); + auto status = _to_block(context, keys, values, null_key_data); RETURN_IF_ERROR(status); if (is_last) { @@ -306,17 +322,33 @@ Status PartitionedAggSinkLocalState::_spill_partition( std::vector tmp_values; keys.swap(tmp_keys); values.swap(tmp_values); - } else { keys.clear(); values.clear(); } - status = spill_stream->spill_block(state, block_, false); - RETURN_IF_ERROR(status); - status = spill_partition->flush_if_full(); + // Ensure _spill_partitions is initialized to correct size + auto& partitions = Base::_shared_state->_spill_partitions; + auto& parent = Base::_parent->template cast(); + if (partitions.size() == 0) { + partitions.resize(parent._partition_count); + } + + // Lazy-create SpillFile + writer on first write for this partition + auto& spill_file = partitions[partition_idx]; + auto& writer = _spill_writers[partition_idx]; + if (!writer) { + auto relative_path = fmt::format("{}/agg_{}-{}-{}-{}", print_id(state->query_id()), + partition_idx, parent.node_id(), state->task_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + spill_file)); + RETURN_IF_ERROR(spill_file->create_writer(state, Base::operator_profile(), writer)); + } + + RETURN_IF_ERROR(writer->write_block(state, _block)); _reset_tmp_data(); - return status; + return Status::OK(); } template @@ -325,149 +357,93 @@ Status PartitionedAggSinkLocalState::_spill_hash_table(RuntimeState* state, HashTableType& hash_table, const size_t size_to_revoke, bool eos) { Status status; - Defer defer {[&]() { - if (!status.ok()) { - Base::_shared_state->close(); - } - }}; context.init_iterator(); + auto& parent = _parent->template cast(); - Base::_shared_state->in_mem_shared_state->aggregate_data_container->init_once(); + Base::_shared_state->_in_mem_shared_state->aggregate_data_container->init_once(); - const auto total_rows = - Base::_shared_state->in_mem_shared_state->aggregate_data_container->total_count(); + const auto total_rows = parent._agg_sink_operator->get_hash_table_size(_runtime_state.get()); + + if (total_rows == 0) { + return Status::OK(); + } const size_t size_to_revoke_ = std::max(size_to_revoke, 1); // `spill_batch_rows` will be between 4k and 1M // and each block to spill will not be larger than 32MB(`MAX_SPILL_WRITE_BATCH_MEM`) + // TODO: yiguolei, should review this logic const auto spill_batch_rows = std::min( - 1024 * 1024, std::max(4096, SpillStream::MAX_SPILL_WRITE_BATCH_MEM * - total_rows / size_to_revoke_)); + + 1024 * 1024, std::max(4096, SpillFile::MAX_SPILL_WRITE_BATCH_MEM * total_rows / + size_to_revoke_)); VLOG_DEBUG << "Query: " << print_id(state->query_id()) << ", node: " << _parent->node_id() - << ", spill_batch_rows: " << spill_batch_rows << ", total rows: " << total_rows; + << ", spill_batch_rows: " << spill_batch_rows << ", total rows: " << total_rows + << ", size_to_revoke: " << size_to_revoke; size_t 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() && + parent._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() && !state->is_cancelled()) { const auto& key = iter.template get_key(); - auto partition_index = Base::_shared_state->get_partition_index(hash_table.hash(key)); + auto partition_index = hash_table.hash(key) % parent._partition_count; 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) { + for (int i = 0; i < parent._partition_count && !state->is_cancelled(); ++i) { if (spill_infos[i].keys_.size() >= spill_batch_rows) { _rows_in_partitions[i] += spill_infos[i].keys_.size(); - status = _spill_partition( - state, context, Base::_shared_state->spill_partitions[i], - spill_infos[i].keys_, spill_infos[i].values_, nullptr, false); + status = _spill_partition(state, context, i, spill_infos[i].keys_, + spill_infos[i].values_, nullptr, false); RETURN_IF_ERROR(status); + spill_infos[i].keys_.clear(); + spill_infos[i].values_.clear(); } } } ++iter; } - auto hash_null_key_data = hash_table.has_null_key_data(); - for (int i = 0; i < Base::_shared_state->partition_count && !state->is_cancelled(); ++i) { - auto spill_null_key_data = - (hash_null_key_data && i == Base::_shared_state->partition_count - 1); + const auto has_null_key_data = hash_table.has_null_key_data(); + for (int i = 0; i < parent._partition_count && !state->is_cancelled(); ++i) { + auto spill_null_key_data = (has_null_key_data && i == parent._partition_count - 1); if (spill_infos[i].keys_.size() > 0 || spill_null_key_data) { _rows_in_partitions[i] += spill_infos[i].keys_.size(); status = _spill_partition( - state, context, Base::_shared_state->spill_partitions[i], spill_infos[i].keys_, - spill_infos[i].values_, + state, context, i, spill_infos[i].keys_, spill_infos[i].values_, spill_null_key_data ? hash_table.template get_null_key_data() : 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(); } -Status PartitionedAggSinkLocalState::_execute_spill_process(RuntimeState* state, - size_t size_to_revoke) { - Status status; - auto& parent = Base::_parent->template cast(); - auto query_id = state->query_id(); - - DBUG_EXECUTE_IF("fault_inject::partitioned_agg_sink::revoke_memory_cancel", { - status = Status::InternalError("fault_inject partitioned_agg_sink revoke_memory canceled"); - state->get_query_ctx()->cancel(status); - return status; - }); - - Defer defer {[&]() { - if (!status.ok() || state->is_cancelled()) { - if (!status.ok()) { - LOG(WARNING) << fmt::format( - "Query:{}, agg sink:{}, task:{}, revoke_memory error:{}", - print_id(query_id), Base::_parent->node_id(), state->task_id(), status); - } - _shared_state->close(); - } else { - LOG(INFO) << fmt::format( - "Query:{}, agg sink:{}, task:{}, revoke_memory finish, eos:{}, revocable " - "memory:{}", - print_id(state->query_id()), _parent->node_id(), state->task_id(), _eos, - PrettyPrinter::print_bytes(_parent->revocable_mem_size(state))); - } - - if (_eos) { - Base::_dependency->set_ready_to_read(); - } - state->get_query_ctx()->resource_ctx()->task_controller()->decrease_revoking_tasks_count(); - }}; - - auto* runtime_state = _runtime_state.get(); - auto* agg_data = parent._agg_sink_operator->get_agg_data(runtime_state); - status = std::visit(Overload {[&](std::monostate& arg) -> Status { - return Status::InternalError("Unit hash table"); - }, - [&](auto& agg_method) -> Status { - auto& hash_table = *agg_method.hash_table; - RETURN_IF_CATCH_EXCEPTION(return _spill_hash_table( - state, agg_method, hash_table, size_to_revoke, _eos)); - }}, - agg_data->method_variant); - RETURN_IF_ERROR(status); - status = parent._agg_sink_operator->reset_hash_table(runtime_state); - return status; -} +Status PartitionedAggSinkLocalState::_revoke_memory(RuntimeState* state) { + if (_eos) { + return Status::OK(); + } -Status PartitionedAggSinkLocalState::revoke_memory( - RuntimeState* state, const std::shared_ptr& spill_context) { const auto size_to_revoke = _parent->revocable_mem_size(state); - LOG(INFO) << fmt::format( - "Query:{}, agg sink:{}, task:{}, revoke_memory, eos:{}, need spill:{}, revocable " + VLOG_DEBUG << fmt::format( + "Query:{}, agg sink:{}, task:{}, revoke_memory, eos:{}, is_spilled:{}, revocable " "memory:{}", print_id(state->query_id()), _parent->node_id(), state->task_id(), _eos, - _shared_state->is_spilled, - PrettyPrinter::print_bytes(_parent->revocable_mem_size(state))); + _shared_state->_is_spilled, PrettyPrinter::print_bytes(size_to_revoke)); auto* sink_local_state = _runtime_state->get_sink_local_state(); - if (!_shared_state->is_spilled) { - _shared_state->is_spilled = true; + if (!_shared_state->_is_spilled) { + _shared_state->_is_spilled = true; custom_profile()->add_info_string("Spilled", "true"); - update_profile(sink_local_state->custom_profile()); + _update_profile(sink_local_state->custom_profile()); } else { - update_profile(sink_local_state->custom_profile()); + _update_profile(sink_local_state->custom_profile()); } DBUG_EXECUTE_IF("fault_inject::partitioned_agg_sink::revoke_memory_submit_func", { @@ -477,52 +453,98 @@ Status PartitionedAggSinkLocalState::revoke_memory( state->get_query_ctx()->resource_ctx()->task_controller()->increase_revoking_tasks_count(); - SpillSinkRunnable spill_runnable(state, spill_context, operator_profile(), - [this, state, size_to_revoke] { - return _execute_spill_process(state, size_to_revoke); - }); + auto& parent = Base::_parent->template cast(); + auto query_id = state->query_id(); + + auto spill_func = [this, state, &parent, query_id, size_to_revoke]() -> Status { + Status status; + + DBUG_EXECUTE_IF("fault_inject::partitioned_agg_sink::revoke_memory_cancel", { + status = Status::InternalError( + "fault_inject partitioned_agg_sink revoke_memory canceled"); + state->get_query_ctx()->cancel(status); + return status; + }); + + Defer defer {[&]() { + if (!status.ok()) { + LOG(WARNING) << fmt::format( + "Query:{}, agg sink:{}, task:{}, revoke_memory error:{}", + print_id(query_id), Base::_parent->node_id(), state->task_id(), status); + } else { + VLOG_DEBUG << fmt::format( + "Query:{}, agg sink:{}, task:{}, revoke_memory finish, eos:{}, " + "revocable " + "memory:{}", + print_id(state->query_id()), _parent->node_id(), state->task_id(), _eos, + PrettyPrinter::print_bytes(_parent->revocable_mem_size(state))); + } + state->get_query_ctx() + ->resource_ctx() + ->task_controller() + ->decrease_revoking_tasks_count(); + }}; + + auto* runtime_state = _runtime_state.get(); + auto* agg_data = parent._agg_sink_operator->get_agg_data(runtime_state); + status = std::visit( + Overload {[&](std::monostate& arg) -> Status { + return Status::InternalError("Unit hash table"); + }, + [&](auto& agg_method) -> Status { + auto& hash_table = *agg_method.hash_table; + RETURN_IF_CATCH_EXCEPTION(return _spill_hash_table( + state, agg_method, hash_table, size_to_revoke, _eos)); + }}, + agg_data->method_variant); + RETURN_IF_ERROR(status); + status = parent._agg_sink_operator->reset_hash_table(runtime_state); + return status; + }; - return spill_runnable.run(); + // old code used SpillSinkRunnable, but spills are synchronous and counters + // are tracked externally. Call the spill function directly. + return run_spill_task(state, std::move(spill_func)); } void PartitionedAggSinkLocalState::_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(); + _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 PartitionedAggSinkLocalState::_clear_tmp_data() { { Block empty_block; - block_.swap(empty_block); + _block.swap(empty_block); } { Block empty_block; - key_block_.swap(empty_block); + _key_block.swap(empty_block); } { Block empty_block; - value_block_.swap(empty_block); + _value_block.swap(empty_block); } { MutableColumns cols; - key_columns_.swap(cols); + _key_columns.swap(cols); } { MutableColumns cols; - value_columns_.swap(cols); + _value_columns.swap(cols); } DataTypes tmp_value_data_types; - value_data_types_.swap(tmp_value_data_types); + _value_data_types.swap(tmp_value_data_types); } bool PartitionedAggSinkLocalState::is_blockable() const { - return _shared_state->is_spilled; + return _shared_state->_is_spilled; } #include "common/compile_check_end.h" diff --git a/be/src/exec/operator/partitioned_aggregation_sink_operator.h b/be/src/exec/operator/partitioned_aggregation_sink_operator.h index 2130ec9471de9e..0ca4817eb9f906 100644 --- a/be/src/exec/operator/partitioned_aggregation_sink_operator.h +++ b/be/src/exec/operator/partitioned_aggregation_sink_operator.h @@ -22,11 +22,11 @@ #include "exec/operator/aggregation_sink_operator.h" #include "exec/operator/operator.h" #include "exec/pipeline/dependency.h" -#include "exec/spill/spill_stream.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_writer.h" #include "exprs/vectorized_agg_fn.h" #include "exprs/vexpr.h" -#include "util/pretty_printer.h" namespace doris { #include "common/compile_check_begin.h" @@ -41,22 +41,21 @@ class PartitionedAggSinkLocalState 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; - Status revoke_memory(RuntimeState* state, const std::shared_ptr& spill_context); + bool is_blockable() const override; - Status _execute_spill_process(RuntimeState* state, size_t size_to_revoke); +private: + friend class PartitionedAggSinkOperatorX; - Status setup_in_memory_agg_op(RuntimeState* state); + Status _revoke_memory(RuntimeState* state); - template - void update_profile(RuntimeProfile* child_profile); + Status _setup_in_memory_agg_op(RuntimeState* state); - bool is_blockable() const override; + template + void _update_profile(RuntimeProfile* child_profile); template struct TmpSpillInfo { @@ -69,14 +68,14 @@ class PartitionedAggSinkLocalState HashTableType& hash_table, const size_t size_to_revoke, bool eos); template - Status _spill_partition(RuntimeState* state, HashTableCtxType& context, - AggSpillPartitionSPtr& spill_partition, std::vector& keys, - std::vector& values, + + Status _spill_partition(RuntimeState* state, HashTableCtxType& context, size_t partition_idx, + std::vector& keys, std::vector& values, const AggregateDataPtr null_key_data, bool is_last); template - Status to_block(HashTableCtxType& context, std::vector& keys, - std::vector& values, const AggregateDataPtr null_key_data); + Status _to_block(HashTableCtxType& context, std::vector& keys, + std::vector& values, const AggregateDataPtr null_key_data); void _reset_tmp_data(); void _clear_tmp_data(); @@ -85,18 +84,20 @@ class PartitionedAggSinkLocalState std::unique_ptr _runtime_state; // temp structures during spilling - MutableColumns key_columns_; - MutableColumns value_columns_; - DataTypes value_data_types_; - Block block_; - Block key_block_; - Block value_block_; + MutableColumns _key_columns; + MutableColumns _value_columns; + DataTypes _value_data_types; + Block _block; + Block _key_block; + Block _value_block; std::unique_ptr _internal_runtime_profile; RuntimeProfile::Counter* _memory_usage_reserved = nullptr; RuntimeProfile::Counter* _spill_serialize_hash_table_timer = nullptr; + std::vector _spill_writers; + std::atomic _eos = false; }; @@ -139,16 +140,15 @@ class PartitionedAggSinkOperatorX : public DataSinkOperatorX& spill_context) override; + Status revoke_memory(RuntimeState* state) override; size_t get_reserve_mem_size(RuntimeState* state, bool eos) override; private: friend class PartitionedAggSinkLocalState; std::unique_ptr _agg_sink_operator; - - size_t _spill_partition_count = 32; + // each operator tracks its own partition count for spilling + size_t _partition_count = 32; }; #include "common/compile_check_end.h" } // namespace doris \ No newline at end of file diff --git a/be/src/exec/operator/partitioned_aggregation_source_operator.cpp b/be/src/exec/operator/partitioned_aggregation_source_operator.cpp index 0d27dacefb9249..bfefcfb9af2051 100644 --- a/be/src/exec/operator/partitioned_aggregation_source_operator.cpp +++ b/be/src/exec/operator/partitioned_aggregation_source_operator.cpp @@ -19,6 +19,7 @@ #include +#include #include #include "common/exception.h" @@ -28,12 +29,14 @@ #include "exec/operator/operator.h" #include "exec/operator/spill_utils.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_repartitioner.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" namespace doris { + #include "common/compile_check_begin.h" PartitionedAggLocalState::PartitionedAggLocalState(RuntimeState* state, OperatorXBase* parent) @@ -44,6 +47,14 @@ Status PartitionedAggLocalState::init(RuntimeState* state, LocalStateInfo& info) SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); _internal_runtime_profile = std::make_unique("internal_profile"); + // Counters for partition spill metrics + _max_partition_level = ADD_COUNTER(custom_profile(), "SpillMaxPartitionLevel", TUnit::UNIT); + _total_partition_spills = ADD_COUNTER(custom_profile(), "SpillTotalPartitions", TUnit::UNIT); + + init_spill_write_counters(); + + // Nothing else to init for repartitioner here; fanout is configured when + // repartitioner is initialized with key columns during actual repartition. return Status::OK(); } @@ -54,7 +65,8 @@ Status PartitionedAggLocalState::open(RuntimeState* state) { return Status::OK(); } _opened = true; - RETURN_IF_ERROR(setup_in_memory_agg_op(state)); + RETURN_IF_ERROR(_setup_in_memory_agg_op(state)); + return Status::OK(); } @@ -62,7 +74,7 @@ Status PartitionedAggLocalState::open(RuntimeState* state) { update_profile_from_inner_profile(name, custom_profile(), child_profile) template -void PartitionedAggLocalState::update_profile(RuntimeProfile* child_profile) { +void PartitionedAggLocalState::_update_profile(RuntimeProfile* child_profile) { UPDATE_COUNTER_FROM_INNER("GetResultsTime"); UPDATE_COUNTER_FROM_INNER("HashTableIterateTime"); UPDATE_COUNTER_FROM_INNER("InsertKeysToColumnTime"); @@ -86,7 +98,33 @@ Status PartitionedAggLocalState::close(RuntimeState* state) { if (_closed) { return Status::OK(); } - return Base::close(state); + + Status first_error; + if (_current_reader) { + auto st = _current_reader->close(); + if (!st.ok() && first_error.ok()) { + first_error = st; + } + _current_reader.reset(); + } + + // Clean up partition queue resources. + for (auto& partition : _partition_queue) { + if (partition.spill_file) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(partition.spill_file); + } + } + _partition_queue.clear(); + if (_current_partition.spill_file) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(_current_partition.spill_file); + } + _current_partition.spill_file.reset(); + + auto st = Base::close(state); + if (!first_error.ok()) { + return first_error; + } + return st; } PartitionedAggSourceOperatorX::PartitionedAggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, @@ -99,6 +137,11 @@ PartitionedAggSourceOperatorX::PartitionedAggSourceOperatorX(ObjectPool* pool, Status PartitionedAggSourceOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(OperatorXBase::init(tnode, state)); _op_name = "PARTITIONED_AGGREGATION_OPERATOR"; + // copy partition count from session variable so source knows how many + // spill partitions to expect (used by local states during spill). + _partition_count = state->spill_aggregation_partition_count(); + // default repartition max depth; can be overridden from session variable + _repartition_max_depth = state->spill_repartition_max_depth(); return _agg_source_operator->init(tnode, state); } @@ -109,6 +152,14 @@ Status PartitionedAggSourceOperatorX::prepare(RuntimeState* state) { Status PartitionedAggSourceOperatorX::close(RuntimeState* state) { RETURN_IF_ERROR(OperatorXBase::close(state)); + + // Centralize shared_state cleanup here so resources are released when + // the pipeline task finishes, matching the Sort operator pattern. + auto& local_state = get_local_state(state); + if (local_state._shared_state) { + local_state._shared_state->close(); + } + return _agg_source_operator->close(state); } @@ -135,79 +186,218 @@ bool PartitionedAggSourceOperatorX::is_shuffled_operator() const { return _agg_source_operator->is_shuffled_operator(); } +size_t PartitionedAggSourceOperatorX::revocable_mem_size(RuntimeState* state) const { + auto& local_state = get_local_state(state); + if (!local_state._shared_state->_is_spilled || !local_state._current_partition.spill_file) { + return 0; + } + + size_t bytes = 0; + for (const auto& block : local_state._blocks) { + bytes += block.allocated_bytes(); + } + if (local_state._shared_state->_in_mem_shared_state != nullptr && + local_state._shared_state->_in_mem_shared_state->agg_data != nullptr) { + auto* agg_data = local_state._shared_state->_in_mem_shared_state->agg_data.get(); + bytes += std::visit(Overload {[&](std::monostate& arg) -> size_t { return 0; }, + [&](auto& agg_method) -> size_t { + return agg_method.hash_table->get_buffer_size_in_bytes(); + }}, + agg_data->method_variant); + + if (auto& aggregate_data_container = + local_state._shared_state->_in_mem_shared_state->aggregate_data_container; + aggregate_data_container) { + bytes += aggregate_data_container->memory_usage(); + } + } + return bytes > state->spill_min_revocable_mem() ? bytes : 0; +} + +Status PartitionedAggSourceOperatorX::revoke_memory(RuntimeState* state) { + auto& local_state = get_local_state(state); + if (!local_state._shared_state->_is_spilled) { + return Status::OK(); + } + VLOG_DEBUG << fmt::format("Query:{}, agg source:{}, task:{}, revoke_memory, hash table size:{}", + print_id(state->query_id()), node_id(), state->task_id(), + PrettyPrinter::print_bytes(local_state._estimate_memory_usage)); + + // Flush hash table + repartition remaining spill files of the current partition. + RETURN_IF_ERROR(local_state._flush_and_repartition(state)); + local_state._current_partition = AggSpillPartitionInfo {}; + local_state._need_to_setup_partition = true; + return Status::OK(); +} + Status PartitionedAggSourceOperatorX::get_block(RuntimeState* state, Block* block, bool* eos) { auto& local_state = get_local_state(state); - local_state.copy_shared_spill_profile(); Status status; - Defer defer {[&]() { - if (!status.ok() || *eos) { - local_state._shared_state->close(); - } - }}; SCOPED_TIMER(local_state.exec_time_counter()); - if (local_state._shared_state->is_spilled && - local_state._need_to_merge_data_for_current_partition) { - if (local_state._blocks.empty() && !local_state._current_partition_eos) { - bool has_recovering_data = false; - status = local_state.recover_blocks_from_disk(state, has_recovering_data); - RETURN_IF_ERROR(status); - *eos = !has_recovering_data; + // ── Fast path: not spilled ───────────────────────────────────────── + if (!local_state._shared_state->_is_spilled) { + auto* runtime_state = local_state._runtime_state.get(); + local_state._shared_state->_in_mem_shared_state->aggregate_data_container->init_once(); + status = _agg_source_operator->get_block(runtime_state, block, eos); + RETURN_IF_ERROR(status); + if (*eos) { + auto* source_local_state = + runtime_state->get_local_state(_agg_source_operator->operator_id()); + local_state._update_profile(source_local_state->custom_profile()); + } + local_state.reached_limit(block, eos); + return Status::OK(); + } + + // ── Spilled path ─────────────────────────────────────────────────── + // One-time: move original spill_partitions from shared state into unified queue. + if (local_state._partition_queue.empty() && local_state._need_to_setup_partition && + !local_state._shared_state->_spill_partitions.empty()) { + local_state._init_partition_queue(); + } + + // Phase 1: Pop next partition from queue if needed. + if (local_state._need_to_setup_partition) { + if (local_state._partition_queue.empty()) { + *eos = true; return Status::OK(); - } else if (!local_state._blocks.empty()) { - size_t merged_rows = 0; - while (!local_state._blocks.empty()) { - auto block_ = std::move(local_state._blocks.front()); - merged_rows += block_.rows(); - local_state._blocks.erase(local_state._blocks.begin()); - status = _agg_source_operator->merge_with_serialized_key_helper( - local_state._runtime_state.get(), &block_); - RETURN_IF_ERROR(status); - } - local_state._estimate_memory_usage += - _agg_source_operator->get_estimated_memory_size_for_merging( - local_state._runtime_state.get(), merged_rows); + } - if (!local_state._current_partition_eos) { - return Status::OK(); + local_state._current_partition = std::move(local_state._partition_queue.front()); + local_state._partition_queue.pop_front(); + local_state._blocks.clear(); + local_state._estimate_memory_usage = 0; + + VLOG_DEBUG << fmt::format( + "Query:{}, agg source:{}, task:{}, setup partition level:{}, " + "queue remaining:{}", + print_id(state->query_id()), node_id(), state->task_id(), + local_state._current_partition.level, local_state._partition_queue.size()); + local_state._need_to_setup_partition = false; + } + + // Phase 2: Recover blocks from disk into _blocks (batch of ~8MB). + if (local_state._blocks.empty() && local_state._current_partition.spill_file) { + RETURN_IF_ERROR( + local_state._recover_blocks_from_partition(state, local_state._current_partition)); + // Return empty block to yield to pipeline scheduler. + // Pipeline task will check memory and call revoke_memory if needed. + *eos = false; + return Status::OK(); + } + + auto* memory_sufficient_dependency = state->get_query_ctx()->get_memory_sufficient_dependency(); + // Phase 3: Merge recovered blocks into hash table. + if (!local_state._blocks.empty()) { + size_t merged_rows = 0; + while (!local_state._blocks.empty()) { + auto blk = std::move(local_state._blocks.front()); + merged_rows += blk.rows(); + local_state._blocks.erase(local_state._blocks.begin()); + status = _agg_source_operator->merge_with_serialized_key_helper( + local_state._runtime_state.get(), &blk); + RETURN_IF_ERROR(status); + + if (memory_sufficient_dependency && !memory_sufficient_dependency->ready()) { + break; } } - local_state._need_to_merge_data_for_current_partition = false; + local_state._estimate_memory_usage += + _agg_source_operator->get_estimated_memory_size_for_merging( + local_state._runtime_state.get(), merged_rows); + + // Return empty block to yield — pipeline task will check memory pressure + // and call revoke_memory() if the hash table grew too large. + *eos = false; + return Status::OK(); } - // not spilled in sink or current partition still has data + // Phase 4: All spill files consumed and merged — output aggregated results from hash table. auto* runtime_state = local_state._runtime_state.get(); - local_state._shared_state->in_mem_shared_state->aggregate_data_container->init_once(); - status = _agg_source_operator->get_block(runtime_state, block, eos); - if (!local_state._shared_state->is_spilled) { + local_state._shared_state->_in_mem_shared_state->aggregate_data_container->init_once(); + bool inner_eos = false; + RETURN_IF_ERROR(_agg_source_operator->get_block(runtime_state, block, &inner_eos)); + + if (inner_eos) { auto* source_local_state = - local_state._runtime_state->get_local_state(_agg_source_operator->operator_id()); - local_state.update_profile(source_local_state->custom_profile()); + runtime_state->get_local_state(_agg_source_operator->operator_id()); + local_state._update_profile(source_local_state->custom_profile()); + + // Current partition fully output. Reset hash table, pop next partition. + RETURN_IF_ERROR(_agg_source_operator->reset_hash_table(runtime_state)); + + local_state._current_partition = AggSpillPartitionInfo {}; + local_state._estimate_memory_usage = 0; + local_state._need_to_setup_partition = true; + + if (local_state._partition_queue.empty()) { + *eos = true; + } } - RETURN_IF_ERROR(status); - if (*eos) { - if (local_state._shared_state->is_spilled) { - auto* source_local_state = local_state._runtime_state->get_local_state( - _agg_source_operator->operator_id()); - local_state.update_profile(source_local_state->custom_profile()); - - if (!local_state._shared_state->spill_partitions.empty()) { - local_state._current_partition_eos = false; - local_state._need_to_merge_data_for_current_partition = true; - status = _agg_source_operator->reset_hash_table(runtime_state); - RETURN_IF_ERROR(status); - *eos = false; + local_state.reached_limit(block, eos); + return Status::OK(); +} + +// ════════════════════════════════════════════════════════════════════════ +// PartitionedAggLocalState implementation +// ════════════════════════════════════════════════════════════════════════ + +void PartitionedAggLocalState::_init_partition_queue() { + for (auto& spill_file : _shared_state->_spill_partitions) { + _partition_queue.emplace_back(std::move(spill_file), /*level=*/0); + // Track metrics: each queued partition counts as one spill at level 0 + COUNTER_UPDATE(_total_partition_spills, 1); + _max_partition_level_seen = 0; + COUNTER_SET(_max_partition_level, int64_t(_max_partition_level_seen)); + } + _shared_state->_spill_partitions.clear(); +} + +Status PartitionedAggLocalState::_recover_blocks_from_partition(RuntimeState* state, + AggSpillPartitionInfo& partition) { + size_t accumulated_bytes = 0; + if (!partition.spill_file || state->is_cancelled()) { + return Status::OK(); + } + + // Create or reuse a persistent reader for this file + if (!_current_reader) { + _current_reader = partition.spill_file->create_reader(state, operator_profile()); + RETURN_IF_ERROR(_current_reader->open()); + } + + bool eos = false; + + while (!eos && !state->is_cancelled()) { + Block block; + DBUG_EXECUTE_IF("fault_inject::partitioned_agg_source::recover_spill_data", { + return Status::Error( + "fault_inject partitioned_agg_source recover_spill_data failed"); + }); + RETURN_IF_ERROR(_current_reader->read(&block, &eos)); + + if (!block.empty()) { + accumulated_bytes += block.allocated_bytes(); + _blocks.emplace_back(std::move(block)); + + if (accumulated_bytes >= state->spill_buffer_size_bytes()) { + return Status::OK(); } } } - local_state.reached_limit(block, eos); + + if (eos) { + _current_reader.reset(); + partition.spill_file.reset(); + } return Status::OK(); } -Status PartitionedAggLocalState::setup_in_memory_agg_op(RuntimeState* state) { +Status PartitionedAggLocalState::_setup_in_memory_agg_op(RuntimeState* state) { _runtime_state = RuntimeState::create_unique( state->fragment_instance_id(), state->query_id(), state->fragment_id(), state->query_options(), TQueryGlobals {}, state->exec_env(), state->get_query_ctx()); @@ -220,10 +410,10 @@ Status PartitionedAggLocalState::setup_in_memory_agg_op(RuntimeState* state) { auto& parent = Base::_parent->template cast(); - DCHECK(Base::_shared_state->in_mem_shared_state); + DCHECK(Base::_shared_state->_in_mem_shared_state); LocalStateInfo state_info {.parent_profile = _internal_runtime_profile.get(), .scan_ranges = {}, - .shared_state = Base::_shared_state->in_mem_shared_state, + .shared_state = Base::_shared_state->_in_mem_shared_state, .shared_state_map = {}, .task_idx = 0}; @@ -236,115 +426,129 @@ Status PartitionedAggLocalState::setup_in_memory_agg_op(RuntimeState* state) { return source_local_state->open(state); } -Status PartitionedAggLocalState::_recover_spill_data_from_disk(RuntimeState* state, - const UniqueId& query_id) { - Status status; - Defer defer {[&]() { - if (!status.ok() || state->is_cancelled()) { - if (!status.ok()) { - LOG(WARNING) << fmt::format( - "Query:{}, agg probe:{}, task:{}, recover agg data error:{}", - print_id(query_id), _parent->node_id(), state->task_id(), status); - } - _shared_state->close(); +Status PartitionedAggLocalState::_flush_hash_table_to_sub_spill_files(RuntimeState* state) { + auto* runtime_state = _runtime_state.get(); + auto& p = _parent->cast(); + auto* in_mem_state = _shared_state->_in_mem_shared_state; + + // setup_output must have been called by the caller (_flush_and_repartition) + // before calling this function. The repartitioner writes to the persistent output writers. + + in_mem_state->aggregate_data_container->init_once(); + bool inner_eos = false; + while (!inner_eos && !state->is_cancelled()) { + Block block; + RETURN_IF_ERROR( + p._agg_source_operator->get_serialized_block(runtime_state, &block, &inner_eos)); + if (!block.empty()) { + RETURN_IF_ERROR(_repartitioner.route_block(state, block)); } - }}; - bool has_agg_data = false; - size_t accumulated_blocks_size = 0; - while (!state->is_cancelled() && !has_agg_data && !_shared_state->spill_partitions.empty()) { - while (!_shared_state->spill_partitions[0]->spill_streams_.empty() && - !state->is_cancelled() && !has_agg_data) { - auto& stream = _shared_state->spill_partitions[0]->spill_streams_[0]; - stream->set_read_counters(operator_profile()); - Block block; - bool eos = false; - while (!eos && !state->is_cancelled()) { - { - DBUG_EXECUTE_IF("fault_inject::partitioned_agg_source::recover_spill_data", { - status = Status::Error( - "fault_inject partitioned_agg_source " - "recover_spill_data failed"); - }); - if (status.ok()) { - status = stream->read_next_block_sync(&block, &eos); - } - } - RETURN_IF_ERROR(status); - - if (!block.empty()) { - has_agg_data = true; - accumulated_blocks_size += block.allocated_bytes(); - _blocks.emplace_back(std::move(block)); - - if (accumulated_blocks_size >= SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { - break; - } - } - } + } - _current_partition_eos = eos; + RETURN_IF_ERROR(p._agg_source_operator->reset_hash_table(runtime_state)); + return Status::OK(); +} - if (_current_partition_eos) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); - _shared_state->spill_partitions[0]->spill_streams_.pop_front(); - } - } +Status PartitionedAggLocalState::_flush_and_repartition(RuntimeState* state) { + auto& p = _parent->cast(); + const int new_level = _current_partition.level + 1; - if (_shared_state->spill_partitions[0]->spill_streams_.empty()) { - _shared_state->spill_partitions.pop_front(); - } + if (new_level >= p._repartition_max_depth) { + return Status::InternalError( + "query:{}, node:{}, Agg spill repartition exceeded max depth {} during " + "_flush_and_repartition. Likely due to extreme data skew.", + print_id(state->query_id()), p.node_id(), p._repartition_max_depth); } VLOG_DEBUG << fmt::format( - "Query:{}, agg probe:{}, task:{}, recover partitioned finished, partitions " - "left:{}, bytes read:{}", - print_id(query_id), _parent->node_id(), state->task_id(), - _shared_state->spill_partitions.size(), accumulated_blocks_size); - return status; -} + "Query:{}, agg source:{}, task:{}, _flush_and_repartition: " + "flushing hash table and repartitioning remaining spill file at level {} -> {}", + print_id(state->query_id()), p.node_id(), state->task_id(), _current_partition.level, + new_level); -Status PartitionedAggLocalState::recover_blocks_from_disk(RuntimeState* state, bool& has_data) { - const auto query_id = state->query_id(); + { + auto* source_local_state = + _runtime_state->get_local_state(p._agg_source_operator->operator_id()); + _update_profile(source_local_state->custom_profile()); + } - if (_shared_state->spill_partitions.empty()) { - _shared_state->close(); - has_data = false; - return Status::OK(); + // 1. Create FANOUT output sub-spill-files. + std::vector output_spill_files; + RETURN_IF_ERROR(SpillRepartitioner::create_output_spill_files( + state, p.node_id(), fmt::format("agg_repart_l{}", new_level), + static_cast(p._partition_count), output_spill_files)); + + auto* in_mem_state = _shared_state->_in_mem_shared_state; + size_t num_keys = in_mem_state->probe_expr_ctxs.size(); + std::vector key_column_indices(num_keys); + std::vector key_data_types(num_keys); + for (size_t i = 0; i < num_keys; ++i) { + key_column_indices[i] = i; + key_data_types[i] = in_mem_state->probe_expr_ctxs[i]->root()->data_type(); } - has_data = true; - auto exception_catch_func = [this, state, query_id]() { - DBUG_EXECUTE_IF("fault_inject::partitioned_agg_source::merge_spill_data_cancel", { - auto st = Status::InternalError( - "fault_inject partitioned_agg_source " - "merge spill data canceled"); - state->get_query_ctx()->cancel(st); - return st; - }); + _repartitioner.init_with_key_columns(std::move(key_column_indices), std::move(key_data_types), + operator_profile(), static_cast(p._partition_count), + new_level); - auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION({ return _recover_spill_data_from_disk(state, query_id); }); - }(); - LOG_IF(INFO, !status.ok()) << fmt::format( - "Query:{}, agg probe:{}, task:{}, recover exception:{}", print_id(query_id), - _parent->node_id(), state->task_id(), status.to_string()); - return status; - }; + // Setup persistent output writers for the repartitioner. + RETURN_IF_ERROR(_repartitioner.setup_output(state, output_spill_files)); - DBUG_EXECUTE_IF("fault_inject::partitioned_agg_source::submit_func", { - return Status::Error( - "fault_inject partitioned_agg_source submit_func failed"); - }); + // 2. Flush the in-memory hash table into the sub-spill-files. + RETURN_IF_ERROR(_flush_hash_table_to_sub_spill_files(state)); - VLOG_DEBUG << fmt::format( - "Query:{}, agg probe:{}, task:{}, begin to recover, partitions left:{}, ", - print_id(query_id), _parent->node_id(), state->task_id(), - _shared_state->spill_partitions.size()); - return SpillRecoverRunnable(state, operator_profile(), exception_catch_func).run(); + // 3. Route any in-memory blocks that were recovered but not yet merged + // into the hash table. These blocks were already read from the file + // by _current_reader and must not be re-read. + for (auto& blk : _blocks) { + if (!blk.empty()) { + RETURN_IF_ERROR(_repartitioner.route_block(state, blk)); + } + } + _blocks.clear(); + + // 4. Repartition remaining unread data from the spill file. + // + // If _current_reader exists, the file has been partially read. Pass + // the existing reader to repartitioner so it continues from the current + // position. This avoids re-reading from block 0 and duplicating data + // already represented by the hash table flush + _blocks routed above. + if (_current_reader) { + bool done = false; + while (!done && !state->is_cancelled()) { + RETURN_IF_ERROR(_repartitioner.repartition(state, _current_reader, &done)); + } + // reader is reset by repartitioner on completion + } else if (_current_partition.spill_file) { + // No partial read — repartition the entire file from scratch. + bool done = false; + while (!done && !state->is_cancelled()) { + RETURN_IF_ERROR( + _repartitioner.repartition(state, _current_partition.spill_file, &done)); + } + } + _current_reader.reset(); + _current_partition.spill_file.reset(); + + RETURN_IF_ERROR(_repartitioner.finalize()); + + // 4. Push non-empty sub-partitions into the work queue. + for (int i = 0; i < static_cast(p._partition_count); ++i) { + _partition_queue.emplace_back(std::move(output_spill_files[i]), new_level); + // Metrics + COUNTER_UPDATE(_total_partition_spills, 1); + if (new_level > _max_partition_level_seen) { + _max_partition_level_seen = new_level; + COUNTER_SET(_max_partition_level, int64_t(_max_partition_level_seen)); + } + } + + _estimate_memory_usage = 0; + return Status::OK(); } bool PartitionedAggLocalState::is_blockable() const { - return _shared_state->is_spilled; + return _shared_state->_is_spilled; } #include "common/compile_check_end.h" diff --git a/be/src/exec/operator/partitioned_aggregation_source_operator.h b/be/src/exec/operator/partitioned_aggregation_source_operator.h index f91e6c1bd5db96..d59fea263b0397 100644 --- a/be/src/exec/operator/partitioned_aggregation_source_operator.h +++ b/be/src/exec/operator/partitioned_aggregation_source_operator.h @@ -16,10 +16,15 @@ // under the License. #pragma once +#include #include +#include #include "common/status.h" -#include "exec/operator/operator.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_repartitioner.h" +#include "operator.h" namespace doris { #include "common/compile_check_begin.h" @@ -28,6 +33,19 @@ class RuntimeState; class PartitionedAggSourceOperatorX; class PartitionedAggLocalState; +/// Represents one partition in the multi-level spill queue for aggregation. +/// Unlike Join (which has build + probe), Agg only has a single data flow: +/// spilled aggregation intermediate results stored in one SpillFile. +struct AggSpillPartitionInfo { + // The spill file for this partition. + SpillFileSPtr spill_file; + // The depth level in the repartition tree (level-0 = original). + int level = 0; + + AggSpillPartitionInfo() = default; + AggSpillPartitionInfo(SpillFileSPtr s, int lvl) : spill_file(std::move(s)), level(lvl) {} +}; + class PartitionedAggLocalState MOCK_REMOVE(final) : public PipelineXSpillLocalState { public: @@ -41,30 +59,58 @@ class PartitionedAggLocalState MOCK_REMOVE(final) Status open(RuntimeState* state) override; Status close(RuntimeState* state) override; - Status recover_blocks_from_disk(RuntimeState* state, bool& has_data); - Status setup_in_memory_agg_op(RuntimeState* state); + bool is_blockable() const override; + +private: + friend class PartitionedAggSourceOperatorX; + + Status _setup_in_memory_agg_op(RuntimeState* state); template - void update_profile(RuntimeProfile* child_profile); + void _update_profile(RuntimeProfile* child_profile); - bool is_blockable() const override; + /// Flush the current in-memory hash table by draining it as blocks and routing + /// each block through the repartitioner into the output sub-spill-files. + Status _flush_hash_table_to_sub_spill_files(RuntimeState* state); -private: - Status _recover_spill_data_from_disk(RuntimeState* state, const UniqueId& query_id); + /// Flush the in-memory hash table into FANOUT sub-spill-files, repartition remaining + /// unread spill files from `remaining_spill_files`, and push resulting sub-partitions into + /// `_partition_queue`. After this call the hash table is reset and + /// `remaining_spill_files` is cleared. + Status _flush_and_repartition(RuntimeState* state); -protected: - friend class PartitionedAggSourceOperatorX; - std::unique_ptr _runtime_state; + /// Move all original spill_partitions from shared state into `_partition_queue`. + /// Called once when spilled get_block is first entered. + void _init_partition_queue(); + + /// Read up to SpillFile::MAX_SPILL_WRITE_BATCH_MEM bytes from `partition.spill_files` into + /// `_blocks`. Returns has_data=true if any blocks were read. + /// Consumes and deletes exhausted spill files from the partition. + Status _recover_blocks_from_partition(RuntimeState* state, AggSpillPartitionInfo& partition); + // ── State ────────────────────────────────────────────────────────── + std::unique_ptr _runtime_state; bool _opened = false; - std::unique_ptr> _spill_merge_promise; - std::future _spill_merge_future; - bool _current_partition_eos = true; - bool _need_to_merge_data_for_current_partition = true; + std::unique_ptr _internal_runtime_profile; + + // ── Partition queue (unified for original + repartitioned) ──────── + std::deque _partition_queue; + AggSpillPartitionInfo _current_partition; + // True when we need to pop the next partition from `_partition_queue`. + bool _need_to_setup_partition = true; + // Blocks recovered from disk, pending merge into hash table. std::vector _blocks; - std::unique_ptr _internal_runtime_profile; + // Counters to track spill partition metrics + RuntimeProfile::Counter* _max_partition_level = nullptr; + RuntimeProfile::Counter* _total_partition_spills = nullptr; + int _max_partition_level_seen = 0; + + SpillRepartitioner _repartitioner; + + // Persistent reader for _recover_blocks_from_partition (survives across yield calls) + SpillFileReaderSPtr _current_reader; }; class AggSourceOperatorX; @@ -93,10 +139,23 @@ class PartitionedAggSourceOperatorX : public OperatorX bool is_colocated_operator() const override; bool is_shuffled_operator() const override; + // Returns the current in-memory hash table size for the active partition. + // The scheduler uses this to decide whether to trigger revoke_memory. + size_t revocable_mem_size(RuntimeState* state) const override; + + // Called by the pipeline task scheduler under memory pressure. Flushes the + // current in-memory aggregation hash table to sub-spill-files and repartitions, + // freeing the hash table memory so it can be recovered in smaller slices. + Status revoke_memory(RuntimeState* state) override; + private: friend class PartitionedAggLocalState; std::unique_ptr _agg_source_operator; + // number of spill partitions configured for this operator + size_t _partition_count = 0; + // max repartition depth (configured from session variable in FE) + int _repartition_max_depth = SpillRepartitioner::MAX_DEPTH; }; #include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/exec/operator/partitioned_hash_join_probe_operator.cpp b/be/src/exec/operator/partitioned_hash_join_probe_operator.cpp index 69950ad41a7db3..d5267969cd4285 100644 --- a/be/src/exec/operator/partitioned_hash_join_probe_operator.cpp +++ b/be/src/exec/operator/partitioned_hash_join_probe_operator.cpp @@ -20,6 +20,7 @@ #include #include +#include #include #include @@ -28,12 +29,16 @@ #include "common/status.h" #include "core/block/block.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" +#include "exec/spill/spill_repartitioner.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" namespace doris { + #include "common/compile_check_begin.h" PartitionedHashJoinProbeLocalState::PartitionedHashJoinProbeLocalState(RuntimeState* state, @@ -50,19 +55,24 @@ Status PartitionedHashJoinProbeLocalState::init(RuntimeState* state, LocalStateI auto& p = _parent->cast(); _partitioned_blocks.resize(p._partition_count); - _probe_spilling_streams.resize(p._partition_count); + _probe_spilling_groups.resize(p._partition_count); + _probe_writers.resize(p._partition_count); + // The repartitioner fanout will be configured when the repartitioner is + // initialized with a fanout-sized partitioner clone in the repartition path. init_counters(); return Status::OK(); } void PartitionedHashJoinProbeLocalState::init_counters() { - _partition_timer = ADD_TIMER(custom_profile(), "SpillPartitionTime"); - _partition_shuffle_timer = ADD_TIMER(custom_profile(), "SpillPartitionShuffleTime"); + _partition_shuffle_timer = ADD_TIMER(custom_profile(), "SpillRePartitionTime"); _spill_build_rows = ADD_COUNTER(custom_profile(), "SpillBuildRows", TUnit::UNIT); _spill_build_timer = ADD_TIMER_WITH_LEVEL(custom_profile(), "SpillBuildTime", 1); _recovery_build_rows = ADD_COUNTER(custom_profile(), "SpillRecoveryBuildRows", TUnit::UNIT); + _recovery_level0_build_rows = + ADD_COUNTER(custom_profile(), "SpillRecoveryLevel0BuildRows", TUnit::UNIT); _recovery_build_timer = ADD_TIMER_WITH_LEVEL(custom_profile(), "SpillRecoveryBuildTime", 1); _spill_probe_rows = ADD_COUNTER(custom_profile(), "SpillProbeRows", TUnit::UNIT); + _build_rows = ADD_COUNTER(custom_profile(), "BuildRows", TUnit::UNIT); _recovery_probe_rows = ADD_COUNTER(custom_profile(), "SpillRecoveryProbeRows", TUnit::UNIT); _spill_build_blocks = ADD_COUNTER(custom_profile(), "SpillBuildBlocks", TUnit::UNIT); _recovery_build_blocks = ADD_COUNTER(custom_profile(), "SpillRecoveryBuildBlocks", TUnit::UNIT); @@ -73,9 +83,13 @@ void PartitionedHashJoinProbeLocalState::init_counters() { _get_child_next_timer = ADD_TIMER_WITH_LEVEL(custom_profile(), "GetChildNextTime", 1); _probe_blocks_bytes = - ADD_COUNTER_WITH_LEVEL(custom_profile(), "ProbeBloksBytesInMem", TUnit::BYTES, 1); + ADD_COUNTER_WITH_LEVEL(custom_profile(), "ProbeBlocksBytesInMem", TUnit::BYTES, 1); _memory_usage_reserved = ADD_COUNTER_WITH_LEVEL(custom_profile(), "MemoryUsageReserved", TUnit::BYTES, 1); + + // Counters for partition spill metrics + _max_partition_level = ADD_COUNTER(custom_profile(), "SpillMaxPartitionLevel", TUnit::UNIT); + _total_partition_spills = ADD_COUNTER(custom_profile(), "SpillTotalPartitions", TUnit::UNIT); } template @@ -131,11 +145,11 @@ void PartitionedHashJoinProbeLocalState::update_probe_common_profile( void PartitionedHashJoinProbeLocalState::update_profile_from_inner() { auto& p = _parent->cast(); - if (_shared_state->inner_runtime_state) { - auto* sink_local_state = _shared_state->inner_runtime_state->get_sink_local_state(); - auto* probe_local_state = _shared_state->inner_runtime_state->get_local_state( + if (_shared_state->_inner_runtime_state) { + auto* sink_local_state = _shared_state->_inner_runtime_state->get_sink_local_state(); + auto* probe_local_state = _shared_state->_inner_runtime_state->get_local_state( p._inner_probe_operator->operator_id()); - if (_shared_state->is_spilled) { + if (_shared_state->_is_spilled) { update_build_custom_profile(sink_local_state->custom_profile()); update_probe_custom_profile(probe_local_state->custom_profile()); update_build_common_profile(sink_local_state->common_profile()); @@ -151,8 +165,24 @@ void PartitionedHashJoinProbeLocalState::update_profile_from_inner() { Status PartitionedHashJoinProbeLocalState::open(RuntimeState* state) { RETURN_IF_ERROR(PipelineXSpillLocalState::open(state)); - return _parent->cast()._partitioner->clone(state, - _partitioner); + auto& p = _parent->cast(); + RETURN_IF_ERROR(p._partitioner->clone(state, _partitioner)); + + // Create a fanout-sized partitioner for repartitioning. + // Use operator-configured partition count instead of static FANOUT. + _fanout_partitioner = + std::make_unique(static_cast(p._partition_count)); + RETURN_IF_ERROR(_fanout_partitioner->init(p._probe_exprs)); + RETURN_IF_ERROR(_fanout_partitioner->prepare(state, p._child->row_desc())); + RETURN_IF_ERROR(_fanout_partitioner->open(state)); + + _build_fanout_partitioner = + std::make_unique(static_cast(p._partition_count)); + RETURN_IF_ERROR(_build_fanout_partitioner->init(p._build_exprs)); + RETURN_IF_ERROR(_build_fanout_partitioner->prepare(state, p._build_side_child->row_desc())); + RETURN_IF_ERROR(_build_fanout_partitioner->open(state)); + + return Status::OK(); } Status PartitionedHashJoinProbeLocalState::close(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); @@ -160,334 +190,340 @@ Status PartitionedHashJoinProbeLocalState::close(RuntimeState* state) { if (_closed) { return Status::OK(); } - RETURN_IF_ERROR(PipelineXSpillLocalState::close(state)); - return Status::OK(); -} -Status PartitionedHashJoinProbeLocalState::_execute_spill_probe_blocks(RuntimeState* state, - const UniqueId& query_id) { - SCOPED_TIMER(_spill_probe_timer); - - size_t not_revoked_size = 0; - auto& p = _parent->cast(); - for (uint32_t partition_index = 0; partition_index != p._partition_count; ++partition_index) { - auto& blocks = _probe_blocks[partition_index]; - auto& partitioned_block = _partitioned_blocks[partition_index]; - if (partitioned_block) { - const auto size = partitioned_block->allocated_bytes(); - if (size >= SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { - blocks.emplace_back(partitioned_block->to_block()); - partitioned_block.reset(); - } else { - not_revoked_size += size; + Status first_error; + for (auto& writer : _probe_writers) { + if (writer) { + auto st = writer->close(); + if (!st.ok() && first_error.ok()) { + first_error = st; } + writer.reset(); } + } + _probe_writers.clear(); - if (blocks.empty()) { - continue; + if (_current_build_reader) { + auto st = _current_build_reader->close(); + if (!st.ok() && first_error.ok()) { + first_error = st; } - - 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->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), operator_profile())); + _current_build_reader.reset(); + } + if (_current_probe_reader) { + auto st = _current_probe_reader->close(); + if (!st.ok() && first_error.ok()) { + first_error = st; } + _current_probe_reader.reset(); + } - auto merged_block = MutableBlock::create_unique(std::move(blocks.back())); - blocks.pop_back(); - - while (!blocks.empty() && !state->is_cancelled()) { - auto block = std::move(blocks.back()); - blocks.pop_back(); - - RETURN_IF_ERROR(merged_block->merge(std::move(block))); - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::spill_probe_blocks", { - return Status::Error( - "fault_inject partitioned_hash_join_probe " - "spill_probe_blocks failed"); - }); + // Clean up any remaining spill partition queue entries + for (auto& entry : _spill_partition_queue) { + if (entry.build_file) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(entry.build_file); } - - if (!merged_block->empty()) [[likely]] { - COUNTER_UPDATE(_spill_probe_rows, merged_block->rows()); - RETURN_IF_ERROR(spilling_stream->spill_block(state, merged_block->to_block(), false)); - COUNTER_UPDATE(_spill_probe_blocks, 1); + if (entry.probe_file) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(entry.probe_file); } } + _spill_partition_queue.clear(); + if (_current_partition.build_file) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(_current_partition.build_file); + } + if (_current_partition.probe_file) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(_current_partition.probe_file); + } + _current_partition = JoinSpillPartitionInfo {}; + _queue_probe_blocks.clear(); - COUNTER_SET(_probe_blocks_bytes, int64_t(not_revoked_size)); - - VLOG_DEBUG << fmt::format( - "Query:{}, hash join probe:{}, task:{}," - " spill_probe_blocks done", - print_id(query_id), p.node_id(), state->task_id()); - return Status::OK(); -} - -Status PartitionedHashJoinProbeLocalState::spill_probe_blocks(RuntimeState* state) { - auto query_id = state->query_id(); - - auto exception_catch_func = [this, query_id, state]() { - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::spill_probe_blocks_cancel", { - auto status = Status::InternalError( - "fault_inject partitioned_hash_join_probe " - "spill_probe_blocks canceled"); - state->get_query_ctx()->cancel(status); - return status; - }); - - auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION({ return _execute_spill_probe_blocks(state, query_id); }); - }(); - return status; - }; - - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::spill_probe_blocks_submit_func", { - return Status::Error( - "fault_inject partitioned_hash_join_probe spill_probe_blocks " - "submit_func failed"); - }); - - SpillNonSinkRunnable spill_runnable(state, operator_profile(), exception_catch_func); - return spill_runnable.run(); + auto st = PipelineXSpillLocalState::close(state); + if (!first_error.ok()) { + return first_error; + } + return st; } -Status PartitionedHashJoinProbeLocalState::finish_spilling(uint32_t partition_index) { - auto& probe_spilling_stream = _probe_spilling_streams[partition_index]; - - if (probe_spilling_stream) { - RETURN_IF_ERROR(probe_spilling_stream->spill_eof()); - probe_spilling_stream->set_read_counters(operator_profile()); +Status PartitionedHashJoinProbeLocalState::acquire_spill_writer(RuntimeState* state, + int partition_index, + SpillFileWriterSPtr& writer) { + if (!_probe_writers[partition_index]) { + auto& spill_file = _probe_spilling_groups[partition_index]; + auto relative_path = fmt::format("{}/{}-{}-{}-{}", print_id(state->query_id()), + "hash_probe", _parent->node_id(), state->task_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + spill_file)); + RETURN_IF_ERROR(spill_file->create_writer(state, operator_profile(), + _probe_writers[partition_index])); } - + writer = _probe_writers[partition_index]; return Status::OK(); } -Status PartitionedHashJoinProbeLocalState::recover_build_blocks_from_disk(RuntimeState* state, - uint32_t partition_index, - bool& has_data) { - VLOG_DEBUG << fmt::format( - "Query:{}, hash join probe:{}, task:{}," - " partition:{}, recover_build_blocks_from_disk", - print_id(state->query_id()), _parent->node_id(), state->task_id(), partition_index); - auto& spilled_stream = _shared_state->spilled_streams[partition_index]; - has_data = false; - if (!spilled_stream) { - return Status::OK(); - } - spilled_stream->set_read_counters(operator_profile()); - +Status PartitionedHashJoinProbeLocalState::spill_probe_blocks(RuntimeState* state, bool flush_all) { auto query_id = state->query_id(); + SCOPED_TIMER(_spill_probe_timer); - auto read_func = [this, query_id, state, spilled_stream = spilled_stream, partition_index] { - SCOPED_TIMER(_recovery_build_timer); - - bool eos = false; - VLOG_DEBUG << fmt::format( - "Query:{}, hash join probe:{}, task:{}," - " partition:{}, recoverying build data", - print_id(state->query_id()), _parent->node_id(), state->task_id(), partition_index); - Status status; - while (!eos) { - Block block; - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recover_build_blocks", { - status = Status::Error( - "fault_inject partitioned_hash_join_probe " - "recover_build_blocks failed"); - }); - if (status.ok()) { - status = spilled_stream->read_next_block_sync(&block, &eos); - } - if (!status.ok()) { - break; - } - COUNTER_UPDATE(_recovery_build_rows, block.rows()); - COUNTER_UPDATE(_recovery_build_blocks, 1); - - if (block.empty()) { - continue; - } - - if (UNLIKELY(state->is_cancelled())) { - LOG(INFO) << fmt::format( - "Query:{}, hash join probe:{}, task:{}," - " partition:{}, recovery build data canceled", - print_id(state->query_id()), _parent->node_id(), state->task_id(), - partition_index); - break; - } - - if (!_recovered_build_block) { - _recovered_build_block = MutableBlock::create_unique(std::move(block)); - } else { - DCHECK_EQ(_recovered_build_block->columns(), block.columns()); - status = _recovered_build_block->merge(std::move(block)); - if (!status.ok()) { - break; - } - } - - if (_recovered_build_block->allocated_bytes() >= - SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { - break; - } + DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::spill_probe_blocks_cancel", { + return Status::InternalError( + "fault_inject partitioned_hash_join_probe " + "spill_probe_blocks canceled"); + }); + auto& p = _parent->cast(); + for (uint32_t partition_index = 0; partition_index != p._partition_count; ++partition_index) { + auto& partitioned_block = _partitioned_blocks[partition_index]; + if (!partitioned_block || partitioned_block->empty()) { + continue; } - if (eos) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(spilled_stream); - _shared_state->spilled_streams[partition_index].reset(); - VLOG_DEBUG << fmt::format( - "Query:{}, hash join probe:{}, task:{}," - " partition:{}, recovery build data eos", - print_id(state->query_id()), _parent->node_id(), state->task_id(), - partition_index); + if (!flush_all && + partitioned_block->allocated_bytes() < SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { + continue; } - return status; - }; - auto exception_catch_func = [read_func, state, query_id]() { - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recover_build_blocks_cancel", { - auto status = Status::InternalError( - "fault_inject partitioned_hash_join_probe " - "recover_build_blocks canceled"); + SpillFileWriterSPtr writer; + RETURN_IF_ERROR(acquire_spill_writer(state, partition_index, writer)); - state->get_query_ctx()->cancel(status); - return status; + DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::spill_probe_blocks", { + return Status::Error( + "fault_inject partitioned_hash_join_probe " + "spill_probe_blocks failed"); }); - auto status = [&]() { - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(read_func()); - return Status::OK(); - }(); - - return status; - }; - - has_data = true; - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recovery_build_blocks_submit_func", - { - return Status::Error( - "fault_inject partitioned_hash_join_probe " - "recovery_build_blocks submit_func failed"); - }); + COUNTER_UPDATE(_spill_probe_rows, partitioned_block->rows()); + RETURN_IF_ERROR(writer->write_block(state, partitioned_block->to_block())); + COUNTER_UPDATE(_spill_probe_blocks, 1); + partitioned_block.reset(); + } - SpillRecoverRunnable spill_runnable(state, operator_profile(), exception_catch_func); - return spill_runnable.run(); + VLOG_DEBUG << fmt::format( + "Query:{}, hash join probe:{}, task:{}," + " spill_probe_blocks done", + print_id(query_id), p.node_id(), state->task_id()); + return Status::OK(); } std::string PartitionedHashJoinProbeLocalState::debug_string(int indentation_level) const { auto& p = _parent->cast(); bool need_more_input_data; - if (_shared_state->is_spilled) { + if (_shared_state->_is_spilled) { need_more_input_data = !_child_eos; - } else if (_shared_state->inner_runtime_state) { + } else if (_shared_state->_inner_runtime_state) { need_more_input_data = p._inner_probe_operator->need_more_input_data( - _shared_state->inner_runtime_state.get()); + _shared_state->_inner_runtime_state.get()); } else { need_more_input_data = true; } fmt::memory_buffer debug_string_buffer; fmt::format_to(debug_string_buffer, "{}, short_circuit_for_probe: {}, is_spilled: {}, child_eos: {}, " - "_shared_state->inner_runtime_state: {}, need_more_input_data: {}", + "_shared_state->_inner_runtime_state: {}, need_more_input_data: {}", PipelineXSpillLocalState::debug_string( indentation_level), _shared_state ? std::to_string(_shared_state->short_circuit_for_probe) : "NULL", - _shared_state->is_spilled, _child_eos, - _shared_state->inner_runtime_state != nullptr, need_more_input_data); + _shared_state->_is_spilled, _child_eos, + _shared_state->_inner_runtime_state != nullptr, need_more_input_data); return fmt::to_string(debug_string_buffer); } -Status PartitionedHashJoinProbeLocalState::recover_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) { +bool PartitionedHashJoinProbeLocalState::is_blockable() const { + return _shared_state->_is_spilled; +} + +Status PartitionedHashJoinProbeLocalState::recover_build_blocks_from_partition( + RuntimeState* state, JoinSpillPartitionInfo& partition_info) { + if (!partition_info.build_file) { + // Build file is already exhausted for this partition. return Status::OK(); } + SCOPED_TIMER(_recovery_build_timer); + DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recover_build_blocks", { + return Status::InternalError( + "fault_inject partitioned_hash_join_probe " + "recover_build_blocks failed"); + }); + // Create reader if needed (persistent across scheduling slices) + if (!_current_build_reader) { + _current_build_reader = partition_info.build_file->create_reader(state, operator_profile()); + RETURN_IF_ERROR(_current_build_reader->open()); + } + bool eos = false; + while (!eos) { + Block block; + RETURN_IF_ERROR(_current_build_reader->read(&block, &eos)); + COUNTER_UPDATE(_recovery_build_rows, block.rows()); + if (partition_info.level == 0) { + COUNTER_UPDATE(_recovery_level0_build_rows, block.rows()); + } + if (block.empty()) { + continue; + } + COUNTER_UPDATE(_recovery_build_blocks, 1); + if (UNLIKELY(state->is_cancelled())) { + return state->cancel_reason(); + } + if (!_recovered_build_block) { + // This will merge the block to recover build block, so that has to use else here. + _recovered_build_block = MutableBlock::create_unique(std::move(block)); + } else { + RETURN_IF_ERROR(_recovered_build_block->merge(std::move(block))); + } + if (_recovered_build_block->allocated_bytes() >= state->spill_buffer_size_bytes()) { + return Status::OK(); // yield — buffer full, more data may remain + } + } + // Build file fully consumed. + RETURN_IF_ERROR(_current_build_reader->close()); + _current_build_reader.reset(); + partition_info.build_file.reset(); + return Status::OK(); +} - spilled_stream->set_read_counters(operator_profile()); - auto& blocks = _probe_blocks[partition_index]; +Status PartitionedHashJoinProbeLocalState::recover_probe_blocks_from_partition( + RuntimeState* state, JoinSpillPartitionInfo& partition_info) { + if (!partition_info.probe_file) { + // Probe file is already exhausted for this partition. + return Status::OK(); + } - auto query_id = state->query_id(); + // For multi-level queue partitions, store recovered probe blocks in _queue_probe_blocks. + SCOPED_TIMER(_recovery_probe_timer); + size_t read_size = 0; + // Create reader if needed + if (!_current_probe_reader) { + _current_probe_reader = partition_info.probe_file->create_reader(state, operator_profile()); + RETURN_IF_ERROR(_current_probe_reader->open()); + } + bool eos = false; + while (!eos && !state->is_cancelled()) { + Block block; + RETURN_IF_ERROR(_current_probe_reader->read(&block, &eos)); + if (!block.empty()) { + COUNTER_UPDATE(_recovery_probe_rows, block.rows()); + COUNTER_UPDATE(_recovery_probe_blocks, 1); + read_size += block.allocated_bytes(); + _queue_probe_blocks.emplace_back(std::move(block)); + } + if (read_size >= state->spill_buffer_size_bytes()) { + return Status::OK(); // yield — enough data read + } + } + // Probe file fully consumed. + RETURN_IF_ERROR(_current_probe_reader->close()); + _current_probe_reader.reset(); + partition_info.probe_file.reset(); + return Status::OK(); +} - auto read_func = [this, query_id, partition_index, &spilled_stream, &blocks] { - SCOPED_TIMER(_recovery_probe_timer); +Status PartitionedHashJoinProbeLocalState::repartition_current_partition( + RuntimeState* state, JoinSpillPartitionInfo& partition) { + auto& p = _parent->cast(); + const int new_level = partition.level + 1; - Block block; - bool eos = false; - Status st; - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recover_probe_blocks", { - st = Status::Error( - "fault_inject partitioned_hash_join_probe recover_probe_blocks failed"); - }); + if (new_level >= p._repartition_max_depth) { + return Status::InternalError( + "query:{}, node:{}, Hash join spill repartition exceeded max depth {}. " + "Likely due to extreme data skew.", + print_id(state->query_id()), p.node_id(), p._repartition_max_depth); + } - size_t read_size = 0; - while (!eos && !_state->is_cancelled() && st.ok()) { - st = spilled_stream->read_next_block_sync(&block, &eos); - if (!st.ok()) { - break; - } else if (!block.empty()) { - COUNTER_UPDATE(_recovery_probe_rows, block.rows()); - COUNTER_UPDATE(_recovery_probe_blocks, 1); - read_size += block.allocated_bytes(); - blocks.emplace_back(std::move(block)); - } + VLOG_DEBUG << fmt::format( + "Query:{}, hash join probe:{}, task:{}, repartitioning partition at level {} to " + "level {}", + print_id(state->query_id()), p.node_id(), state->task_id(), partition.level, new_level); + + // Create a partitioner for repartitioning build data. + std::unique_ptr build_fanout_clone; + RETURN_IF_ERROR(_build_fanout_partitioner->clone(state, build_fanout_clone)); + _repartitioner.init(std::move(build_fanout_clone), operator_profile(), + static_cast(p._partition_count), new_level); + + // Repartition build files + std::vector build_output_spill_files; + RETURN_IF_ERROR(SpillRepartitioner::create_output_spill_files( + state, p.node_id(), fmt::format("hash_build_repart_l{}", new_level), + static_cast(p._partition_count), build_output_spill_files)); + + RETURN_IF_ERROR(_repartitioner.setup_output(state, build_output_spill_files)); + + // Route already-recovered in-memory build data first — these rows have + // already been read from the file by _current_build_reader and must not + // be re-read by the repartitioner. + if (_recovered_build_block && _recovered_build_block->rows() > 0) { + auto recovered_block = _recovered_build_block->to_block(); + RETURN_IF_ERROR(_repartitioner.route_block(state, recovered_block)); + } - if (read_size >= SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { - break; - } + // Repartition the remaining unread portion of the build file. + // If _current_build_reader exists the file was partially read; pass the + // reader directly so the repartitioner continues from the current + // position instead of re-reading from block 0 (which would duplicate + // the rows already routed above). + if (_current_build_reader) { + bool done = false; + while (!done && !state->is_cancelled()) { + RETURN_IF_ERROR(_repartitioner.repartition(state, _current_build_reader, &done)); } - if (eos) { - VLOG_DEBUG << fmt::format( - "Query:{}, hash join probe:{}, task:{}," - " partition:{}, recovery probe data done", - print_id(query_id), _parent->node_id(), _state->task_id(), partition_index); - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(spilled_stream); - spilled_stream.reset(); + // reader is reset by repartitioner on completion + } else if (partition.build_file) { + // No partial read — repartition the entire file from scratch. + bool done = false; + while (!done && !state->is_cancelled()) { + RETURN_IF_ERROR(_repartitioner.repartition(state, partition.build_file, &done)); + } + } + RETURN_IF_ERROR(_repartitioner.finalize()); + _recovered_build_block.reset(); + _current_build_reader.reset(); // clear any leftover reader state + partition.build_file.reset(); + + // Repartition probe files + std::vector probe_output_spill_files; + RETURN_IF_ERROR(SpillRepartitioner::create_output_spill_files( + state, p.node_id(), fmt::format("hash_probe_repart_l{}", new_level), + static_cast(p._partition_count), probe_output_spill_files)); + + if (partition.probe_file) { + // Re-init repartitioner with a fresh FANOUT partitioner clone for probe data + std::unique_ptr probe_fanout_clone; + RETURN_IF_ERROR(_fanout_partitioner->clone(state, probe_fanout_clone)); + _repartitioner.init(std::move(probe_fanout_clone), operator_profile(), + static_cast(p._partition_count), new_level); + + RETURN_IF_ERROR(_repartitioner.setup_output(state, probe_output_spill_files)); + + bool done = false; + while (!done && !state->is_cancelled()) { + RETURN_IF_ERROR(_repartitioner.repartition(state, partition.probe_file, &done)); } - return st; - }; + partition.probe_file.reset(); - auto exception_catch_func = [read_func, state, query_id]() { - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recover_probe_blocks_cancel", { - auto status = Status::InternalError( - "fault_inject partitioned_hash_join_probe " - "recover_probe_blocks canceled"); - state->get_query_ctx()->cancel(status); - return status; - }); + RETURN_IF_ERROR(_repartitioner.finalize()); + _current_probe_reader.reset(); + } - auto status = [&]() { - RETURN_IF_ERROR_OR_CATCH_EXCEPTION(read_func()); - return Status::OK(); - }(); - - return status; - }; - - has_data = true; - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recovery_probe_blocks_submit_func", - { - return Status::Error( - "fault_inject partitioned_hash_join_probe " - "recovery_probe_blocks submit_func failed"); - }); - return SpillRecoverRunnable(state, operator_profile(), exception_catch_func).run(); -} + // Push all sub-partitions into work queue; build/probe emptiness is handled + // later during recovery. New sub-partitions start with build_finished = + // probe_finished = false (via constructor). + for (int i = 0; i < static_cast(p._partition_count); ++i) { + _spill_partition_queue.emplace_back(std::move(build_output_spill_files[i]), + std::move(probe_output_spill_files[i]), new_level); + // Metrics + COUNTER_UPDATE(_total_partition_spills, 1); + if (new_level > _max_partition_level_seen) { + _max_partition_level_seen = new_level; + COUNTER_SET(_max_partition_level, int64_t(_max_partition_level_seen)); + } + } -bool PartitionedHashJoinProbeLocalState::is_blockable() const { - return _shared_state->is_spilled; + return Status::OK(); } PartitionedHashJoinProbeOperatorX::PartitionedHashJoinProbeOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, - const DescriptorTbl& descs, - uint32_t partition_count) + const DescriptorTbl& descs) : JoinProbeOperatorX(pool, tnode, operator_id, descs), _join_distribution(tnode.hash_join_node.__isset.dist_type ? tnode.hash_join_node.dist_type : TJoinDistributionType::NONE), @@ -495,10 +531,18 @@ PartitionedHashJoinProbeOperatorX::PartitionedHashJoinProbeOperatorX(ObjectPool* ? tnode.distribute_expr_lists[0] : std::vector {}), _tnode(tnode), - _descriptor_tbl(descs), - _partition_count(partition_count) {} + _descriptor_tbl(descs) {} Status PartitionedHashJoinProbeOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + _partition_count = state->spill_hash_join_partition_count(); + if (_partition_count < 2 || _partition_count > 32) { + return Status::InternalError( + "query:{}, node:{}, invalid partition count {}. Must be between 2 and 32.", + print_id(state->query_id()), node_id(), _partition_count); + } + + // default repartition max depth; can be overridden from session variable + _repartition_max_depth = state->spill_repartition_max_depth(); RETURN_IF_ERROR(JoinProbeOperatorX::init(tnode, state)); _op_name = "PARTITIONED_HASH_JOIN_PROBE_OPERATOR"; auto tnode_ = _tnode; @@ -506,6 +550,7 @@ Status PartitionedHashJoinProbeOperatorX::init(const TPlanNode& tnode, RuntimeSt for (const auto& conjunct : tnode.hash_join_node.eq_join_conjuncts) { _probe_exprs.emplace_back(conjunct.left); + _build_exprs.emplace_back(conjunct.right); } _partitioner = std::make_unique(_partition_count); RETURN_IF_ERROR(_partitioner->init(_probe_exprs)); @@ -535,20 +580,11 @@ Status PartitionedHashJoinProbeOperatorX::push(RuntimeState* state, Block* input 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)); - } + SCOPED_TIMER(local_state._partition_shuffle_timer); + + RETURN_IF_ERROR(local_state._partitioner->do_partitioning(state, input_block)); std::vector> partition_indexes(_partition_count); const auto& channel_ids = local_state._partitioner->get_channel_ids(); @@ -556,7 +592,6 @@ Status PartitionedHashJoinProbeOperatorX::push(RuntimeState* state, Block* input partition_indexes[channel_ids[i]].emplace_back(i); } - SCOPED_TIMER(local_state._partition_shuffle_timer); int64_t bytes_of_blocks = 0; for (uint32_t i = 0; i != _partition_count; ++i) { const auto count = partition_indexes[i].size(); @@ -570,16 +605,17 @@ Status PartitionedHashJoinProbeOperatorX::push(RuntimeState* state, Block* input RETURN_IF_ERROR(partitioned_blocks[i]->add_rows(input_block, partition_indexes[i].data(), partition_indexes[i].data() + 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()); + const auto bytes = partitioned_blocks[i]->allocated_bytes(); + if (bytes >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { + SpillFileWriterSPtr writer; + RETURN_IF_ERROR(local_state.acquire_spill_writer(state, i, writer)); + + COUNTER_UPDATE(local_state._spill_probe_rows, partitioned_blocks[i]->rows()); + RETURN_IF_ERROR(writer->write_block(state, partitioned_blocks[i]->to_block())); + COUNTER_UPDATE(local_state._spill_probe_blocks, 1); partitioned_blocks[i].reset(); } else { - bytes_of_blocks += partitioned_blocks[i]->allocated_bytes(); - } - - for (auto& block : local_state._probe_blocks[i]) { - bytes_of_blocks += block.allocated_bytes(); + bytes_of_blocks += bytes; } } @@ -588,24 +624,23 @@ Status PartitionedHashJoinProbeOperatorX::push(RuntimeState* state, Block* input return Status::OK(); } -Status PartitionedHashJoinProbeOperatorX::_setup_internal_operators( +Status PartitionedHashJoinProbeOperatorX::_setup_internal_operators_from_partition( PartitionedHashJoinProbeLocalState& local_state, RuntimeState* state) const { - local_state._shared_state->inner_runtime_state = RuntimeState::create_unique( + local_state._shared_state->_inner_runtime_state = RuntimeState::create_unique( state->fragment_instance_id(), state->query_id(), state->fragment_id(), state->query_options(), TQueryGlobals {}, state->exec_env(), state->get_query_ctx()); - local_state._shared_state->inner_runtime_state->set_task_execution_context( + local_state._shared_state->_inner_runtime_state->set_task_execution_context( state->get_task_execution_context().lock()); - local_state._shared_state->inner_runtime_state->set_be_number(state->be_number()); + local_state._shared_state->_inner_runtime_state->set_be_number(state->be_number()); - local_state._shared_state->inner_runtime_state->set_desc_tbl(&state->desc_tbl()); - local_state._shared_state->inner_runtime_state->resize_op_id_to_local_state(-1); - local_state._shared_state->inner_runtime_state->set_runtime_filter_mgr( + local_state._shared_state->_inner_runtime_state->set_desc_tbl(&state->desc_tbl()); + local_state._shared_state->_inner_runtime_state->resize_op_id_to_local_state(-1); + local_state._shared_state->_inner_runtime_state->set_runtime_filter_mgr( state->local_runtime_filter_mgr()); local_state._in_mem_shared_state_sptr = _inner_sink_operator->create_shared_state(); - // set sink local state LocalSinkStateInfo info {.task_idx = 0, .parent_profile = local_state._internal_runtime_profile.get(), .sender_id = -1, @@ -613,7 +648,7 @@ Status PartitionedHashJoinProbeOperatorX::_setup_internal_operators( .shared_state_map = {}, .tsink = {}}; RETURN_IF_ERROR(_inner_sink_operator->setup_local_state( - local_state._shared_state->inner_runtime_state.get(), info)); + local_state._shared_state->_inner_runtime_state.get(), info)); LocalStateInfo state_info {.parent_profile = local_state._internal_runtime_profile.get(), .scan_ranges = {}, @@ -621,38 +656,37 @@ Status PartitionedHashJoinProbeOperatorX::_setup_internal_operators( .shared_state_map = {}, .task_idx = 0}; RETURN_IF_ERROR(_inner_probe_operator->setup_local_state( - local_state._shared_state->inner_runtime_state.get(), state_info)); + local_state._shared_state->_inner_runtime_state.get(), state_info)); - auto* sink_local_state = local_state._shared_state->inner_runtime_state->get_sink_local_state(); + auto* sink_local_state = + local_state._shared_state->_inner_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._shared_state->inner_runtime_state->get_local_state( + auto* probe_local_state = local_state._shared_state->_inner_runtime_state->get_local_state( _inner_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]; + // Use the recovered build block from the partition stream Block block; - if (partitioned_block && partitioned_block->rows() > 0) { - block = partitioned_block->to_block(); - partitioned_block.reset(); + if (local_state._recovered_build_block && local_state._recovered_build_block->rows() > 0) { + block = local_state._recovered_build_block->to_block(); + local_state._recovered_build_block.reset(); } - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::sink", { - return Status::Error( - "fault_inject partitioned_hash_join_probe sink failed"); - }); - RETURN_IF_ERROR(_inner_sink_operator->sink(local_state._shared_state->inner_runtime_state.get(), - &block, true)); + COUNTER_UPDATE(local_state._build_rows, block.rows()); + + RETURN_IF_ERROR(_inner_sink_operator->sink( + local_state._shared_state->_inner_runtime_state.get(), &block, true)); + local_state._current_partition.build_finished = true; VLOG_DEBUG << fmt::format( "Query:{}, hash join probe:{}, task:{}," - " internal build operator finished, partition:{}, rows:{}, memory usage:{}", - print_id(state->query_id()), node_id(), state->task_id(), local_state._partition_cursor, - block.rows(), + " internal build from partition (level:{}) finished, rows:{}, memory usage:{}", + print_id(state->query_id()), node_id(), state->task_id(), + local_state._current_partition.level, block.rows(), _inner_sink_operator->get_memory_usage( - local_state._shared_state->inner_runtime_state.get())); + local_state._shared_state->_inner_runtime_state.get())); return Status::OK(); } @@ -660,56 +694,113 @@ Status PartitionedHashJoinProbeOperatorX::pull(doris::RuntimeState* state, Block bool* eos) const { auto& local_state = get_local_state(state); - const auto partition_index = local_state._partition_cursor; - auto& probe_blocks = local_state._probe_blocks[partition_index]; - - if (local_state._recovered_build_block && !local_state._recovered_build_block->empty()) { - local_state._estimate_memory_usage += local_state._recovered_build_block->allocated_bytes(); - auto& mutable_block = local_state._shared_state->partitioned_build_blocks[partition_index]; - if (!mutable_block) { - mutable_block = std::move(local_state._recovered_build_block); - } else { - RETURN_IF_ERROR(mutable_block->merge(local_state._recovered_build_block->to_block())); - local_state._recovered_build_block.reset(); + // On first entry after child EOS, populate _spill_partition_queue from the + // per-partition build and probe spill streams. After this point every partition + // (including the original "level-0" ones) is accessed uniformly via the queue. + if (!local_state._spill_queue_initialized) { + DCHECK(local_state._child_eos) << "pull() with is_spilled=true called before child EOS"; + // There maybe some blocks still in partitioned block or probe blocks. Flush them to disk. + RETURN_IF_ERROR(local_state.spill_probe_blocks(state, true)); + // Close all probe writers so that SpillFile metadata (part_count, etc.) + // is finalized and the files become readable. Without this the readers + // would see _part_count == 0 and return no data. + for (auto& writer : local_state._probe_writers) { + if (writer) { + RETURN_IF_ERROR(writer->close()); + } } + for (uint32_t i = 0; i < _partition_count; ++i) { + auto& build_file = local_state._shared_state->_spilled_build_groups[i]; + auto& probe_file = local_state._probe_spilling_groups[i]; + // Transfer SpillFiles into JoinSpillPartitionInfo unconditionally. + local_state._spill_partition_queue.emplace_back(std::move(build_file), + std::move(probe_file), 0); + // Metrics: count this queued partition + COUNTER_UPDATE(local_state._total_partition_spills, 1); + } + local_state._max_partition_level_seen = 0; + COUNTER_SET(local_state._max_partition_level, + int64_t(local_state._max_partition_level_seen)); + local_state._spill_queue_initialized = true; + VLOG_DEBUG << fmt::format( + "Query:{}, hash join probe:{}, task:{}, initialized spill queue with {} partitions", + print_id(state->query_id()), node_id(), state->task_id(), + local_state._spill_partition_queue.size()); } - if (local_state._need_to_setup_internal_operators) { - bool has_data = false; - RETURN_IF_ERROR(local_state.recover_build_blocks_from_disk( - state, local_state._partition_cursor, has_data)); - if (has_data) { + return _pull_from_spill_queue(local_state, state, output_block, eos); +} + +Status PartitionedHashJoinProbeOperatorX::_pull_from_spill_queue( + PartitionedHashJoinProbeLocalState& local_state, RuntimeState* state, Block* output_block, + bool* eos) const { + *eos = false; + + if (local_state._need_to_setup_queue_partition) { + // No more partitions to process and no active partition — EOS. + if (local_state._spill_partition_queue.empty() && + (!local_state._current_partition.is_valid() || + local_state._current_partition.probe_finished)) { + *eos = true; return Status::OK(); } - *eos = false; - RETURN_IF_ERROR(local_state.finish_spilling(partition_index)); - RETURN_IF_ERROR(_setup_internal_operators(local_state, state)); - local_state._need_to_setup_internal_operators = false; - auto& mutable_block = local_state._partitioned_blocks[partition_index]; - if (mutable_block && !mutable_block->empty()) { - probe_blocks.emplace_back(mutable_block->to_block()); + // Pop next partition to process. + // Invariant: we only pop when there is no active current partition and + // no pending recovered build data waiting to be consumed. + if (!local_state._current_partition.is_valid() || + local_state._current_partition.probe_finished) { + local_state._current_partition = std::move(local_state._spill_partition_queue.front()); + local_state._spill_partition_queue.pop_front(); + local_state._recovered_build_block.reset(); + local_state._queue_probe_blocks.clear(); + VLOG_DEBUG << fmt::format( + "Query:{}, hash join probe:{}, task:{}," + " processing queue partition at level:{}, queue remaining:{}", + print_id(state->query_id()), node_id(), state->task_id(), + local_state._current_partition.level, + local_state._spill_partition_queue.size()); } + + // Continue recovering build data while there is unread build file. + if (local_state._current_partition.build_file) { + // Partially read build data — yield so it can be consumed + // before continuing recovery in the next scheduling slice. + return local_state.recover_build_blocks_from_partition(state, + local_state._current_partition); + } + RETURN_IF_ERROR(_setup_internal_operators_from_partition(local_state, state)); + local_state._current_partition.build_finished = true; + local_state._need_to_setup_queue_partition = false; + return Status::OK(); } + + // Probe phase: feed probe blocks from the current partition's probe stream + // into the inner probe operator. bool in_mem_eos = false; - auto* runtime_state = local_state._shared_state->inner_runtime_state.get(); + auto* runtime_state = local_state._shared_state->_inner_runtime_state.get(); + auto& probe_blocks = local_state._queue_probe_blocks; + while (_inner_probe_operator->need_more_input_data(runtime_state)) { if (probe_blocks.empty()) { - *eos = false; - bool has_data = false; - RETURN_IF_ERROR( - local_state.recover_probe_blocks_from_disk(state, partition_index, has_data)); - if (!has_data) { + // Try to recover more probe blocks. If the probe stream is + // finished (probe_file == nullptr) and no blocks are buffered, + // we send EOS to the inner probe operator. + if (!local_state._current_partition.probe_file) { Block block; RETURN_IF_ERROR(_inner_probe_operator->push(runtime_state, &block, true)); VLOG_DEBUG << fmt::format( "Query:{}, hash join probe:{}, task:{}," - " partition:{}, has no data to recovery", - print_id(state->query_id()), node_id(), state->task_id(), partition_index); + " queue partition (level:{}) probe eos", + print_id(state->query_id()), node_id(), state->task_id(), + local_state._current_partition.level); break; - } else { - return Status::OK(); } + + // Probe data recovered — yield to let the pipeline scheduler + // re-schedule us so we can push the recovered blocks. + return local_state.recover_probe_blocks_from_partition(state, + local_state._current_partition); } auto block = std::move(probe_blocks.back()); @@ -718,23 +809,24 @@ Status PartitionedHashJoinProbeOperatorX::pull(doris::RuntimeState* state, Block RETURN_IF_ERROR(_inner_probe_operator->push(runtime_state, &block, false)); } } - - RETURN_IF_ERROR(_inner_probe_operator->pull( - local_state._shared_state->inner_runtime_state.get(), output_block, &in_mem_eos)); - - *eos = false; + RETURN_IF_ERROR(_inner_probe_operator->pull(runtime_state, output_block, &in_mem_eos)); if (in_mem_eos) { VLOG_DEBUG << fmt::format( "Query:{}, hash join probe:{}, task:{}," - " partition:{}, probe done", + " queue partition (level:{}) probe done", print_id(state->query_id()), node_id(), state->task_id(), - local_state._partition_cursor); - local_state._partition_cursor++; + local_state._current_partition.level); local_state.update_profile_from_inner(); - if (local_state._partition_cursor == _partition_count) { + local_state._current_partition.probe_finished = true; + + // Reset for next queue entry — default-constructed partition has + // is_valid() == false, signaling "no partition in progress". + local_state._current_partition = JoinSpillPartitionInfo {}; + local_state._need_to_setup_queue_partition = true; + local_state._queue_probe_blocks.clear(); + + if (local_state._spill_partition_queue.empty()) { *eos = true; - } else { - local_state._need_to_setup_internal_operators = true; } } @@ -743,68 +835,86 @@ Status PartitionedHashJoinProbeOperatorX::pull(doris::RuntimeState* state, Block bool PartitionedHashJoinProbeOperatorX::need_more_input_data(RuntimeState* state) const { auto& local_state = get_local_state(state); - if (local_state._shared_state->is_spilled) { + if (local_state._shared_state->_is_spilled) { return !local_state._child_eos; - } else if (local_state._shared_state->inner_runtime_state) { + } else if (local_state._shared_state->_inner_runtime_state) { return _inner_probe_operator->need_more_input_data( - local_state._shared_state->inner_runtime_state.get()); + local_state._shared_state->_inner_runtime_state.get()); } else { return true; } } +// Report only this operator's own revocable memory. The pipeline task +// iterates all operators to sum revocable sizes and revoke each individually. +// Sum up memory used by in-memory probe blocks and any partially-recovered build block for the current partition. +// This is the memory that can be freed if we choose to revoke and repartition the current size_t PartitionedHashJoinProbeOperatorX::revocable_mem_size(RuntimeState* state) const { auto& local_state = get_local_state(state); - if (local_state._child_eos) { + if (!local_state._shared_state->_is_spilled) { return 0; } - auto revocable_size = _revocable_mem_size(state, true); - if (_child) { - revocable_size += _child->revocable_mem_size(state); - } - return revocable_size; -} - -size_t PartitionedHashJoinProbeOperatorX::_revocable_mem_size(RuntimeState* state, - bool force) const { - const auto spill_size_threshold = - force ? SpillStream::MIN_SPILL_WRITE_BATCH_MEM : SpillStream::MAX_SPILL_WRITE_BATCH_MEM; - auto& local_state = get_local_state(state); size_t mem_size = 0; - auto& probe_blocks = local_state._probe_blocks; - for (uint32_t i = 0; i < _partition_count; ++i) { - for (auto& block : probe_blocks[i]) { - mem_size += block.allocated_bytes(); - } - - auto& partitioned_block = local_state._partitioned_blocks[i]; - if (partitioned_block) { - auto block_bytes = partitioned_block->allocated_bytes(); - if (block_bytes >= spill_size_threshold) { - mem_size += block_bytes; + if (!local_state._child_eos) { + for (uint32_t i = 0; i < _partition_count; ++i) { + auto& partitioned_block = local_state._partitioned_blocks[i]; + if (!partitioned_block) { + continue; + } + const auto bytes = partitioned_block->allocated_bytes(); + if (bytes >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { + mem_size += bytes; } } + return mem_size > state->spill_min_revocable_mem() ? mem_size : 0; + } + if (!local_state._current_partition.is_valid() || + local_state._current_partition.build_finished) { + // No active partition — no revocable memory. + // Or if current partition has finished build hash table. + return 0; + } + + // Include build-side memory that has been recovered but not yet consumed by the hash table. + // This data is revocable because we can repartition instead of building the hash table. + if (local_state._recovered_build_block) { + mem_size += local_state._recovered_build_block->allocated_bytes(); } - return mem_size; + + return mem_size > state->spill_min_revocable_mem() ? mem_size : 0; } size_t PartitionedHashJoinProbeOperatorX::get_reserve_mem_size(RuntimeState* state) { auto& local_state = get_local_state(state); - const auto is_spilled = local_state._shared_state->is_spilled; - if (!is_spilled || local_state._child_eos) { + const bool is_spilled = local_state._shared_state->_is_spilled; + + // Non-spill path: delegate to the inner probe operator / base class. + if (!is_spilled) { return Base::get_reserve_mem_size(state); } - size_t size_to_reserve = SpillStream::MAX_SPILL_WRITE_BATCH_MEM; + // Spill path, probe data still flowing in (child not yet EOS): + // We only need room for incoming probe blocks being partitioned. Reserve + // one batch worth of spill-write memory; no hash table will be built yet. + if (!local_state._child_eos) { + return state->minimum_operator_memory_required_bytes(); + } + + // Spill path, child EOS — we are in the recovery / build / probe phase. + // Baseline reservation is one block of spill I/O. + size_t size_to_reserve = state->minimum_operator_memory_required_bytes(); - if (local_state._need_to_setup_internal_operators) { - const size_t rows = - (local_state._recovered_build_block ? local_state._recovered_build_block->rows() - : 0) + - state->batch_size(); - size_t bucket_size = hash_join_table_calc_bucket_size(rows); + const bool about_to_build = local_state._current_partition.is_valid() && + !local_state._current_partition.build_finished; + if (about_to_build && local_state._recovered_build_block) { + // Estimate rows that will land in the hash table so we can reserve + // enough for JoinHashTable::first[] + JoinHashTable::next[]. + size_t rows = std::max(static_cast(state->batch_size()), + static_cast(local_state._recovered_build_block->rows())); + + const size_t bucket_size = hash_join_table_calc_bucket_size(rows); size_to_reserve += bucket_size * sizeof(uint32_t); // JoinHashTable::first size_to_reserve += rows * sizeof(uint32_t); // JoinHashTable::next @@ -812,51 +922,87 @@ size_t PartitionedHashJoinProbeOperatorX::get_reserve_mem_size(RuntimeState* sta _join_op == TJoinOp::RIGHT_ANTI_JOIN || _join_op == TJoinOp::RIGHT_SEMI_JOIN) { size_to_reserve += rows * sizeof(uint8_t); // JoinHashTable::visited } + + // It is hard to precisely estimate the memory needed for serialized + // keys when building the hash table, so use the current build block + // size as an estimate. This may be imprecise, but it should not + // underestimate the requirement. Hash table construction also merges + // blocks, so this approximation is reasonable here. + size_to_reserve += local_state._recovered_build_block->allocated_bytes(); } + // Otherwise (not about to build): we only need the spill I/O baseline + // already included above — no hash table allocation is imminent. COUNTER_SET(local_state._memory_usage_reserved, int64_t(size_to_reserve)); return size_to_reserve; } -Status PartitionedHashJoinProbeOperatorX::_revoke_memory(RuntimeState* state) { - auto& local_state = get_local_state(state); - VLOG_DEBUG << fmt::format("Query:{}, hash join probe:{}, task:{}, revoke_memory", - print_id(state->query_id()), node_id(), state->task_id()); +// Revoke in-memory build data by repartitioning and spilling to disk. +// +// Called when `revoke_memory` is invoked after child EOS. At that point all +// build data is represented as JoinSpillPartitionInfo entries in _spill_partition_queue +// (after queue initialization). The current partition being processed may have +// partially-recovered build data in _recovered_build_block. We repartition that +// data into FANOUT sub-partitions and push them back onto _spill_partition_queue +// so the hash table build can proceed later under a smaller memory footprint. +// +// Build data lives in either: +// (a) _current_partition.build_file (SpillFile, may have been partially read) +// (b) _recovered_build_block (partially-recovered MutableBlock) +// +// During repartition we route (b) directly into sub-streams first, then +// continue reading (a), avoiding an extra round of spill write/read for (b). +Status PartitionedHashJoinProbeLocalState::revoke_build_data(RuntimeState* state) { + auto& p = _parent->cast(); + DCHECK(_child_eos) << "revoke_build_data should only be called after child EOS"; + DCHECK(_spill_queue_initialized) << "queue must be initialized before revoke_build_data"; + + VLOG_DEBUG << fmt::format( + "Query:{}, hash join probe:{}, task:{}, revoke_build_data: " + "repartitioning queue partition at level {} (build in SpillFile)", + print_id(state->query_id()), p.node_id(), state->task_id(), _current_partition.level); + + RETURN_IF_ERROR(repartition_current_partition(state, _current_partition)); + + _current_partition = JoinSpillPartitionInfo {}; + _need_to_setup_queue_partition = true; + _queue_probe_blocks.clear(); - RETURN_IF_ERROR(local_state.spill_probe_blocks(state)); return Status::OK(); } -bool PartitionedHashJoinProbeOperatorX::_should_revoke_memory(RuntimeState* state) const { +// Public revoke_memory: called by the pipeline task scheduler when memory +// pressure requires this operator's in-memory data to be spilled. +// +// Before child EOS: probe blocks are still being accumulated → spill them. +// After child EOS: we are in the recovery/build phase. All build data is +// represented in _spill_partition_queue (after queue initialization). The +// current partition's in-memory recovered data (_recovered_build_block) is +// repartitioned and pushed back to the queue so the hash table build can +// proceed later with a smaller footprint. +Status PartitionedHashJoinProbeOperatorX::revoke_memory(RuntimeState* state) { auto& local_state = get_local_state(state); - if (local_state._shared_state->is_spilled) { - const auto revocable_size = _revocable_mem_size(state); + VLOG_DEBUG << fmt::format("Query:{}, hash join probe:{}, task:{}, revoke_memory, child_eos:{}", + print_id(state->query_id()), node_id(), state->task_id(), + local_state._child_eos); - if (local_state.low_memory_mode()) { - return revocable_size >= SpillStream::MIN_SPILL_WRITE_BATCH_MEM; - } else { - return revocable_size >= SpillStream::MAX_SPILL_WRITE_BATCH_MEM; - } + if (!local_state._child_eos) { + // Probe-data accumulation phase: spill in-memory probe blocks to disk. + return local_state.spill_probe_blocks(state, false); + } + if (!local_state._current_partition.is_valid() || + local_state._current_partition.build_finished) { + return Status::OK(); } - return false; + // Recovery/build phase: repartition the current partition's in-memory build + // data so the hash table build can be deferred to a smaller sub-partition. + return local_state.revoke_build_data(state); } Status PartitionedHashJoinProbeOperatorX::get_block(RuntimeState* state, Block* block, bool* eos) { *eos = false; auto& local_state = get_local_state(state); - local_state.copy_shared_spill_profile(); - const auto is_spilled = local_state._shared_state->is_spilled; -#ifndef NDEBUG - Defer eos_check_defer([&] { - if (*eos) { - LOG(INFO) << fmt::format( - "Query:{}, hash join probe:{}, task:{}, child eos:{}, need spill:{}", - print_id(state->query_id()), node_id(), state->task_id(), - local_state._child_eos, is_spilled); - } - }); -#endif - + const bool is_spilled = local_state._shared_state->_is_spilled; Defer defer([&]() { COUNTER_SET(local_state._memory_usage_reserved, int64_t(local_state.estimate_memory_usage())); @@ -877,13 +1023,10 @@ Status PartitionedHashJoinProbeOperatorX::get_block(RuntimeState* state, Block* Defer clear_defer([&] { local_state._child_block->clear_column_data(); }); if (is_spilled) { RETURN_IF_ERROR(push(state, local_state._child_block.get(), local_state._child_eos)); - if (_should_revoke_memory(state)) { - return _revoke_memory(state); - } } else { - DCHECK(local_state._shared_state->inner_runtime_state); + DCHECK(local_state._shared_state->_inner_runtime_state); RETURN_IF_ERROR(_inner_probe_operator->push( - local_state._shared_state->inner_runtime_state.get(), + local_state._shared_state->_inner_runtime_state.get(), local_state._child_block.get(), local_state._child_eos)); } } @@ -894,12 +1037,14 @@ Status PartitionedHashJoinProbeOperatorX::get_block(RuntimeState* state, Block* RETURN_IF_ERROR(pull(state, block, eos)); } else { RETURN_IF_ERROR(_inner_probe_operator->pull( - local_state._shared_state->inner_runtime_state.get(), block, eos)); + local_state._shared_state->_inner_runtime_state.get(), block, eos)); local_state.update_profile_from_inner(); } - local_state.add_num_rows_returned(block->rows()); - COUNTER_UPDATE(local_state._blocks_returned_counter, 1); + if (!block->empty()) { + local_state.add_num_rows_returned(block->rows()); + COUNTER_UPDATE(local_state._blocks_returned_counter, 1); + } } return Status::OK(); } diff --git a/be/src/exec/operator/partitioned_hash_join_probe_operator.h b/be/src/exec/operator/partitioned_hash_join_probe_operator.h index 10530d5c8ad283..2a53458e12983e 100644 --- a/be/src/exec/operator/partitioned_hash_join_probe_operator.h +++ b/be/src/exec/operator/partitioned_hash_join_probe_operator.h @@ -28,6 +28,10 @@ #include "exec/operator/operator.h" #include "exec/operator/spill_utils.h" #include "exec/pipeline/dependency.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" +#include "exec/spill/spill_repartitioner.h" namespace doris { #include "common/compile_check_begin.h" @@ -35,6 +39,51 @@ class RuntimeState; class PartitionedHashJoinProbeOperatorX; +/// Represents a spilled partition pair (build + probe file) that needs to be processed +/// during recovery. For multi-level spill, when a partition is too large to fit in +/// memory, it gets repartitioned into FANOUT sub-partitions, each represented by a +/// new JoinSpillPartitionInfo at level + 1. +/// +/// Lifecycle of partition progress: +/// build_file == nullptr: +/// - all build-side spill data has been read from disk for this partition +/// probe_file == nullptr: +/// - all probe-side spill data has been read from disk for this partition +/// build_finished = true: +/// - build side has completed hash table construction +/// probe_finished = true: +/// - probe side has completed probing all rows for this partition +/// +/// A default-constructed instance has is_valid() == false, representing "no partition". +/// New sub-partitions created by repartitioning start with both flags = false and +/// initialized = true. +struct JoinSpillPartitionInfo { + // build_file == nullptr means all build data has been read from disk. + SpillFileSPtr build_file; + // probe_file == nullptr means all probe data has been read from disk. + SpillFileSPtr probe_file; + int level = 0; // 0 = original level-0 partition, 1+ = repartitioned sub-partition + + // Read all build data from disk and finished building the hash table. + bool build_finished = false; + // Read all probe data from disk and probed all rows against the hash table. + bool probe_finished = false; + // Whether this struct currently represents an active queue partition. + bool initialized = false; + + JoinSpillPartitionInfo() = default; + JoinSpillPartitionInfo(SpillFileSPtr build, SpillFileSPtr probe, int lvl) + : build_file(std::move(build)), + probe_file(std::move(probe)), + level(lvl), + initialized(true) {} + + /// Returns true if this struct currently represents an active partition entry + /// from the spill queue. A default-constructed partition is "invalid" and + /// serves as a sentinel meaning "no partition is being processed". + bool is_valid() const { return initialized; } +}; + class PartitionedHashJoinProbeLocalState MOCK_REMOVE(final) : public PipelineXSpillLocalState { public: @@ -47,14 +96,28 @@ class PartitionedHashJoinProbeLocalState MOCK_REMOVE(final) Status open(RuntimeState* state) override; Status close(RuntimeState* state) override; - Status spill_probe_blocks(RuntimeState* state); - - Status recover_build_blocks_from_disk(RuntimeState* state, uint32_t partition_index, - bool& has_data); - Status recover_probe_blocks_from_disk(RuntimeState* state, uint32_t partition_index, - bool& has_data); - - Status finish_spilling(uint32_t partition_index); + Status spill_probe_blocks(RuntimeState* state, bool flush_all); + + /// Revoke in-memory build data by repartitioning it and pushing the result back onto + /// _spill_partition_queue. Used by revoke_memory when child_eos is true (recovery/build + /// phase) and we have significant in-memory build data that cannot be kept in memory. + /// + /// After queue initialization, all partitions are represented as JoinSpillPartitionInfo entries + /// in _spill_partition_queue. Repartition reads from _current_partition's streams (or the + /// already-recovered _recovered_build_block) and pushes FANOUT sub-partitions back onto the + /// queue. + Status revoke_build_data(RuntimeState* state); + + /// Recover build blocks from a JoinSpillPartitionInfo's build stream (for multi-level recovery). + Status recover_build_blocks_from_partition(RuntimeState* state, + JoinSpillPartitionInfo& partition_info); + /// Recover probe blocks from a JoinSpillPartitionInfo's probe stream (for multi-level recovery). + Status recover_probe_blocks_from_partition(RuntimeState* state, + JoinSpillPartitionInfo& partition_info); + + /// Repartition the current partition's build and probe streams into FANOUT sub-partitions + /// and push them into _spill_partition_queue for subsequent processing. + Status repartition_current_partition(RuntimeState* state, JoinSpillPartitionInfo& partition); template void update_build_custom_profile(RuntimeProfile* child_profile); @@ -76,47 +139,77 @@ class PartitionedHashJoinProbeLocalState MOCK_REMOVE(final) bool is_blockable() const override; + Status acquire_spill_writer(RuntimeState* state, int partition_index, + SpillFileWriterSPtr& writer); + friend class PartitionedHashJoinProbeOperatorX; private: template friend class StatefulOperatorX; - // Spill probe blocks to disk - Status _execute_spill_probe_blocks(RuntimeState* state, const UniqueId& query_id); - std::shared_ptr _in_mem_shared_state_sptr; - uint32_t _partition_cursor {0}; std::unique_ptr _child_block; bool _child_eos {false}; std::vector> _partitioned_blocks; std::unique_ptr _recovered_build_block; - std::map> _probe_blocks; - std::vector _probe_spilling_streams; + std::vector _probe_spilling_groups; + std::vector _probe_writers; std::unique_ptr _partitioner; std::unique_ptr _internal_runtime_profile; - bool _need_to_setup_internal_operators {true}; + // Persistent readers for recovery across scheduling slices + SpillFileReaderSPtr _current_build_reader; + SpillFileReaderSPtr _current_probe_reader; + + // ---- Spill partition queue state ---- + // Whether _spill_partition_queue has been initialized from spilled build groups + + // _probe_spilling_groups. Set to true the first time pull() enters the spill + // path after child EOS. Once true, all partitions are accessed via the queue. + bool _spill_queue_initialized {false}; + // Work queue of spilled partition pairs to process. Populated during + // initialization from the level-0 spilled streams and also when a partition is + // too large to build a hash table (repartitioned into FANOUT new entries). + std::deque _spill_partition_queue; + // The partition currently being processed from _spill_partition_queue. + JoinSpillPartitionInfo _current_partition; + // Repartitioner instance (reused across repartition calls) + SpillRepartitioner _repartitioner; + // A partitioner with partition_count = FANOUT for use during repartitioning. + // The main _partitioner uses the original _partition_count (e.g., 32), which + // is wrong for repartitioning that needs FANOUT (8) sub-partitions. + std::unique_ptr _fanout_partitioner; + std::unique_ptr _build_fanout_partitioner; + // Whether internal operators need to be set up for the current queue partition. + bool _need_to_setup_queue_partition {true}; + // Probe blocks recovered from the current queue partition's probe stream. + std::vector _queue_probe_blocks; - 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* _spill_build_timer = nullptr; RuntimeProfile::Counter* _recovery_build_rows = nullptr; + RuntimeProfile::Counter* _recovery_level0_build_rows = nullptr; RuntimeProfile::Counter* _recovery_build_blocks = nullptr; RuntimeProfile::Counter* _recovery_build_timer = nullptr; RuntimeProfile::Counter* _spill_probe_rows = nullptr; RuntimeProfile::Counter* _spill_probe_blocks = nullptr; RuntimeProfile::Counter* _spill_probe_timer = nullptr; + RuntimeProfile::Counter* _build_rows = nullptr; RuntimeProfile::Counter* _recovery_probe_rows = nullptr; RuntimeProfile::Counter* _recovery_probe_blocks = nullptr; RuntimeProfile::Counter* _recovery_probe_timer = nullptr; + // Counters to track spill partition metrics + RuntimeProfile::Counter* _max_partition_level = nullptr; + RuntimeProfile::Counter* _total_partition_spills = nullptr; + int _max_partition_level_seen = 0; + RuntimeProfile::Counter* _probe_blocks_bytes = nullptr; RuntimeProfile::Counter* _memory_usage_reserved = nullptr; RuntimeProfile::Counter* _get_child_next_timer = nullptr; @@ -126,7 +219,7 @@ class PartitionedHashJoinProbeOperatorX final : public JoinProbeOperatorX { public: PartitionedHashJoinProbeOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, - const DescriptorTbl& descs, uint32_t partition_count); + const DescriptorTbl& descs); Status init(const TPlanNode& tnode, RuntimeState* state) override; Status prepare(RuntimeState* state) override; @@ -150,6 +243,11 @@ class PartitionedHashJoinProbeOperatorX final size_t revocable_mem_size(RuntimeState* state) const override; + // Called by the pipeline task scheduler when memory pressure requires spilling + // probe-side blocks. Probe-side memory is NOT managed by the sink, so the + // probe operator must expose this interface so the scheduler can reach it. + Status revoke_memory(RuntimeState* state) override; + size_t get_reserve_mem_size(RuntimeState* state) override; void set_inner_operators(const std::shared_ptr& sink_operator, @@ -174,16 +272,19 @@ class PartitionedHashJoinProbeOperatorX final } private: - Status _revoke_memory(RuntimeState* state); - - size_t _revocable_mem_size(RuntimeState* state, bool force = false) const; - friend class PartitionedHashJoinProbeLocalState; - [[nodiscard]] Status _setup_internal_operators(PartitionedHashJoinProbeLocalState& local_state, - RuntimeState* state) const; + /// Setup internal operators using build data from a JoinSpillPartitionInfo + /// (for multi-level recovery, where build data comes from repartitioned streams). + [[nodiscard]] Status _setup_internal_operators_from_partition( + PartitionedHashJoinProbeLocalState& local_state, RuntimeState* state) const; - bool _should_revoke_memory(RuntimeState* state) const; + /// Process entries from the _spill_partition_queue. + /// All spilled partitions (both original level-0 and repartitioned sub-partitions) + /// are processed via this single path. + [[nodiscard]] Status _pull_from_spill_queue(PartitionedHashJoinProbeLocalState& local_state, + RuntimeState* state, Block* output_block, + bool* eos) const; const TJoinDistributionType::type _join_distribution; @@ -192,14 +293,17 @@ class PartitionedHashJoinProbeOperatorX final // probe expr std::vector _probe_exprs; + std::vector _build_exprs; const std::vector _distribution_partition_exprs; const TPlanNode _tnode; const DescriptorTbl _descriptor_tbl; - const uint32_t _partition_count; + uint32_t _partition_count; std::unique_ptr _partitioner; + // max repartition depth configured per-operator (default to static MAX_DEPTH) + int _repartition_max_depth = SpillRepartitioner::MAX_DEPTH; }; #include "common/compile_check_end.h" diff --git a/be/src/exec/operator/partitioned_hash_join_sink_operator.cpp b/be/src/exec/operator/partitioned_hash_join_sink_operator.cpp index a702421ae08640..c75db33799b395 100644 --- a/be/src/exec/operator/partitioned_hash_join_sink_operator.cpp +++ b/be/src/exec/operator/partitioned_hash_join_sink_operator.cpp @@ -27,8 +27,9 @@ #include "exec/operator/operator.h" #include "exec/operator/spill_utils.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_writer.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" #include "util/pretty_printer.h" @@ -42,16 +43,15 @@ Status PartitionedHashJoinSinkLocalState::init(doris::RuntimeState* state, SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); auto& p = _parent->cast(); - _shared_state->partitioned_build_blocks.resize(p._partition_count); - _shared_state->spilled_streams.resize(p._partition_count); + _shared_state->_partitioned_build_blocks.resize(p._partition_count); + _shared_state->_spilled_build_groups.resize(p._partition_count); + _build_writers.resize(p._partition_count); _rows_in_partitions.assign(p._partition_count, 0); _internal_runtime_profile = std::make_unique("internal_profile"); - _partition_timer = ADD_TIMER_WITH_LEVEL(custom_profile(), "SpillPartitionTime", 1); - _partition_shuffle_timer = - ADD_TIMER_WITH_LEVEL(custom_profile(), "SpillPartitionShuffleTime", 1); + _partition_shuffle_timer = ADD_TIMER_WITH_LEVEL(custom_profile(), "SpillRePartitionTime", 1); _spill_build_timer = ADD_TIMER_WITH_LEVEL(custom_profile(), "SpillBuildTime", 1); _in_mem_rows_counter = ADD_COUNTER_WITH_LEVEL(custom_profile(), "SpillInMemRow", TUnit::UNIT, 1); @@ -64,17 +64,8 @@ Status PartitionedHashJoinSinkLocalState::init(doris::RuntimeState* state, Status PartitionedHashJoinSinkLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - _shared_state->setup_shared_profile(custom_profile()); RETURN_IF_ERROR(PipelineXSpillSinkLocalState::open(state)); auto& p = _parent->cast(); - for (uint32_t i = 0; i != p._partition_count; ++i) { - auto& spilling_stream = _shared_state->spilled_streams[i]; - RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - state, spilling_stream, print_id(state->query_id()), - fmt::format("hash_build_sink_{}", i), _parent->node_id(), - std::numeric_limits::max(), std::numeric_limits::max(), - operator_profile())); - } return p._partitioner->clone(state, _partitioner); } @@ -84,23 +75,32 @@ Status PartitionedHashJoinSinkLocalState::close(RuntimeState* state, Status exec if (PipelineXSpillSinkLocalState::_closed) { return Status::OK(); } - DCHECK(_shared_state->inner_runtime_state != nullptr); + DCHECK(_shared_state->_inner_runtime_state != nullptr); VLOG_DEBUG << "Query:" << print_id(state->query_id()) << ", hash join sink:" << _parent->node_id() << ", task:" << state->task_id() << ", close"; auto& p = _parent->cast(); - if (!_shared_state->is_spilled && _shared_state->inner_runtime_state) { - RETURN_IF_ERROR(p._inner_sink_operator->close(_shared_state->inner_runtime_state.get(), + if (!_shared_state->_is_spilled && _shared_state->_inner_runtime_state) { + RETURN_IF_ERROR(p._inner_sink_operator->close(_shared_state->_inner_runtime_state.get(), exec_status)); } + + for (auto& writer : _build_writers) { + if (writer) { + RETURN_IF_ERROR(writer->close()); + writer.reset(); + } + } + _build_writers.clear(); + return PipelineXSpillSinkLocalState::close(state, exec_status); } size_t PartitionedHashJoinSinkLocalState::revocable_mem_size(RuntimeState* state) const { /// If no need to spill, all rows were sunk into the `_inner_sink_operator` without partitioned. - if (!_shared_state->is_spilled) { - if (_shared_state->inner_shared_state) { - auto* inner_sink_state_ = _shared_state->inner_runtime_state->get_sink_local_state(); + if (!_shared_state->_is_spilled) { + if (_shared_state->_inner_shared_state) { + auto* inner_sink_state_ = _shared_state->_inner_runtime_state->get_sink_local_state(); if (inner_sink_state_) { auto* inner_sink_state = assert_cast(inner_sink_state_); @@ -111,22 +111,22 @@ size_t PartitionedHashJoinSinkLocalState::revocable_mem_size(RuntimeState* state } size_t mem_size = 0; - auto& partitioned_blocks = _shared_state->partitioned_build_blocks; + auto& partitioned_blocks = _shared_state->_partitioned_build_blocks; for (auto& block : partitioned_blocks) { if (block) { auto block_bytes = block->allocated_bytes(); - if (block_bytes >= SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + if (block_bytes >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { mem_size += block_bytes; } } } - return mem_size; + return mem_size > state->spill_min_revocable_mem() ? mem_size : 0; } void PartitionedHashJoinSinkLocalState::update_memory_usage() { - if (!_shared_state->is_spilled) { - if (_shared_state->inner_shared_state) { - auto* inner_sink_state_ = _shared_state->inner_runtime_state->get_sink_local_state(); + if (!_shared_state->_is_spilled) { + if (_shared_state->_inner_shared_state) { + auto* inner_sink_state_ = _shared_state->_inner_runtime_state->get_sink_local_state(); if (inner_sink_state_) { auto* inner_sink_state = assert_cast(inner_sink_state_); @@ -137,7 +137,7 @@ void PartitionedHashJoinSinkLocalState::update_memory_usage() { } int64_t mem_size = 0; - auto& partitioned_blocks = _shared_state->partitioned_build_blocks; + auto& partitioned_blocks = _shared_state->_partitioned_build_blocks; for (auto& block : partitioned_blocks) { if (block) { mem_size += block->allocated_bytes(); @@ -149,12 +149,13 @@ void PartitionedHashJoinSinkLocalState::update_memory_usage() { size_t PartitionedHashJoinSinkLocalState::get_reserve_mem_size(RuntimeState* state, bool eos) { size_t size_to_reserve = 0; auto& p = _parent->cast(); - if (_shared_state->is_spilled) { - size_to_reserve = p._partition_count * SpillStream::MIN_SPILL_WRITE_BATCH_MEM; + + if (_shared_state->_is_spilled) { + size_to_reserve = p._partition_count * SpillFile::MIN_SPILL_WRITE_BATCH_MEM; } else { - if (_shared_state->inner_runtime_state) { + if (_shared_state->_inner_runtime_state) { size_to_reserve = p._inner_sink_operator->get_reserve_mem_size( - _shared_state->inner_runtime_state.get(), eos); + _shared_state->_inner_runtime_state.get(), eos); } } @@ -166,99 +167,15 @@ Dependency* PartitionedHashJoinSinkLocalState::finishdependency() { return _finish_dependency.get(); } -Status PartitionedHashJoinSinkLocalState::_execute_spill_unpartitioned_block(RuntimeState* state, - Block&& build_block) { - Defer defer1 {[&]() { update_memory_usage(); }}; - auto& p = _parent->cast(); - auto& partitioned_blocks = _shared_state->partitioned_build_blocks; - std::vector> partitions_indexes(p._partition_count); - - const size_t reserved_size = 4096; - std::ranges::for_each(partitions_indexes, - [](std::vector& indices) { indices.reserve(reserved_size); }); - - size_t total_rows = build_block.rows(); - size_t offset = 1; - while (offset < total_rows) { - auto sub_block = build_block.clone_empty(); - size_t this_run = std::min(reserved_size, total_rows - offset); - - for (size_t i = 0; i != build_block.columns(); ++i) { - sub_block.get_by_position(i).column = - build_block.get_by_position(i).column->cut(offset, this_run); - } - int64_t sub_blocks_memory_usage = sub_block.allocated_bytes(); - COUNTER_UPDATE(_memory_used_counter, sub_blocks_memory_usage); - Defer defer2 {[&]() { COUNTER_UPDATE(_memory_used_counter, -sub_blocks_memory_usage); }}; - - offset += this_run; - const auto is_last_block = offset == total_rows; - - { - SCOPED_TIMER(_partition_timer); - (void)_partitioner->do_partitioning(state, &sub_block); - } - - const auto& channel_ids = _partitioner->get_channel_ids(); - for (size_t i = 0; i != sub_block.rows(); ++i) { - partitions_indexes[channel_ids[i]].emplace_back(i); - } - - for (uint32_t partition_idx = 0; partition_idx != p._partition_count; ++partition_idx) { - auto* begin = partitions_indexes[partition_idx].data(); - auto* end = begin + partitions_indexes[partition_idx].size(); - auto& partition_block = partitioned_blocks[partition_idx]; - SpillStreamSPtr& spilling_stream = _shared_state->spilled_streams[partition_idx]; - if (UNLIKELY(!partition_block)) { - partition_block = MutableBlock::create_unique(build_block.clone_empty()); - } - - int64_t old_mem = partition_block->allocated_bytes(); - { - SCOPED_TIMER(_partition_shuffle_timer); - RETURN_IF_ERROR(partition_block->add_rows(&sub_block, begin, end)); - partitions_indexes[partition_idx].clear(); - } - int64_t new_mem = partition_block->allocated_bytes(); - - if (partition_block->rows() >= reserved_size || is_last_block) { - auto block = partition_block->to_block(); - RETURN_IF_ERROR(spilling_stream->spill_block(state, block, false)); - partition_block = MutableBlock::create_unique(build_block.clone_empty()); - COUNTER_UPDATE(_memory_used_counter, -new_mem); - } else { - COUNTER_UPDATE(_memory_used_counter, new_mem - old_mem); - } - } - } - - Status status; - if (_child_eos) { - std::ranges::for_each(_shared_state->partitioned_build_blocks, [&](auto& block) { - if (block) { - COUNTER_UPDATE(_in_mem_rows_counter, block->rows()); - } - }); - status = _finish_spilling(); - VLOG_DEBUG << fmt::format( - "Query:{}, hash join sink:{}, task:{}, _revoke_unpartitioned_block, " - "set_ready_to_read", - print_id(state->query_id()), _parent->node_id(), state->task_id()); - _dependency->set_ready_to_read(); - } - - return status; -} - -Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block( - RuntimeState* state, const std::shared_ptr& spill_context) { +Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block(RuntimeState* state) { + // SpillFiles and writers will be created lazily in _spill_to_disk auto& p = _parent->cast(); HashJoinBuildSinkLocalState* inner_sink_state {nullptr}; - if (auto* tmp_sink_state = _shared_state->inner_runtime_state->get_sink_local_state()) { + if (auto* tmp_sink_state = _shared_state->_inner_runtime_state->get_sink_local_state()) { inner_sink_state = assert_cast(tmp_sink_state); } - DCHECK_EQ(_shared_state->inner_shared_state->hash_table_variant_vector.size(), 1); - _shared_state->inner_shared_state->hash_table_variant_vector.front().reset(); + DCHECK_EQ(_shared_state->_inner_shared_state->hash_table_variant_vector.size(), 1); + _shared_state->_inner_shared_state->hash_table_variant_vector.front().reset(); if (inner_sink_state) { COUNTER_UPDATE(_memory_used_counter, -(inner_sink_state->_hash_table_memory_usage->value() + @@ -274,7 +191,7 @@ Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block( // If spilling was triggered, constructing runtime filters is meaningless, // therefore, all runtime filters are temporarily disabled. RETURN_IF_ERROR(inner_sink_state->_runtime_filter_producer_helper->skip_process( - _shared_state->inner_runtime_state.get())); + _shared_state->_inner_runtime_state.get())); _finish_dependency->set_ready(); } @@ -283,9 +200,6 @@ Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block( "Query:{}, hash join sink:{}, task:{}," " has no data to revoke", print_id(state->query_id()), _parent->node_id(), state->task_id()); - if (spill_context) { - spill_context->on_task_finished(); - } return Status::OK(); } @@ -294,24 +208,28 @@ Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block( COUNTER_UPDATE(_memory_used_counter, build_block.allocated_bytes() - block_old_mem); } - auto exception_catch_func = [this, state, build_block = std::move(build_block)]() mutable { - auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION( - return _execute_spill_unpartitioned_block(state, std::move(build_block))); - }(); - return status; - }; - - SpillSinkRunnable spill_runnable(state, spill_context, operator_profile(), - exception_catch_func); - - DBUG_EXECUTE_IF( - "fault_inject::partitioned_hash_join_sink::revoke_unpartitioned_block_submit_func", { - return Status::Error( - "fault_inject partitioned_hash_join_sink " - "revoke_unpartitioned_block submit_func failed"); - }); - return spill_runnable.run(); + // The inner sink's _build_side_mutable_block has a sentinel row at + // index 0 (used for column type evaluation), so real data starts at + // row 1. Split the big block into sub-blocks and reuse the normal + // _partition_block + _execute_spill_partitioned_blocks path to avoid + // duplicating the partitioning logic. + // make the batch size larger, because after partitioning, the + // number of rows per partition is much smaller. To avoid too much small blocks. + const size_t batch_size = 4096 * _shared_state->_partitioned_build_blocks.size(); + const size_t total_rows = build_block.rows(); + for (size_t offset = 1; offset < total_rows;) { + const size_t this_run = std::min(batch_size, total_rows - offset); + auto sub_block = build_block.clone_empty(); + for (size_t c = 0; c != build_block.columns(); ++c) { + sub_block.get_by_position(c).column = + build_block.get_by_position(c).column->cut(offset, this_run); + } + offset += this_run; + RETURN_IF_ERROR(_partition_block(state, &sub_block, 0, sub_block.rows())); + RETURN_IF_ERROR(_execute_spill_partitioned_blocks(state, true /*force_spill*/)); + } + RETURN_IF_ERROR(_force_flush_partitions(state)); + return Status::OK(); } Status PartitionedHashJoinSinkLocalState::terminate(RuntimeState* state) { @@ -319,42 +237,60 @@ Status PartitionedHashJoinSinkLocalState::terminate(RuntimeState* state) { return Status::OK(); } HashJoinBuildSinkLocalState* inner_sink_state {nullptr}; - if (auto* tmp_sink_state = _shared_state->inner_runtime_state->get_sink_local_state()) { + if (auto* tmp_sink_state = _shared_state->_inner_runtime_state->get_sink_local_state()) { inner_sink_state = assert_cast(tmp_sink_state); } - if (_parent->cast()._inner_sink_operator) { - RETURN_IF_ERROR(inner_sink_state->_runtime_filter_producer_helper->skip_process(state)); + if (inner_sink_state) { + if (_parent->cast()._inner_sink_operator) { + RETURN_IF_ERROR(inner_sink_state->_runtime_filter_producer_helper->skip_process(state)); + } + inner_sink_state->_terminated = true; } - inner_sink_state->_terminated = true; return PipelineXSpillSinkLocalState::terminate(state); } -Status PartitionedHashJoinSinkLocalState::_finish_spilling_callback( - RuntimeState* state, TUniqueId query_id, - const std::shared_ptr& spill_context) { - Status status; - if (_child_eos) { - LOG(INFO) << fmt::format( - "Query:{}, hash join sink:{}, task:{}, finish spilling, set_ready_to_read", - print_id(query_id), _parent->node_id(), state->task_id()); - std::ranges::for_each(_shared_state->partitioned_build_blocks, [&](auto& block) { - if (block) { - COUNTER_UPDATE(_in_mem_rows_counter, block->rows()); +Status PartitionedHashJoinSinkLocalState::_force_flush_partitions(RuntimeState* state) { + for (size_t i = 0; i != _shared_state->_partitioned_build_blocks.size(); ++i) { + auto& block = _shared_state->_partitioned_build_blocks[i]; + if (block && block->rows() > 0) { + auto spilled_block = block->to_block(); + block.reset(); + // Lazy-create SpillFile + Writer if needed + auto& spill_file = _shared_state->_spilled_build_groups[i]; + auto& writer = _build_writers[i]; + if (!writer) { + auto label = fmt::format("hash_build_sink_{}", i); + auto relative_path = fmt::format( + "{}/{}-{}-{}-{}", print_id(state->query_id()), label, _parent->node_id(), + state->task_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + relative_path, spill_file)); + RETURN_IF_ERROR(spill_file->create_writer(state, operator_profile(), writer)); } - }); - status = _finish_spilling(); - _dependency->set_ready_to_read(); + RETURN_IF_ERROR(writer->write_block(state, spilled_block)); + } } + return Status::OK(); +} - if (spill_context) { - spill_context->on_task_finished(); +Status PartitionedHashJoinSinkLocalState::_finish_spilling(RuntimeState* state) { + if (!_shared_state->_is_spilled) { + return Status::OK(); } - - return status; + // Close all writers (Writer::close() finalizes SpillFile metadata) + for (auto& writer : _build_writers) { + if (writer) { + RETURN_IF_ERROR(writer->close()); + } + } + return Status::OK(); } +/// If revoke memory API call this method, we has to flush all memory to avoid dead loop. For example, maybe +/// revocable memory size calcuateld by memory usage is not enough using limit 100K, but we can't spill all memory to disk +/// because we use limit 1MB here. So we need to force spill all memory to disk to make sure we can make progress. Status PartitionedHashJoinSinkLocalState::_execute_spill_partitioned_blocks(RuntimeState* state, - TUniqueId query_id) { + bool force_spill) { DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_sink::revoke_memory_cancel", { auto status = Status::InternalError( "fault_inject partitioned_hash_join_sink revoke_memory canceled"); @@ -363,56 +299,34 @@ Status PartitionedHashJoinSinkLocalState::_execute_spill_partitioned_blocks(Runt }); SCOPED_TIMER(_spill_build_timer); - for (size_t i = 0; i != _shared_state->partitioned_build_blocks.size(); ++i) { - SpillStreamSPtr& spilling_stream = _shared_state->spilled_streams[i]; - DCHECK(spilling_stream != nullptr); - auto& mutable_block = _shared_state->partitioned_build_blocks[i]; - - if (!mutable_block || - mutable_block->allocated_bytes() < SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + for (size_t i = 0; i != _shared_state->_partitioned_build_blocks.size(); ++i) { + auto& mutable_block = _shared_state->_partitioned_build_blocks[i]; + if (!mutable_block) { continue; } - - auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION( - return _spill_to_disk(static_cast(i), spilling_stream)); - }(); - - RETURN_IF_ERROR(status); + if (force_spill || mutable_block->allocated_bytes() >= state->spill_buffer_size_bytes()) { + RETURN_IF_ERROR(_spill_to_disk(static_cast(i))); + } } return Status::OK(); } -Status PartitionedHashJoinSinkLocalState::revoke_memory( - RuntimeState* state, const std::shared_ptr& spill_context) { - SCOPED_TIMER(_spill_total_timer); +Status PartitionedHashJoinSinkLocalState::revoke_memory(RuntimeState* state) { VLOG_DEBUG << fmt::format("Query:{}, hash join sink:{}, task:{}, revoke_memory, eos:{}", print_id(state->query_id()), _parent->node_id(), state->task_id(), _child_eos); - if (!_shared_state->is_spilled) { + if (!_shared_state->_is_spilled) { custom_profile()->add_info_string("Spilled", "true"); - _shared_state->is_spilled = true; - return _revoke_unpartitioned_block(state, spill_context); - } - - const auto query_id = state->query_id(); - SpillSinkRunnable spill_runnable( - state, nullptr, operator_profile(), - [this, state, query_id] { return _execute_spill_partitioned_blocks(state, query_id); }, - [this, state, query_id, spill_context]() { - return _finish_spilling_callback(state, query_id, spill_context); - }); - - return spill_runnable.run(); -} - -Status PartitionedHashJoinSinkLocalState::_finish_spilling() { - for (auto& stream : _shared_state->spilled_streams) { - if (stream) { - RETURN_IF_ERROR(stream->spill_eof()); - } - } + _shared_state->_is_spilled = true; + Status st = _revoke_unpartitioned_block(state); + DCHECK(revocable_mem_size(state) == 0); + return st; + } + RETURN_IF_ERROR(_execute_spill_partitioned_blocks(state, true /*force_spill*/)); + // force flush all partitions to make sure data is written to disk + RETURN_IF_ERROR(_force_flush_partitions(state)); + DCHECK(revocable_mem_size(state) == 0); return Status::OK(); } @@ -422,15 +336,12 @@ Status PartitionedHashJoinSinkLocalState::_partition_block(RuntimeState* state, if (!rows) { return Status::OK(); } + + SCOPED_TIMER(_partition_shuffle_timer); Defer defer {[&]() { update_memory_usage(); }}; - { - /// TODO: DO NOT execute build exprs twice(when partition and building hash table) - SCOPED_TIMER(_partition_timer); - RETURN_IF_ERROR(_partitioner->do_partitioning(state, in_block)); - } + RETURN_IF_ERROR(_partitioner->do_partitioning(state, in_block)); auto& p = _parent->cast(); - SCOPED_TIMER(_partition_shuffle_timer); const auto& channel_ids = _partitioner->get_channel_ids(); std::vector> partition_indexes(p._partition_count); DCHECK_LT(begin, end); @@ -438,7 +349,7 @@ Status PartitionedHashJoinSinkLocalState::_partition_block(RuntimeState* state, partition_indexes[channel_ids[i]].emplace_back(i); } - auto& partitioned_blocks = _shared_state->partitioned_build_blocks; + auto& partitioned_blocks = _shared_state->_partitioned_build_blocks; for (uint32_t i = 0; i != p._partition_count; ++i) { const auto count = partition_indexes[i].size(); if (UNLIKELY(count == 0)) { @@ -458,26 +369,33 @@ Status PartitionedHashJoinSinkLocalState::_partition_block(RuntimeState* state, return Status::OK(); } -Status PartitionedHashJoinSinkLocalState::_spill_to_disk(uint32_t partition_index, - const SpillStreamSPtr& spilling_stream) { - auto& partitioned_block = _shared_state->partitioned_build_blocks[partition_index]; - - if (!_state->is_cancelled()) { - auto block = partitioned_block->to_block(); - int64_t block_mem_usage = block.allocated_bytes(); - Defer defer {[&]() { COUNTER_UPDATE(memory_used_counter(), -block_mem_usage); }}; - partitioned_block = MutableBlock::create_unique(block.clone_empty()); - return spilling_stream->spill_block(state(), block, false); - } else { - return _state->cancel_reason(); - } +Status PartitionedHashJoinSinkLocalState::_spill_to_disk(uint32_t partition_index) { + auto& partitioned_block = _shared_state->_partitioned_build_blocks[partition_index]; + + auto block = partitioned_block->to_block(); + int64_t block_mem_usage = block.allocated_bytes(); + Defer defer {[&]() { COUNTER_UPDATE(memory_used_counter(), -block_mem_usage); }}; + partitioned_block = MutableBlock::create_unique(block.clone_empty()); + + // Lazy-create SpillFile + Writer if needed + auto& spill_file = _shared_state->_spilled_build_groups[partition_index]; + auto& writer = _build_writers[partition_index]; + if (!writer) { + auto label = fmt::format("hash_build_sink_{}", partition_index); + auto relative_path = fmt::format("{}/{}-{}-{}-{}", print_id(state()->query_id()), label, + _parent->node_id(), state()->task_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + spill_file)); + RETURN_IF_ERROR(spill_file->create_writer(state(), operator_profile(), writer)); + } + return writer->write_block(state(), block); } PartitionedHashJoinSinkOperatorX::PartitionedHashJoinSinkOperatorX(ObjectPool* pool, int operator_id, int dest_id, const TPlanNode& tnode, - const DescriptorTbl& descs, - uint32_t partition_count) + const DescriptorTbl& descs) : JoinBuildSinkOperatorX(pool, operator_id, dest_id, tnode, descs), _join_distribution(tnode.hash_join_node.__isset.dist_type ? tnode.hash_join_node.dist_type @@ -486,12 +404,12 @@ PartitionedHashJoinSinkOperatorX::PartitionedHashJoinSinkOperatorX(ObjectPool* p ? tnode.distribute_expr_lists[1] : std::vector {}), _tnode(tnode), - _descriptor_tbl(descs), - _partition_count(partition_count) { + _descriptor_tbl(descs) { _spillable = true; } Status PartitionedHashJoinSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { + _partition_count = state->spill_hash_join_partition_count(); 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; @@ -558,8 +476,8 @@ Status PartitionedHashJoinSinkLocalState::_setup_internal_operator(RuntimeState* _finish_dependency = sink_local_state->finishdependency()->shared_from_this(); /// Set these two values after all the work is ready. - _shared_state->inner_shared_state = std::move(inner_shared_state); - _shared_state->inner_runtime_state = std::move(inner_runtime_state); + _shared_state->_inner_shared_state = std::move(inner_shared_state); + _shared_state->_inner_runtime_state = std::move(inner_runtime_state); return Status::OK(); } @@ -567,7 +485,7 @@ Status PartitionedHashJoinSinkLocalState::_setup_internal_operator(RuntimeState* update_profile_from_inner_profile(name, custom_profile(), inner_profile) void PartitionedHashJoinSinkLocalState::update_profile_from_inner() { - auto* sink_local_state = _shared_state->inner_runtime_state->get_sink_local_state(); + auto* sink_local_state = _shared_state->_inner_runtime_state->get_sink_local_state(); if (sink_local_state) { auto* inner_sink_state = assert_cast(sink_local_state); auto* inner_profile = inner_sink_state->custom_profile(); @@ -583,138 +501,66 @@ void PartitionedHashJoinSinkLocalState::update_profile_from_inner() { #undef UPDATE_COUNTER_FROM_INNER -// After building hash table it will not be able to spill later -// even if memory is low, and will cause cancel of queries. -// So make a check here, if build blocks mem usage is too high, -// then trigger revoke memory. -static bool is_revocable_mem_high_watermark(RuntimeState* state, size_t revocable_size, - int64_t query_mem_limit) { - auto revocable_memory_high_watermark_percent = - state->spill_revocable_memory_high_watermark_percent(); - return revocable_memory_high_watermark_percent > 0 && - static_cast(revocable_size) >= - (double)query_mem_limit / 100.0 * revocable_memory_high_watermark_percent; -} - Status PartitionedHashJoinSinkOperatorX::sink(RuntimeState* state, Block* in_block, bool eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); - - local_state._child_eos = eos; - const auto rows = in_block->rows(); + if (rows > 0) { + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)rows); + } - const auto is_spilled = local_state._shared_state->is_spilled; - size_t revocable_size = 0; - int64_t query_mem_limit = 0; - if (eos) { - revocable_size = revocable_mem_size(state); - query_mem_limit = state->get_query_ctx()->resource_ctx()->memory_context()->mem_limit(); - LOG(INFO) << fmt::format( - "Query:{}, hash join sink:{}, task:{}, eos, need spill:{}, query mem limit:{}, " - "revocable memory:{}", - print_id(state->query_id()), node_id(), state->task_id(), is_spilled, - PrettyPrinter::print_bytes(query_mem_limit), - PrettyPrinter::print_bytes(revocable_size)); - } - - if (rows == 0) { - if (eos) { - if (is_spilled) { - return revoke_memory(state, nullptr); - } else { - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_sink::sink_eos", { - return Status::Error( - "fault_inject partitioned_hash_join_sink " - "sink_eos failed"); - }); - - if (is_revocable_mem_high_watermark(state, revocable_size, query_mem_limit)) { - LOG(INFO) << fmt::format( - "Query:{}, hash join sink:{}, task:{} eos, revoke_memory " - "because revocable memory is high", - print_id(state->query_id()), node_id(), state->task_id()); - return revoke_memory(state, nullptr); - } - - Defer defer {[&]() { local_state.update_memory_usage(); }}; - RETURN_IF_ERROR(_inner_sink_operator->sink( - local_state._shared_state->inner_runtime_state.get(), in_block, eos)); - - local_state.update_profile_from_inner(); - - LOG(INFO) << fmt::format( - "Query:{}, hash join sink:{}, task:{}, eos, set_ready_to_read, nonspill " - "memory usage:{}", - print_id(state->query_id()), node_id(), state->task_id(), - _inner_sink_operator->get_memory_usage_debug_str( - local_state._shared_state->inner_runtime_state.get())); - } - - std::ranges::for_each( - local_state._shared_state->partitioned_build_blocks, [&](auto& block) { - if (block) { - COUNTER_UPDATE(local_state._in_mem_rows_counter, block->rows()); - } - }); - local_state._dependency->set_ready_to_read(); + if (local_state._shared_state->_is_spilled) { + // ---- Spilled path: data is partitioned and spilled to disk ---- + if (rows > 0) { + RETURN_IF_ERROR(local_state._partition_block(state, in_block, 0, rows)); + // If any partition block exceeds the spill buffer size, immediately spill that partition to disk to avoid large block accumulation. + RETURN_IF_ERROR( + local_state._execute_spill_partitioned_blocks(state, false /*force_spill*/)); } - return Status::OK(); - } - COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); - if (is_spilled) { - RETURN_IF_ERROR(local_state._partition_block(state, in_block, 0, rows)); - if (eos) { - return revoke_memory(state, nullptr); - } else if (revocable_mem_size(state) > SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { - return revoke_memory(state, nullptr); + // Flush partitioned blocks when eos or when accumulated data is large enough. + if (revocable_mem_size(state) > state->spill_join_build_sink_mem_limit_bytes()) { + RETURN_IF_ERROR(revoke_memory(state)); } } else { + // ---- Non-spill path: forward data to the inner hash join sink ---- DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_sink::sink", { return Status::Error( - "fault_inject partitioned_hash_join_sink " - "sink failed"); + "fault_inject partitioned_hash_join_sink sink failed"); }); - if (eos) { - if (is_revocable_mem_high_watermark(state, revocable_size, query_mem_limit)) { - LOG(INFO) << fmt::format( - "Query:{}, hash join sink:{}, task:{}, eos, revoke_memory " - "because revocable memory is high", - print_id(state->query_id()), node_id(), state->task_id()); - return revoke_memory(state, nullptr); - } - } + // Sink the block into the inner (non-partitioned) hash join build operator. RETURN_IF_ERROR(_inner_sink_operator->sink( - local_state._shared_state->inner_runtime_state.get(), in_block, eos)); + local_state._shared_state->_inner_runtime_state.get(), in_block, eos)); local_state.update_memory_usage(); local_state.update_profile_from_inner(); - if (eos) { - LOG(INFO) << fmt::format( - "Query:{}, hash join sink:{}, task:{}, eos, set_ready_to_read, nonspill memory " - "usage:{}", - print_id(state->query_id()), node_id(), state->task_id(), - _inner_sink_operator->get_memory_usage_debug_str( - local_state._shared_state->inner_runtime_state.get())); - local_state._dependency->set_ready_to_read(); - } } + if (eos) { + local_state._child_eos = true; + if (local_state._shared_state->_is_spilled) { + RETURN_IF_ERROR(local_state._force_flush_partitions(state)); + RETURN_IF_ERROR(local_state._finish_spilling(state)); + } + 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); + if (local_state._child_eos) { + return 0; + } SCOPED_TIMER(local_state.exec_time_counter()); return local_state.revocable_mem_size(state); } -Status PartitionedHashJoinSinkOperatorX::revoke_memory( - RuntimeState* state, const std::shared_ptr& spill_context) { +Status PartitionedHashJoinSinkOperatorX::revoke_memory(RuntimeState* state) { auto& local_state = get_local_state(state); + CHECK(!local_state._child_eos); SCOPED_TIMER(local_state.exec_time_counter()); - return local_state.revoke_memory(state, spill_context); + return local_state.revoke_memory(state); } size_t PartitionedHashJoinSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { @@ -723,7 +569,7 @@ size_t PartitionedHashJoinSinkOperatorX::get_reserve_mem_size(RuntimeState* stat } bool PartitionedHashJoinSinkLocalState::is_blockable() const { - return _shared_state->is_spilled; + return _shared_state->_is_spilled; } #include "common/compile_check_end.h" diff --git a/be/src/exec/operator/partitioned_hash_join_sink_operator.h b/be/src/exec/operator/partitioned_hash_join_sink_operator.h index cd355ef1b5f68c..c4ffeb0ce44078 100644 --- a/be/src/exec/operator/partitioned_hash_join_sink_operator.h +++ b/be/src/exec/operator/partitioned_hash_join_sink_operator.h @@ -26,9 +26,10 @@ #include "exec/operator/hashjoin_probe_operator.h" #include "exec/operator/join_build_sink_operator.h" #include "exec/operator/operator.h" -#include "exec/operator/spill_utils.h" #include "exec/partitioner/partitioner.h" #include "exec/pipeline/dependency.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_writer.h" namespace doris { #include "common/compile_check_begin.h" @@ -45,7 +46,7 @@ class PartitionedHashJoinSinkLocalState Status init(RuntimeState* state, LocalSinkStateInfo& info) override; Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; - Status revoke_memory(RuntimeState* state, const std::shared_ptr& spill_context); + Status revoke_memory(RuntimeState* state); size_t revocable_mem_size(RuntimeState* state) const; Status terminate(RuntimeState* state) override; [[nodiscard]] size_t get_reserve_mem_size(RuntimeState* state, bool eos); @@ -60,21 +61,26 @@ class PartitionedHashJoinSinkLocalState PartitionedHashJoinSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : PipelineXSpillSinkLocalState(parent, state) {} - Status _spill_to_disk(uint32_t partition_index, const SpillStreamSPtr& spilling_stream); + Status _spill_to_disk(uint32_t partition_index); Status _partition_block(RuntimeState* state, Block* in_block, size_t begin, size_t end); - Status _revoke_unpartitioned_block(RuntimeState* state, - const std::shared_ptr& spill_context); + Status _revoke_unpartitioned_block(RuntimeState* state); - Status _execute_spill_unpartitioned_block(RuntimeState* state, Block&& build_block); + Status _finish_spilling(RuntimeState* state); + // Flush any remaining partitioned in-memory blocks and close spill streams. + // Called after revoke operations to guarantee memory is cleared. + Status _force_flush_partitions(RuntimeState* state); - Status _finish_spilling(); - - Status _finish_spilling_callback(RuntimeState* state, TUniqueId query_id, - const std::shared_ptr& spill_context); - - Status _execute_spill_partitioned_blocks(RuntimeState* state, TUniqueId query_id); + /** + * @brief Spill partitioned build blocks to disk if needed. + * + * @param state Runtime state for the operator. + * @param force_spill If true, spill all non-empty partition blocks regardless of size; + * if false, only spill blocks whose size exceeds the spill buffer threshold. + * Use force_spill at call sites for clarity. + */ + Status _execute_spill_partitioned_blocks(RuntimeState* state, bool force_spill); Status _setup_internal_operator(RuntimeState* state); @@ -87,19 +93,19 @@ class PartitionedHashJoinSinkLocalState std::unique_ptr _internal_runtime_profile; std::shared_ptr _finish_dependency; - RuntimeProfile::Counter* _partition_timer = nullptr; RuntimeProfile::Counter* _partition_shuffle_timer = nullptr; RuntimeProfile::Counter* _spill_build_timer = nullptr; RuntimeProfile::Counter* _in_mem_rows_counter = nullptr; RuntimeProfile::Counter* _memory_usage_reserved = nullptr; + + std::vector _build_writers; }; class PartitionedHashJoinSinkOperatorX : public JoinBuildSinkOperatorX { public: PartitionedHashJoinSinkOperatorX(ObjectPool* pool, int operator_id, int dest_id, - const TPlanNode& tnode, const DescriptorTbl& descs, - uint32_t partition_count); + const TPlanNode& tnode, const DescriptorTbl& descs); Status init(const TDataSink& tsink) override { return Status::InternalError("{} should not init with TDataSink", @@ -116,8 +122,7 @@ class PartitionedHashJoinSinkOperatorX size_t revocable_mem_size(RuntimeState* state) const override; - Status revoke_memory(RuntimeState* state, - const std::shared_ptr& spill_context) override; + Status revoke_memory(RuntimeState* state) override; size_t get_reserve_mem_size(RuntimeState* state, bool eos) override; @@ -172,7 +177,8 @@ class PartitionedHashJoinSinkOperatorX const std::vector _distribution_partition_exprs; const TPlanNode _tnode; const DescriptorTbl _descriptor_tbl; - const uint32_t _partition_count; + + uint32_t _partition_count; std::unique_ptr _partitioner; }; diff --git a/be/src/exec/operator/spill_iceberg_table_sink_operator.cpp b/be/src/exec/operator/spill_iceberg_table_sink_operator.cpp index 0a5126d2bef740..c557ac58e60e5e 100644 --- a/be/src/exec/operator/spill_iceberg_table_sink_operator.cpp +++ b/be/src/exec/operator/spill_iceberg_table_sink_operator.cpp @@ -79,21 +79,14 @@ size_t SpillIcebergTableSinkLocalState::get_revocable_mem_size(RuntimeState* sta return sort_writer->sorter()->data_size(); } -Status SpillIcebergTableSinkLocalState::revoke_memory( - RuntimeState* state, const std::shared_ptr& spill_context) { +Status SpillIcebergTableSinkLocalState::revoke_memory(RuntimeState* state) { if (!_writer || !_writer->current_writer()) { - if (spill_context) { - spill_context->on_task_finished(); - } return Status::OK(); } auto* sort_writer = dynamic_cast(_writer->current_writer().get()); if (!sort_writer || !sort_writer->sorter()) { - if (spill_context) { - spill_context->on_task_finished(); - } return Status::OK(); } @@ -104,13 +97,7 @@ Status SpillIcebergTableSinkLocalState::revoke_memory( return status; }; - state->get_query_ctx()->resource_ctx()->task_controller()->increase_revoking_tasks_count(); - auto status = - SpillSinkRunnable(state, spill_context, operator_profile(), exception_catch_func).run(); - if (!status.ok()) { - state->get_query_ctx()->resource_ctx()->task_controller()->decrease_revoking_tasks_count(); - } - return status; + return run_spill_task(state, exception_catch_func); } SpillIcebergTableSinkOperatorX::SpillIcebergTableSinkOperatorX( @@ -150,10 +137,9 @@ size_t SpillIcebergTableSinkOperatorX::revocable_mem_size(RuntimeState* state) c return local_state.get_revocable_mem_size(state); } -Status SpillIcebergTableSinkOperatorX::revoke_memory( - RuntimeState* state, const std::shared_ptr& spill_context) { +Status SpillIcebergTableSinkOperatorX::revoke_memory(RuntimeState* state) { auto& local_state = get_local_state(state); - return local_state.revoke_memory(state, spill_context); + return local_state.revoke_memory(state); } void SpillIcebergTableSinkLocalState::_init_spill_counters() { diff --git a/be/src/exec/operator/spill_iceberg_table_sink_operator.h b/be/src/exec/operator/spill_iceberg_table_sink_operator.h index a31a017f28fd1b..d81e515f05d59a 100644 --- a/be/src/exec/operator/spill_iceberg_table_sink_operator.h +++ b/be/src/exec/operator/spill_iceberg_table_sink_operator.h @@ -43,7 +43,7 @@ class SpillIcebergTableSinkLocalState final bool is_blockable() const override; [[nodiscard]] size_t get_reserve_mem_size(RuntimeState* state, bool eos); - Status revoke_memory(RuntimeState* state, const std::shared_ptr& spill_context); + Status revoke_memory(RuntimeState* state); size_t get_revocable_mem_size(RuntimeState* state) const; private: @@ -70,8 +70,7 @@ class SpillIcebergTableSinkOperatorX final size_t revocable_mem_size(RuntimeState* state) const override; - Status revoke_memory(RuntimeState* state, - const std::shared_ptr& spill_context) override; + Status revoke_memory(RuntimeState* state) override; using DataSinkOperatorX::node_id; using DataSinkOperatorX::operator_id; diff --git a/be/src/exec/operator/spill_sort_sink_operator.cpp b/be/src/exec/operator/spill_sort_sink_operator.cpp index 396e069a408a04..3baec697de215b 100644 --- a/be/src/exec/operator/spill_sort_sink_operator.cpp +++ b/be/src/exec/operator/spill_sort_sink_operator.cpp @@ -21,7 +21,9 @@ #include "exec/operator/sort_sink_operator.h" #include "exec/operator/spill_utils.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_writer.h" #include "runtime/fragment_mgr.h" namespace doris { @@ -44,7 +46,6 @@ Status SpillSortSinkLocalState::init(doris::RuntimeState* state, doris::LocalSin Status SpillSortSinkLocalState::open(RuntimeState* state) { SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_open_timer); - _shared_state->setup_shared_profile(custom_profile()); return Base::open(state); } @@ -64,6 +65,11 @@ void SpillSortSinkLocalState::update_profile(RuntimeProfile* child_profile) { #undef UPDATE_PROFILE Status SpillSortSinkLocalState::close(RuntimeState* state, Status execsink_status) { + if (_spilling_writer) { + RETURN_IF_ERROR(_spilling_writer->close()); + _spilling_writer.reset(); + } + _spilling_file.reset(); return Base::close(state, execsink_status); } @@ -127,15 +133,15 @@ size_t SpillSortSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eo auto& local_state = get_local_state(state); return local_state.get_reserve_mem_size(state, eos); } -Status SpillSortSinkOperatorX::revoke_memory(RuntimeState* state, - const std::shared_ptr& spill_context) { +Status SpillSortSinkOperatorX::revoke_memory(RuntimeState* state) { auto& local_state = get_local_state(state); - return local_state.revoke_memory(state, spill_context); + return local_state.revoke_memory(state); } size_t SpillSortSinkOperatorX::revocable_mem_size(RuntimeState* state) const { auto& local_state = get_local_state(state); - return _sort_sink_operator->get_revocable_mem_size(local_state._runtime_state.get()); + auto mem_size = _sort_sink_operator->get_revocable_mem_size(local_state._runtime_state.get()); + return mem_size > state->spill_min_revocable_mem() ? mem_size : 0; } Status SpillSortSinkOperatorX::sink(doris::RuntimeState* state, Block* in_block, bool eos) { @@ -145,7 +151,6 @@ Status SpillSortSinkOperatorX::sink(doris::RuntimeState* state, Block* in_block, if (in_block->rows() > 0) { local_state._shared_state->update_spill_block_batch_row_count(state, in_block); } - local_state._eos = eos; DBUG_EXECUTE_IF("fault_inject::spill_sort_sink::sink", { return Status::InternalError("fault_inject spill_sort_sink sink failed"); }); RETURN_IF_ERROR(_sort_sink_operator->sink(local_state._runtime_state.get(), in_block, false)); @@ -153,19 +158,25 @@ Status SpillSortSinkOperatorX::sink(doris::RuntimeState* state, Block* in_block, int64_t data_size = local_state._shared_state->in_mem_shared_state->sorter->data_size(); COUNTER_SET(local_state._memory_used_counter, data_size); + // Proactive spill: if already spilled, flush when accumulated data exceeds threshold. + if (local_state._shared_state->is_spilled) { + if (revocable_mem_size(state) >= state->spill_sort_sink_mem_limit_bytes()) { + RETURN_IF_ERROR(revoke_memory(state)); + DCHECK(revocable_mem_size(state) == 0); + } + } + if (eos) { if (local_state._shared_state->is_spilled) { - if (revocable_mem_size(state) > 0) { - RETURN_IF_ERROR(revoke_memory(state, nullptr)); - } else { - local_state._dependency->set_ready_to_read(); - } + RETURN_IF_ERROR(revoke_memory(state)); + // TODO guolei: There is one step that needs to change the state. } else { RETURN_IF_ERROR( local_state._shared_state->in_mem_shared_state->sorter->prepare_for_read( false)); - local_state._dependency->set_ready_to_read(); } + local_state._eos = eos; + local_state._dependency->set_ready_to_read(); } return Status::OK(); } @@ -176,64 +187,40 @@ size_t SpillSortSinkLocalState::get_reserve_mem_size(RuntimeState* state, bool e eos); } -Status SpillSortSinkLocalState::_execute_spill_sort(RuntimeState* state, TUniqueId query_id) { +Status SpillSortSinkLocalState::_execute_spill_sort(RuntimeState* state) { auto& parent = Base::_parent->template cast(); - Status status; + state->get_query_ctx()->resource_ctx()->task_controller()->increase_revoking_tasks_count(); Defer defer {[&]() { - if (!status.ok() || state->is_cancelled()) { - if (!status.ok()) { - LOG(WARNING) << fmt::format( - "Query:{}, sort sink:{}, task:{}, revoke memory error:{}", - print_id(query_id), _parent->node_id(), state->task_id(), status); - } - _shared_state->close(); - } else { - VLOG_DEBUG << fmt::format("Query:{}, sort sink:{}, task:{}, revoke memory finish", - print_id(query_id), _parent->node_id(), state->task_id()); - } - - if (!status.ok()) { - _shared_state->close(); - } - - _spilling_stream.reset(); + _spilling_writer.reset(); + _spilling_file.reset(); state->get_query_ctx()->resource_ctx()->task_controller()->decrease_revoking_tasks_count(); - if (_eos) { - _dependency->set_ready_to_read(); - } }}; - status = parent._sort_sink_operator->prepare_for_spill(_runtime_state.get()); - RETURN_IF_ERROR(status); + RETURN_IF_ERROR(parent._sort_sink_operator->prepare_for_spill(_runtime_state.get())); auto* sink_local_state = _runtime_state->get_sink_local_state(); update_profile(sink_local_state->custom_profile()); bool eos = false; - Block block; - int32_t batch_size = - _shared_state->spill_block_batch_row_count > std::numeric_limits::max() - ? std::numeric_limits::max() - : static_cast(_shared_state->spill_block_batch_row_count); + Block block; while (!eos && !state->is_cancelled()) { { SCOPED_TIMER(_spill_merge_sort_timer); - status = parent._sort_sink_operator->merge_sort_read_for_spill( - _runtime_state.get(), &block, batch_size, &eos); + // Currently, using 4096 as batch size, maybe using adptive size is better by using memory size. + RETURN_IF_ERROR(parent._sort_sink_operator->merge_sort_read_for_spill( + _runtime_state.get(), &block, 4096, &eos)); } - RETURN_IF_ERROR(status); - status = _spilling_stream->spill_block(state, block, eos); - RETURN_IF_ERROR(status); + RETURN_IF_ERROR(_spilling_writer->write_block(state, block)); block.clear_column_data(); } - parent._sort_sink_operator->reset(_runtime_state.get()); + RETURN_IF_ERROR(_spilling_writer->close()); + parent._sort_sink_operator->reset(_runtime_state.get()); return Status::OK(); } -Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state, - const std::shared_ptr& spill_context) { +Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state) { auto& parent = Base::_parent->template cast(); if (!_shared_state->is_spilled) { _shared_state->is_spilled = true; @@ -245,44 +232,16 @@ Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state, VLOG_DEBUG << fmt::format("Query:{}, sort sink:{}, task:{}, revoke_memory, eos:{}", print_id(state->query_id()), _parent->node_id(), state->task_id(), _eos); - - int32_t batch_size = - _shared_state->spill_block_batch_row_count > std::numeric_limits::max() - ? std::numeric_limits::max() - : static_cast(_shared_state->spill_block_batch_row_count); - auto status = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - state, _spilling_stream, print_id(state->query_id()), "sort", _parent->node_id(), - batch_size, state->spill_sort_batch_bytes(), operator_profile()); - RETURN_IF_ERROR(status); - - _shared_state->sorted_streams.emplace_back(_spilling_stream); - - auto query_id = state->query_id(); - auto exception_catch_func = [this, query_id, state]() { - DBUG_EXECUTE_IF("fault_inject::spill_sort_sink::revoke_memory_cancel", { - auto status = - Status::InternalError("fault_inject spill_sort_sink revoke_memory canceled"); - state->get_query_ctx()->cancel(status); - return status; - }); - - auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION({ return _execute_spill_sort(state, query_id); }); - }(); - - return status; - }; - - DBUG_EXECUTE_IF("fault_inject::spill_sort_sink::revoke_memory_submit_func", { - status = Status::Error( - "fault_inject spill_sort_sink " - "revoke_memory submit_func failed"); - }); - - RETURN_IF_ERROR(status); - state->get_query_ctx()->resource_ctx()->task_controller()->increase_revoking_tasks_count(); - - return SpillSinkRunnable(state, spill_context, operator_profile(), exception_catch_func).run(); + _spilling_file.reset(); + _spilling_writer.reset(); + auto relative_path = + fmt::format("{}/{}-{}-{}-{}", print_id(state->query_id()), "sort", _parent->node_id(), + state->task_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + _spilling_file)); + RETURN_IF_ERROR(_spilling_file->create_writer(state, operator_profile(), _spilling_writer)); + _shared_state->sorted_spill_groups.emplace_back(_spilling_file); + return _execute_spill_sort(state); } #include "common/compile_check_end.h" -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/exec/operator/spill_sort_sink_operator.h b/be/src/exec/operator/spill_sort_sink_operator.h index 5284d68113966a..823bc5e7f04a90 100644 --- a/be/src/exec/operator/spill_sort_sink_operator.h +++ b/be/src/exec/operator/spill_sort_sink_operator.h @@ -21,6 +21,8 @@ #include "exec/operator/operator.h" #include "exec/operator/sort_sink_operator.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_writer.h" namespace doris { #include "common/compile_check_begin.h" @@ -44,13 +46,13 @@ class SpillSortSinkLocalState : public PipelineXSpillSinkLocalState& spill_context); + Status revoke_memory(RuntimeState* state); private: void _init_counters(); void update_profile(RuntimeProfile* child_profile); - Status _execute_spill_sort(RuntimeState* state, TUniqueId query_id); + Status _execute_spill_sort(RuntimeState* state); friend class SpillSortSinkOperatorX; @@ -59,7 +61,8 @@ class SpillSortSinkLocalState : public PipelineXSpillSinkLocalState _eos = false; }; @@ -101,8 +104,7 @@ class SpillSortSinkOperatorX final : public DataSinkOperatorX& spill_context) override; + Status revoke_memory(RuntimeState* state) override; using DataSinkOperatorX::node_id; using DataSinkOperatorX::operator_id; diff --git a/be/src/exec/operator/spill_sort_source_operator.cpp b/be/src/exec/operator/spill_sort_source_operator.cpp index cb656761f74cb5..7bb1bfa448b5cb 100644 --- a/be/src/exec/operator/spill_sort_source_operator.cpp +++ b/be/src/exec/operator/spill_sort_source_operator.cpp @@ -26,9 +26,11 @@ #include "exec/operator/sort_source_operator.h" #include "exec/operator/spill_utils.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" #include "runtime/fragment_mgr.h" -#include "runtime/runtime_profile.h" namespace doris { #include "common/compile_check_begin.h" @@ -61,152 +63,118 @@ Status SpillSortLocalState::close(RuntimeState* state) { if (_closed) { return Status::OK(); } + + for (auto& reader : _current_merging_readers) { + if (reader) { + RETURN_IF_ERROR(reader->close()); + reader.reset(); + } + } + _current_merging_readers.clear(); + _current_merging_files.clear(); + _merger.reset(); + return Base::close(state); } int SpillSortLocalState::_calc_spill_blocks_to_merge(RuntimeState* state) const { - auto count = state->spill_sort_mem_limit() / state->spill_sort_batch_bytes(); - if (count > std::numeric_limits::max()) [[unlikely]] { - return std::numeric_limits::max(); - } - return std::max(2, static_cast(count)); + auto count = state->spill_sort_merge_mem_limit_bytes() / state->spill_buffer_size_bytes(); + return std::max(8, static_cast(count)); } -Status SpillSortLocalState::_execute_merge_sort_spill_streams(RuntimeState* state, - TUniqueId query_id) { +Status SpillSortLocalState::execute_merge_sort_spill_files(RuntimeState* state) { auto& parent = Base::_parent->template cast(); SCOPED_TIMER(_spill_merge_sort_timer); Status status; - Defer defer {[&]() { - if (!status.ok() || state->is_cancelled()) { - if (!status.ok()) { - LOG(WARNING) << fmt::format( - "Query:{}, sort source:{}, task:{}, merge spill data error:{}", - print_id(query_id), _parent->node_id(), state->task_id(), status); - } - for (auto& stream : _current_merging_streams) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); - } - _current_merging_streams.clear(); - } else { - VLOG_DEBUG << fmt::format("Query:{}, sort source:{}, task:{}, merge spill data finish", - print_id(query_id), _parent->node_id(), state->task_id()); - } - }}; + Block merge_sorted_block; - SpillStreamSPtr tmp_stream; + auto query_id = state->query_id(); while (!state->is_cancelled()) { int max_stream_count = _calc_spill_blocks_to_merge(state); VLOG_DEBUG << fmt::format( "Query:{}, sort source:{}, task:{}, merge spill streams, streams count:{}, " - "curren merge max stream count:{}", + "curren merge max spill file count:{}", print_id(query_id), _parent->node_id(), state->task_id(), - _shared_state->sorted_streams.size(), 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()) { + _shared_state->sorted_spill_groups.size(), max_stream_count); + RETURN_IF_ERROR(_create_intermediate_merger( + state, max_stream_count, + parent._sort_source_operator->get_sort_description(_runtime_state.get()))); + // It is a fast path, because all the remaining streams can be merged in a run + if (_shared_state->sorted_spill_groups.empty()) { return Status::OK(); } - { - int32_t batch_size = - _shared_state->spill_block_batch_row_count > std::numeric_limits::max() - ? std::numeric_limits::max() - : static_cast(_shared_state->spill_block_batch_row_count); - status = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - state, tmp_stream, print_id(state->query_id()), "sort", _parent->node_id(), - batch_size, state->spill_sort_batch_bytes(), operator_profile()); - RETURN_IF_ERROR(status); - - _shared_state->sorted_streams.emplace_back(tmp_stream); - - bool eos = false; - while (!eos && !state->is_cancelled()) { - merge_sorted_block.clear_column_data(); - { - SCOPED_TIMER(Base::_spill_recover_time); - DBUG_EXECUTE_IF("fault_inject::spill_sort_source::recover_spill_data", { - status = Status::Error( - "fault_inject spill_sort_source " - "recover_spill_data failed"); - }); - if (status.ok()) { - status = _merger->get_next(&merge_sorted_block, &eos); - } - } - RETURN_IF_ERROR(status); - status = tmp_stream->spill_block(state, merge_sorted_block, eos); + SpillFileSPtr tmp_file; + auto label = "sort"; + auto relative_path = fmt::format("{}/{}-{}-{}-{}", print_id(state->query_id()), label, + _parent->node_id(), state->task_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + tmp_file)); + SpillFileWriterSPtr tmp_writer; + RETURN_IF_ERROR(tmp_file->create_writer(state, operator_profile(), tmp_writer)); + _shared_state->sorted_spill_groups.emplace_back(tmp_file); + + bool eos = false; + while (!eos && !state->is_cancelled()) { + merge_sorted_block.clear_column_data(); + { + DBUG_EXECUTE_IF("fault_inject::spill_sort_source::recover_spill_data", { + status = Status::Error( + "fault_inject spill_sort_source " + "recover_spill_data failed"); + }); if (status.ok()) { - DBUG_EXECUTE_IF("fault_inject::spill_sort_source::spill_merged_data", { - status = Status::Error( - "fault_inject spill_sort_source " - "spill_merged_data failed"); - }); + status = _merger->get_next(&merge_sorted_block, &eos); } - RETURN_IF_ERROR(status); } + RETURN_IF_ERROR(status); + status = tmp_writer->write_block(state, merge_sorted_block); + if (status.ok()) { + DBUG_EXECUTE_IF("fault_inject::spill_sort_source::spill_merged_data", { + status = Status::Error( + "fault_inject spill_sort_source " + "spill_merged_data failed"); + }); + } + RETURN_IF_ERROR(status); } - for (auto& stream : _current_merging_streams) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); - } - _current_merging_streams.clear(); + RETURN_IF_ERROR(tmp_writer->close()); } return Status::OK(); } -Status SpillSortLocalState::initiate_merge_sort_spill_streams(RuntimeState* state) { - VLOG_DEBUG << fmt::format("Query:{}, sort source:{}, task:{}, merge spill data", - print_id(state->query_id()), _parent->node_id(), state->task_id()); - - auto query_id = state->query_id(); - auto exception_catch_func = [this, state, query_id]() { - auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION( - { return _execute_merge_sort_spill_streams(state, query_id); }); - }(); - return status; - }; - - DBUG_EXECUTE_IF("fault_inject::spill_sort_source::merge_sort_spill_data_submit_func", { - return Status::Error( - "fault_inject spill_sort_source " - "merge_sort_spill_data submit_func failed"); - }); - - return SpillRecoverRunnable(state, operator_profile(), exception_catch_func).run(); -} - -Status SpillSortLocalState::_create_intermediate_merger(int num_blocks, +Status SpillSortLocalState::_create_intermediate_merger(RuntimeState* state, int num_blocks, const SortDescription& sort_description) { std::vector child_block_suppliers; int64_t limit = -1; int64_t offset = 0; - if (num_blocks >= _shared_state->sorted_streams.size()) { + if (num_blocks >= _shared_state->sorted_spill_groups.size()) { // final round use real limit and offset limit = Base::_shared_state->limit; offset = Base::_shared_state->offset; } - _merger = std::make_unique(sort_description, _runtime_state->batch_size(), - limit, offset, custom_profile()); + _merger = std::make_unique(sort_description, state->batch_size(), limit, + offset, custom_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(operator_profile()); - _current_merging_streams.emplace_back(stream); - child_block_suppliers.emplace_back([stream](Block* block, bool* eos) { - return stream->read_next_block_sync(block, eos); - }); + _current_merging_files.clear(); + _current_merging_readers.clear(); + for (int i = 0; i < num_blocks && !_shared_state->sorted_spill_groups.empty(); ++i) { + auto spill_file = _shared_state->sorted_spill_groups.front(); + _shared_state->sorted_spill_groups.pop_front(); + _current_merging_files.emplace_back(spill_file); + + // Each SpillFile's reader handles multi-part reading internally. + auto reader = spill_file->create_reader(state, operator_profile()); + RETURN_IF_ERROR(reader->open()); - _shared_state->sorted_streams.pop_front(); + auto reader_ptr = reader.get(); + _current_merging_readers.emplace_back(std::move(reader)); + + child_block_suppliers.emplace_back([reader_ptr](Block* block, bool* eos) -> Status { + return reader_ptr->read(block, eos); + }); } RETURN_IF_ERROR(_merger->prepare(child_block_suppliers)); return Status::OK(); @@ -261,37 +229,35 @@ Status SpillSortSourceOperatorX::prepare(RuntimeState* state) { Status SpillSortSourceOperatorX::close(RuntimeState* state) { RETURN_IF_ERROR(OperatorXBase::close(state)); + + // Perform final cleanup for local state: delete any merging streams and + // close shared state. Centralize cleanup so resources are released when + // the pipeline task finishes. + auto& local_state = get_local_state(state); + local_state._current_merging_files.clear(); + local_state._current_merging_readers.clear(); + local_state._merger.reset(); + + if (local_state._shared_state) { + local_state._shared_state->close(); + } + return _sort_source_operator->close(state); } Status SpillSortSourceOperatorX::get_block(RuntimeState* state, Block* block, bool* eos) { auto& local_state = get_local_state(state); - local_state.copy_shared_spill_profile(); - Status status; - Defer defer {[&]() { - if (!status.ok() || *eos) { - local_state._shared_state->close(); - for (auto& stream : local_state._current_merging_streams) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); - } - local_state._current_merging_streams.clear(); - local_state._merger.reset(); - } - }}; SCOPED_TIMER(local_state.exec_time_counter()); if (local_state._shared_state->is_spilled) { if (!local_state._merger) { - status = local_state.initiate_merge_sort_spill_streams(state); - return status; + return local_state.execute_merge_sort_spill_files(state); } else { - SCOPED_TIMER(local_state._spill_total_timer); - status = local_state._merger->get_next(block, eos); - RETURN_IF_ERROR(status); + RETURN_IF_ERROR(local_state._merger->get_next(block, eos)); } } else { - status = _sort_source_operator->get_block(local_state._runtime_state.get(), block, eos); - RETURN_IF_ERROR(status); + RETURN_IF_ERROR( + _sort_source_operator->get_block(local_state._runtime_state.get(), block, eos)); } local_state.reached_limit(block, eos); return Status::OK(); diff --git a/be/src/exec/operator/spill_sort_source_operator.h b/be/src/exec/operator/spill_sort_source_operator.h index 5027446b787075..969582243e0d19 100644 --- a/be/src/exec/operator/spill_sort_source_operator.h +++ b/be/src/exec/operator/spill_sort_source_operator.h @@ -21,6 +21,8 @@ #include "common/status.h" #include "exec/operator/operator.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_reader.h" namespace doris { #include "common/compile_check_begin.h" @@ -46,20 +48,21 @@ class SpillSortLocalState final : public PipelineXSpillLocalState _runtime_state; bool _opened = false; - std::vector _current_merging_streams; + std::vector _current_merging_files; + /// Readers held alive during merge; one per SpillFile, reads parts sequentially. + std::vector _current_merging_readers; std::unique_ptr _merger; std::unique_ptr _internal_runtime_profile; diff --git a/be/src/exec/operator/spill_utils.h b/be/src/exec/operator/spill_utils.h index 44c4de17fd43d8..6a84c6fc2b0e45 100644 --- a/be/src/exec/operator/spill_utils.h +++ b/be/src/exec/operator/spill_utils.h @@ -35,8 +35,15 @@ namespace doris { #include "common/compile_check_begin.h" + +// Default spill partitioner for initial partitioning (level-0). Repartition +// paths may use different channel-id policies (e.g. raw-hash mode). using SpillPartitionerType = Crc32HashPartitioner; +// Repartition partitioner: keeps raw hash (no final modulo) so SpillRepartitioner +// can apply level-aware hash mixing and channel mapping. +using SpillRePartitionerType = Crc32HashPartitioner; + struct SpillContext { std::atomic_int running_tasks_count; TUniqueId query_id; @@ -64,154 +71,20 @@ struct SpillContext { } }; -class SpillRunnable { -protected: - SpillRunnable(RuntimeState* state, std::shared_ptr spill_context, - RuntimeProfile* operator_profile, bool is_write, - std::function spill_exec_func, - std::function spill_fin_cb = {}) - : _state(state), - _custom_profile(operator_profile->get_child("CustomCounters")), - _spill_context(std::move(spill_context)), - _is_write_task(is_write), - _spill_exec_func(std::move(spill_exec_func)), - _spill_fin_cb(std::move(spill_fin_cb)) { - RuntimeProfile* common_profile = operator_profile->get_child("CommonCounters"); - DCHECK(common_profile != nullptr); - DCHECK(_custom_profile != nullptr); - _spill_total_timer = _custom_profile->get_counter("SpillTotalTime"); - - if (is_write) { - _write_wait_in_queue_task_count = - _custom_profile->get_counter("SpillWriteTaskWaitInQueueCount"); - _writing_task_count = _custom_profile->get_counter("SpillWriteTaskCount"); - COUNTER_UPDATE(_write_wait_in_queue_task_count, 1); - } - } - -public: - virtual ~SpillRunnable() = default; - - [[nodiscard]] Status run() { - SCOPED_TIMER(_spill_total_timer); - - auto* spill_timer = _get_spill_timer(); - DCHECK(spill_timer != nullptr); - SCOPED_TIMER(spill_timer); - - _on_task_started(); - - Defer defer([&] { - { - std::function tmp; - std::swap(tmp, _spill_exec_func); - } - { - std::function tmp; - std::swap(tmp, _spill_fin_cb); - } - }); - - if (_state->is_cancelled()) { - return _state->cancel_reason(); - } - - RETURN_IF_ERROR(_spill_exec_func()); - _on_task_finished(); - if (_spill_fin_cb) { - return _spill_fin_cb(); - } - - return Status::OK(); - } - -protected: - virtual void _on_task_finished() { - if (_spill_context) { - _spill_context->on_task_finished(); - } - } - - virtual RuntimeProfile::Counter* _get_spill_timer() { - return _custom_profile->get_counter("SpillWriteTime"); - } - - virtual void _on_task_started() { - VLOG_DEBUG << "Query: " << print_id(_state->query_id()) - << " spill task started, pipeline task id: " << _state->task_id(); - if (_is_write_task) { - COUNTER_UPDATE(_write_wait_in_queue_task_count, -1); - COUNTER_UPDATE(_writing_task_count, 1); - } - } - - RuntimeState* _state; - RuntimeProfile* _custom_profile; - std::shared_ptr _spill_context; - bool _is_write_task; - -private: - RuntimeProfile::Counter* _spill_total_timer; - - RuntimeProfile::Counter* _write_wait_in_queue_task_count = nullptr; - RuntimeProfile::Counter* _writing_task_count = nullptr; - - std::function _spill_exec_func; - std::function _spill_fin_cb; -}; - -class SpillSinkRunnable : public SpillRunnable { -public: - SpillSinkRunnable(RuntimeState* state, std::shared_ptr spill_context, - RuntimeProfile* operator_profile, std::function spill_exec_func, - std::function spill_fin_cb = {}) - : SpillRunnable(state, spill_context, operator_profile, true, spill_exec_func, - spill_fin_cb) {} -}; - -class SpillNonSinkRunnable : public SpillRunnable { -public: - SpillNonSinkRunnable(RuntimeState* state, RuntimeProfile* operator_profile, - std::function spill_exec_func, - std::function spill_fin_cb = {}) - : SpillRunnable(state, nullptr, operator_profile, true, spill_exec_func, spill_fin_cb) { - } -}; - -class SpillRecoverRunnable : public SpillRunnable { -public: - SpillRecoverRunnable(RuntimeState* state, RuntimeProfile* operator_profile, - std::function spill_exec_func, - std::function spill_fin_cb = {}) - : SpillRunnable(state, nullptr, operator_profile, false, spill_exec_func, - spill_fin_cb) { - RuntimeProfile* custom_profile = operator_profile->get_child("CustomCounters"); - DCHECK(custom_profile != nullptr); - _spill_revover_timer = custom_profile->get_counter("SpillRecoverTime"); - _read_wait_in_queue_task_count = - custom_profile->get_counter("SpillReadTaskWaitInQueueCount"); - _reading_task_count = custom_profile->get_counter("SpillReadTaskCount"); - - COUNTER_UPDATE(_read_wait_in_queue_task_count, 1); +// helper to execute a spill function synchronously. The old code used +// SpillRunnable/SpillSinkRunnable/SpillRecoverRunnable wrappers to track +// counters and optionally notify a SpillContext. Since spill operations are +// now performed synchronously and external code already maintains any +// necessary counters, those wrappers are no longer necessary. We keep a +// small utility to run the provided callbacks and forward cancellation. +inline Status run_spill_task(RuntimeState* state, std::function exec_func, + std::function fin_cb = {}) { + RETURN_IF_ERROR(exec_func()); + if (fin_cb) { + RETURN_IF_ERROR(fin_cb()); } - -protected: - RuntimeProfile::Counter* _get_spill_timer() override { - return _custom_profile->get_counter("SpillRecoverTime"); - } - - void _on_task_started() override { - VLOG_DEBUG << "SpillRecoverRunnable, Query: " << print_id(_state->query_id()) - << " spill task started, pipeline task id: " << _state->task_id(); - COUNTER_UPDATE(_read_wait_in_queue_task_count, -1); - COUNTER_UPDATE(_reading_task_count, 1); - } - -private: - RuntimeProfile::Counter* _spill_revover_timer; - RuntimeProfile::Counter* _read_wait_in_queue_task_count = nullptr; - RuntimeProfile::Counter* _reading_task_count = nullptr; -}; + return Status::OK(); +} template inline void update_profile_from_inner_profile(const std::string& name, @@ -240,4 +113,5 @@ inline void update_profile_from_inner_profile(const std::string& name, } #include "common/compile_check_end.h" -} // namespace doris \ No newline at end of file + +} // namespace doris diff --git a/be/src/exec/partitioner/partitioner.cpp b/be/src/exec/partitioner/partitioner.cpp index 39914d85d17d07..bd0c9ba7709df5 100644 --- a/be/src/exec/partitioner/partitioner.cpp +++ b/be/src/exec/partitioner/partitioner.cpp @@ -87,5 +87,6 @@ Status Crc32CHashPartitioner::clone(RuntimeState* state, template class Crc32HashPartitioner; template class Crc32HashPartitioner; +template class Crc32HashPartitioner; } // namespace doris diff --git a/be/src/exec/partitioner/partitioner.h b/be/src/exec/partitioner/partitioner.h index b6a4ab631fdbbd..4eaed892cb8114 100644 --- a/be/src/exec/partitioner/partitioner.h +++ b/be/src/exec/partitioner/partitioner.h @@ -114,9 +114,20 @@ struct ShuffleChannelIds { struct SpillPartitionChannelIds { using HashValType = PartitionerBase::HashValType; + // Default spill partition mapping used by level-0 partitioning: + // rotate hash bits and apply modulo to get a channel id directly. HashValType operator()(HashValType l, size_t r) { return ((l >> 16) | (l << 16)) % r; } }; +struct SpillRePartitionChannelIds { + using HashValType = PartitionerBase::HashValType; + + // Repartition mode: return the raw hash value without modulo. + // The caller (SpillRepartitioner) will apply level-aware hash mixing and + // final channel mapping, so repartition behavior can vary by level. + HashValType operator()(HashValType l, size_t /*r*/) { return l; } +}; + static inline PartitionerBase::HashValType crc32c_shuffle_mix(PartitionerBase::HashValType h) { // Step 1: fold high entropy into low bits h ^= h >> 16; diff --git a/be/src/exec/pipeline/dependency.cpp b/be/src/exec/pipeline/dependency.cpp index 6b10ffa942486e..092398350737c0 100644 --- a/be/src/exec/pipeline/dependency.cpp +++ b/be/src/exec/pipeline/dependency.cpp @@ -21,19 +21,13 @@ #include #include "common/logging.h" -#include "exec/common/util.hpp" #include "exec/operator/multi_cast_data_streamer.h" #include "exec/pipeline/pipeline_fragment_context.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/rowid_fetcher.h" -#include "exec/runtime_filter/runtime_filter_consumer.h" -#include "exec/scan/file_scanner.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "exprs/vectorized_agg_fn.h" #include "exprs/vslot_ref.h" #include "runtime/exec_env.h" -#include "runtime/memory/mem_tracker.h" -#include "util/brpc_client_cache.h" namespace doris { #include "common/compile_check_begin.h" @@ -317,71 +311,13 @@ Status AggSharedState::reset_hash_table() { agg_data->method_variant); } -void PartitionedAggSharedState::init_spill_params(size_t spill_partition_count) { - partition_count = spill_partition_count; - max_partition_index = partition_count - 1; - - for (int i = 0; i < partition_count; ++i) { - spill_partitions.emplace_back(std::make_shared()); - } -} - -void PartitionedAggSharedState::update_spill_stream_profiles(RuntimeProfile* source_profile) { - for (auto& partition : spill_partitions) { - if (partition->spilling_stream_) { - partition->spilling_stream_->update_shared_profiles(source_profile); - } - for (auto& stream : partition->spill_streams_) { - if (stream) { - stream->update_shared_profiles(source_profile); - } - } - } -} - -Status AggSpillPartition::get_spill_stream(RuntimeState* state, int node_id, - RuntimeProfile* profile, 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_) { - spilling_stream_.reset(); - } - for (auto& stream : spill_streams_) { - (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); - } - spill_streams_.clear(); -} - void PartitionedAggSharedState::close() { - // need to use CAS instead of only `if (!is_closed)` statement, - // to avoid concurrent entry of close() both pass the if statement - bool false_close = false; - if (!is_closed.compare_exchange_strong(false_close, true)) { - return; - } - DCHECK(!false_close && is_closed); - for (auto partition : spill_partitions) { - partition->close(); - } - spill_partitions.clear(); -} - -void SpillSortSharedState::update_spill_stream_profiles(RuntimeProfile* source_profile) { - for (auto& stream : sorted_streams) { - if (stream) { - stream->update_shared_profiles(source_profile); + for (auto& partition : _spill_partitions) { + if (partition) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(partition); } } + _spill_partitions.clear(); } void SpillSortSharedState::close() { @@ -392,18 +328,13 @@ void SpillSortSharedState::close() { return; } DCHECK(!false_close && is_closed); - for (auto& stream : sorted_streams) { - (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); - } - sorted_streams.clear(); + sorted_spill_groups.clear(); } MultiCastSharedState::MultiCastSharedState(ObjectPool* pool, int cast_sender_count, int node_id) : multi_cast_data_streamer( std::make_unique(pool, cast_sender_count, node_id)) {} -void MultiCastSharedState::update_spill_stream_profiles(RuntimeProfile* source_profile) {} - int AggSharedState::get_slot_column_id(const AggFnEvaluator* evaluator) { auto ctxs = evaluator->input_exprs_ctxs(); CHECK(ctxs.size() == 1 && ctxs[0]->root()->is_slot_ref()) diff --git a/be/src/exec/pipeline/dependency.h b/be/src/exec/pipeline/dependency.h index a13188fb7910ba..28c89b5b990ebf 100644 --- a/be/src/exec/pipeline/dependency.h +++ b/be/src/exec/pipeline/dependency.h @@ -44,7 +44,7 @@ #include "exec/operator/join/process_hash_table_probe.h" #include "exec/sort/partition_sorter.h" #include "exec/sort/sorter.h" -#include "exec/spill/spill_stream.h" +#include "exec/spill/spill_file.h" #include "util/brpc_closure.h" #include "util/stack_util.h" @@ -422,88 +422,23 @@ struct AggSharedState : public BasicSharedState { Status _destroy_agg_status(AggregateDataPtr data); }; -struct BasicSpillSharedState { - virtual ~BasicSpillSharedState() = default; - - // These two counters are shared to spill source operators as the initial value - // of 'SpillWriteFileCurrentBytes' and 'SpillWriteFileCurrentCount'. - // Total bytes of spill data written to disk file(after serialized) - RuntimeProfile::Counter* _spill_write_file_total_size = nullptr; - RuntimeProfile::Counter* _spill_file_total_count = nullptr; - - void setup_shared_profile(RuntimeProfile* sink_profile) { - _spill_file_total_count = - ADD_COUNTER_WITH_LEVEL(sink_profile, "SpillWriteFileTotalCount", TUnit::UNIT, 1); - _spill_write_file_total_size = - ADD_COUNTER_WITH_LEVEL(sink_profile, "SpillWriteFileBytes", TUnit::BYTES, 1); - } - - virtual void update_spill_stream_profiles(RuntimeProfile* source_profile) = 0; -}; - -struct AggSpillPartition; struct PartitionedAggSharedState : public BasicSharedState, - public BasicSpillSharedState, public std::enable_shared_from_this { ENABLE_FACTORY_CREATOR(PartitionedAggSharedState) PartitionedAggSharedState() = default; ~PartitionedAggSharedState() override = default; - void update_spill_stream_profiles(RuntimeProfile* source_profile) override; - - void init_spill_params(size_t spill_partition_count); - void close(); - AggSharedState* in_mem_shared_state = nullptr; - std::shared_ptr in_mem_shared_state_sptr; + AggSharedState* _in_mem_shared_state = nullptr; + std::shared_ptr _in_mem_shared_state_sptr; - size_t partition_count; - size_t max_partition_index; - bool is_spilled = false; - std::atomic_bool is_closed = false; - std::deque> spill_partitions; - - size_t get_partition_index(size_t hash_value) const { return hash_value % partition_count; } + // partition count is no longer stored in shared state; operators maintain their own + std::atomic _is_spilled = false; + std::deque _spill_partitions; }; -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, - SpillStreamSPtr& spilling_stream); - - Status flush_if_full() { - DCHECK(spilling_stream_); - Status 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_; - SpillStreamSPtr spilling_stream_; -}; -using AggSpillPartitionSPtr = std::shared_ptr; struct SortSharedState : public BasicSharedState { ENABLE_FACTORY_CREATOR(SortSharedState) public: @@ -511,7 +446,6 @@ struct SortSharedState : public BasicSharedState { }; struct SpillSortSharedState : public BasicSharedState, - public BasicSpillSharedState, public std::enable_shared_from_this { ENABLE_FACTORY_CREATOR(SpillSortSharedState) @@ -523,13 +457,11 @@ struct SpillSortSharedState : public BasicSharedState, if (rows > 0 && 0 == avg_row_bytes) { avg_row_bytes = std::max((std::size_t)1, block->bytes() / rows); spill_block_batch_row_count = - (state->spill_sort_batch_bytes() + avg_row_bytes - 1) / avg_row_bytes; + (state->spill_buffer_size_bytes() + avg_row_bytes - 1) / avg_row_bytes; LOG(INFO) << "spill sort block batch row count: " << spill_block_batch_row_count; } } - void update_spill_stream_profiles(RuntimeProfile* source_profile) override; - void close(); SortSharedState* in_mem_shared_state = nullptr; @@ -540,7 +472,7 @@ struct SpillSortSharedState : public BasicSharedState, std::atomic_bool is_closed = false; std::shared_ptr in_mem_shared_state_sptr; - std::deque sorted_streams; + std::deque sorted_spill_groups; size_t avg_row_bytes = 0; size_t spill_block_batch_row_count; }; @@ -564,12 +496,10 @@ struct DataQueueSharedState : public BasicSharedState { class MultiCastDataStreamer; struct MultiCastSharedState : public BasicSharedState, - public BasicSpillSharedState, public std::enable_shared_from_this { MultiCastSharedState(ObjectPool* pool, int cast_sender_count, int node_id); - std::unique_ptr multi_cast_data_streamer; - void update_spill_stream_profiles(RuntimeProfile* source_profile) override; + std::unique_ptr multi_cast_data_streamer; }; struct AnalyticSharedState : public BasicSharedState { @@ -645,23 +575,14 @@ struct HashJoinSharedState : public JoinSharedState { struct PartitionedHashJoinSharedState : public HashJoinSharedState, - public BasicSpillSharedState, public std::enable_shared_from_this { ENABLE_FACTORY_CREATOR(PartitionedHashJoinSharedState) - void update_spill_stream_profiles(RuntimeProfile* source_profile) override { - for (auto& stream : spilled_streams) { - if (stream) { - stream->update_shared_profiles(source_profile); - } - } - } - - std::unique_ptr inner_runtime_state; - std::shared_ptr inner_shared_state; - std::vector> partitioned_build_blocks; - std::vector spilled_streams; - bool is_spilled = false; + std::unique_ptr _inner_runtime_state; + std::shared_ptr _inner_shared_state; + std::vector> _partitioned_build_blocks; + std::vector _spilled_build_groups; + std::atomic _is_spilled = false; }; struct NestedLoopJoinSharedState : public JoinSharedState { diff --git a/be/src/exec/pipeline/pipeline_fragment_context.cpp b/be/src/exec/pipeline/pipeline_fragment_context.cpp index fc3d83c229de47..3e3b0fe0778ad0 100644 --- a/be/src/exec/pipeline/pipeline_fragment_context.cpp +++ b/be/src/exec/pipeline/pipeline_fragment_context.cpp @@ -108,7 +108,7 @@ #include "exec/pipeline/task_scheduler.h" #include "exec/runtime_filter/runtime_filter_mgr.h" #include "exec/sort/topn_sorter.h" -#include "exec/spill/spill_stream.h" +#include "exec/spill/spill_file.h" #include "io/fs/stream_load_pipe.h" #include "load/stream_load/new_load_stream_mgr.h" #include "runtime/exec_env.h" @@ -602,9 +602,9 @@ void PipelineFragmentContext::trigger_report_if_necessary() { } int32_t interval_s = config::pipeline_status_report_interval; if (interval_s <= 0) { - LOG(WARNING) - << "config::status_report_interval is equal to or less than zero, do not trigger " - "report."; + LOG(WARNING) << "config::status_report_interval is equal to or less than zero, do not " + "trigger " + "report."; } uint64_t next_report_time = _previous_report_time.load(std::memory_order_acquire) + (uint64_t)(interval_s)*NANOS_PER_SEC; @@ -731,7 +731,8 @@ Status PipelineFragmentContext::_create_tree_helper( // this means we have been given a bad tree and must fail if (*node_idx >= tnodes.size()) { return Status::InternalError( - "Failed to reconstruct plan tree from thrift. Node id: {}, number of nodes: {}", + "Failed to reconstruct plan tree from thrift. Node id: {}, number of " + "nodes: {}", *node_idx, tnodes.size()); } } @@ -1345,8 +1346,9 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo const bool is_streaming_agg = tnode.agg_node.__isset.use_streaming_preaggregation && tnode.agg_node.use_streaming_preaggregation && !tnode.agg_node.grouping_exprs.empty(); + // TODO: distinct streaming agg does not support spill. const bool can_use_distinct_streaming_agg = - tnode.agg_node.aggregate_functions.empty() && + (!enable_spill || is_streaming_agg) && tnode.agg_node.aggregate_functions.empty() && !tnode.agg_node.__isset.agg_sort_info_by_group_key && _params.query_options.__isset.enable_distinct_streaming_aggregation && _params.query_options.enable_distinct_streaming_aggregation; @@ -1428,7 +1430,6 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo if (enable_spill && !is_broadcast_join) { auto tnode_ = tnode; tnode_.runtime_filters.clear(); - uint32_t partition_count = _runtime_state->spill_hash_join_partition_count(); auto inner_probe_operator = std::make_shared(pool, tnode_, 0, descs); @@ -1441,7 +1442,7 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo RETURN_IF_ERROR(probe_side_inner_sink_operator->init(tnode_, _runtime_state.get())); auto probe_operator = std::make_shared( - pool, tnode_, next_operator_id(), descs, partition_count); + pool, tnode_, next_operator_id(), descs); probe_operator->set_inner_operators(probe_side_inner_sink_operator, inner_probe_operator); op = std::move(probe_operator); @@ -1457,8 +1458,7 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo auto inner_sink_operator = std::make_shared(pool, 0, 0, tnode, descs); auto sink_operator = std::make_shared( - pool, next_sink_operator_id(), op->operator_id(), tnode_, descs, - partition_count); + pool, next_sink_operator_id(), op->operator_id(), tnode_, descs); RETURN_IF_ERROR(inner_sink_operator->init(tnode, _runtime_state.get())); sink_operator->set_inner_operators(inner_sink_operator, inner_probe_operator); @@ -1921,7 +1921,7 @@ size_t PipelineFragmentContext::get_revocable_size(bool* has_running_task) const } size_t revocable_size = task.first->get_revocable_size(); - if (revocable_size >= SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + if (revocable_size >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { res += revocable_size; } } @@ -1934,7 +1934,8 @@ std::vector PipelineFragmentContext::get_revocable_tasks() const for (const auto& task_instances : _tasks) { for (const auto& task : task_instances) { size_t revocable_size_ = task.first->get_revocable_size(); - if (revocable_size_ >= SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + + if (revocable_size_ >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { revocable_tasks.emplace_back(task.first.get()); } } diff --git a/be/src/exec/pipeline/pipeline_task.cpp b/be/src/exec/pipeline/pipeline_task.cpp index 6bcfd9b3c5c896..29823545d302a4 100644 --- a/be/src/exec/pipeline/pipeline_task.cpp +++ b/be/src/exec/pipeline/pipeline_task.cpp @@ -39,7 +39,7 @@ #include "exec/pipeline/revokable_task.h" #include "exec/pipeline/task_queue.h" #include "exec/pipeline/task_scheduler.h" -#include "exec/spill/spill_stream.h" +#include "exec/spill/spill_file.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" #include "runtime/query_context.h" @@ -369,6 +369,70 @@ void PipelineTask::terminate() { } } +// When current memory pressure is low, memory usage may increase significantly in the next +// operator run, while there is no revocable memory available for spilling. +// Trigger memory revoking when pressure is high and revocable memory is significant. +// Memory pressure is evaluated using two signals: +// 1. Query memory usage exceeds a threshold ratio of the query memory limit. +// 2. Workload group memory usage reaches the workload group low-watermark threshold. +bool PipelineTask::_should_trigger_revoking(const size_t reserve_size) const { + if (!_state->enable_spill()) { + return false; + } + + auto query_mem_tracker = _state->get_query_ctx()->query_mem_tracker(); + auto wg = _state->get_query_ctx()->workload_group(); + if (!query_mem_tracker || !wg) { + return false; + } + + const auto parallelism = std::max(1, _pipeline->num_tasks()); + const auto query_water_mark = 90; // 90% + const auto group_mem_limit = wg->memory_limit(); + auto query_limit = query_mem_tracker->limit(); + if (query_limit <= 0) { + query_limit = group_mem_limit; + } else if (query_limit > group_mem_limit && group_mem_limit > 0) { + query_limit = group_mem_limit; + } + + if (query_limit <= 0) { + return false; + } + + if ((reserve_size * parallelism) <= (query_limit / 5)) { + return false; + } + + bool is_high_memory_pressure = false; + const auto used_mem = query_mem_tracker->consumption() + reserve_size * parallelism; + if (used_mem >= int64_t((double(query_limit) * query_water_mark / 100))) { + is_high_memory_pressure = true; + } + + if (!is_high_memory_pressure) { + bool is_low_watermark; + bool is_high_watermark; + wg->check_mem_used(&is_low_watermark, &is_high_watermark); + is_high_memory_pressure = is_low_watermark || is_high_watermark; + } + + if (is_high_memory_pressure) { + const auto revocable_size = [&]() { + size_t total = _sink->revocable_mem_size(_state); + for (const auto& op : _operators) { + total += op->revocable_mem_size(_state); + } + return total; + }(); + + const auto total_estimated_revocable = revocable_size * parallelism; + return total_estimated_revocable >= int64_t(double(query_limit) * 0.2); + } + + return false; +} + /** * `_eos` indicates whether the execution phase is done. `done` indicates whether we could close * this task. @@ -516,19 +580,39 @@ Status PipelineTask::execute(bool* done) { SCOPED_TIMER(_get_block_timer); if (_state->low_memory_mode()) { _sink->set_low_memory_mode(_state); - _root->set_low_memory_mode(_state); + for (auto& op : _operators) { + op->set_low_memory_mode(_state); + } } DEFER_RELEASE_RESERVED(); _get_block_counter->update(1); - const auto reserve_size = _root->get_reserve_mem_size(_state); - _root->reset_reserve_mem_size(_state); - + // Sum reserve sizes across all operators in this pipeline. + // Each operator reports only its own requirement (non-recursive). + size_t reserve_size = 0; + for (auto& op : _operators) { + reserve_size += op->get_reserve_mem_size(_state); + op->reset_reserve_mem_size(_state); + } if (workload_group && _state->get_query_ctx() ->resource_ctx() ->task_controller() ->is_enable_reserve_memory() && reserve_size > 0) { + if (_should_trigger_revoking(reserve_size)) { + LOG(INFO) << fmt::format( + "Query: {} sink: {}, node id: {}, task id: {}, reserve size: {} when " + "high memory pressure, try to spill", + print_id(_query_id), _sink->get_name(), _sink->node_id(), + _state->task_id(), reserve_size); + ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query( + _state->get_query_ctx()->resource_ctx()->shared_from_this(), + reserve_size, + Status::Error( + "high memory pressure, try to spill")); + _spilling = true; + continue; + } if (!_try_to_reserve_memory(reserve_size, _root)) { continue; } @@ -550,6 +634,22 @@ Status PipelineTask::execute(bool* done) { ->is_enable_reserve_memory() && workload_group && !(_wake_up_early || _dry_run)) { const auto sink_reserve_size = _sink->get_reserve_mem_size(_state, _eos); + + if (sink_reserve_size > 0 && _should_trigger_revoking(sink_reserve_size)) { + LOG(INFO) << fmt::format( + "Query: {} sink: {}, node id: {}, task id: {}, reserve size: {} when " + "high memory pressure, try to spill", + print_id(_query_id), _sink->get_name(), _sink->node_id(), + _state->task_id(), sink_reserve_size); + ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query( + _state->get_query_ctx()->resource_ctx()->shared_from_this(), + sink_reserve_size, + Status::Error( + "high memory pressure, try to spill")); + _spilling = true; + continue; + } + if (sink_reserve_size > 0 && !_try_to_reserve_memory(sink_reserve_size, _sink.get())) { continue; @@ -637,9 +737,26 @@ Status PipelineTask::do_revoke_memory(const std::shared_ptr& spill THROW_IF_ERROR(_sink->terminate(_state)); _eos = true; } + + // SpillContext tracks pipeline task count, not operator count. + // Notify completion once after all operators + sink have finished revoking. + if (spill_context) { + spill_context->on_task_finished(); + } }}; - return _sink->revoke_memory(_state, spill_context); + // Revoke memory from every operator that has enough revocable memory, + // then revoke from the sink. + for (auto& op : _operators) { + if (op->revocable_mem_size(_state) >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { + RETURN_IF_ERROR(op->revoke_memory(_state)); + } + } + + if (_sink->revocable_mem_size(_state) >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { + RETURN_IF_ERROR(_sink->revoke_memory(_state)); + } + return Status::OK(); } bool PipelineTask::_try_to_reserve_memory(const size_t reserve_size, OperatorBase* op) { @@ -653,58 +770,68 @@ bool PipelineTask::_try_to_reserve_memory(const size_t reserve_size, OperatorBas return true; } COUNTER_UPDATE(_memory_reserve_times, 1); - auto sink_revocable_mem_size = _sink->revocable_mem_size(_state); - if (st.ok() && _state->enable_force_spill() && _sink->is_spillable() && - sink_revocable_mem_size >= SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { - st = Status(ErrorCode::QUERY_MEMORY_EXCEEDED, "Force Spill"); + + // Compute total revocable memory across all operators and the sink. + size_t total_revocable_mem_size = 0; + size_t operator_max_revocable_mem_size = 0; + + if (!st.ok() || _state->enable_force_spill()) { + // Compute total revocable memory across all operators and the sink. + total_revocable_mem_size = _sink->revocable_mem_size(_state); + operator_max_revocable_mem_size = total_revocable_mem_size; + for (auto& cur_op : _operators) { + total_revocable_mem_size += cur_op->revocable_mem_size(_state); + operator_max_revocable_mem_size = + std::max(cur_op->revocable_mem_size(_state), operator_max_revocable_mem_size); + } } + + // During enable force spill, other operators like scan opeartor will also try to reserve memory and will failed + // here, if not add this check, it will always paused and resumed again. + if (st.ok() && _state->enable_force_spill()) { + if (operator_max_revocable_mem_size >= _state->spill_min_revocable_mem()) { + st = Status::Error( + "force spill and there is an operator has memory " + "size {} exceeds min mem size {}", + PrettyPrinter::print_bytes(operator_max_revocable_mem_size), + PrettyPrinter::print_bytes(_state->spill_min_revocable_mem())); + } + } + if (!st.ok()) { COUNTER_UPDATE(_memory_reserve_failed_times, 1); + // build per-operator revocable memory info string for debugging + std::string ops_revocable_info; + { + fmt::memory_buffer buf; + for (auto& cur_op : _operators) { + fmt::format_to(buf, "{}({})-> ", cur_op->get_name(), + PrettyPrinter::print_bytes(cur_op->revocable_mem_size(_state))); + } + if (_sink) { + fmt::format_to(buf, "{}({}) ", _sink->get_name(), + PrettyPrinter::print_bytes(_sink->revocable_mem_size(_state))); + } + ops_revocable_info = fmt::to_string(buf); + } + auto debug_msg = fmt::format( - "Query: {} , try to reserve: {}, operator name: {}, operator " - "id: {}, task id: {}, root revocable mem size: {}, sink revocable mem" - "size: {}, failed: {}", - print_id(_query_id), PrettyPrinter::print_bytes(reserve_size), op->get_name(), - op->node_id(), _state->task_id(), - PrettyPrinter::print_bytes(op->revocable_mem_size(_state)), - PrettyPrinter::print_bytes(sink_revocable_mem_size), st.to_string()); + "Query: {} , try to reserve: {}, total revocable mem size: {}, failed reason: {}", + print_id(_query_id), PrettyPrinter::print_bytes(reserve_size), + PrettyPrinter::print_bytes(total_revocable_mem_size), st.to_string()); + if (!ops_revocable_info.empty()) { + debug_msg += fmt::format(", ops_revocable=[{}]", ops_revocable_info); + } // PROCESS_MEMORY_EXCEEDED error msg already contains process_mem_log_str if (!st.is()) { debug_msg += fmt::format(", debug info: {}", GlobalMemoryArbitrator::process_mem_log_str()); } - // If sink has enough revocable memory, trigger revoke memory - LOG(INFO) << fmt::format( - "Query: {} sink: {}, node id: {}, task id: " - "{}, revocable mem size: {}", - print_id(_query_id), _sink->get_name(), _sink->node_id(), _state->task_id(), - PrettyPrinter::print_bytes(sink_revocable_mem_size)); + LOG(INFO) << debug_msg; ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query( _state->get_query_ctx()->resource_ctx()->shared_from_this(), reserve_size, st); _spilling = true; return false; - // !!! Attention: - // In the past, if reserve failed, not add this query to paused list, because it is very small, will not - // consume a lot of memory. But need set low memory mode to indicate that the system should - // not use too much memory. - // But if we only set _state->get_query_ctx()->set_low_memory_mode() here, and return true, the query will - // continue to run and not blocked, and this reserve maybe the last block of join sink opertorator, and it will - // build hash table directly and will consume a lot of memory. So that should return false directly. - // TODO: we should using a global system buffer management logic to deal with low memory mode. - /** - if (sink_revocable_mem_size >= SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { - LOG(INFO) << fmt::format( - "Query: {} sink: {}, node id: {}, task id: " - "{}, revocable mem size: {}", - print_id(_query_id), _sink->get_name(), _sink->node_id(), _state->task_id(), - PrettyPrinter::print_bytes(sink_revocable_mem_size)); - ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query( - _state->get_query_ctx()->resource_ctx()->shared_from_this(), reserve_size, st); - _spilling = true; - return false; - } else { - _state->get_query_ctx()->set_low_memory_mode(); - } */ } return true; } @@ -852,7 +979,13 @@ size_t PipelineTask::get_revocable_size() const { return 0; } - return _sink->revocable_mem_size(_state); + // Sum revocable memory from every operator in the pipeline + the sink. + // Each operator reports only its own revocable memory (no child recursion). + size_t total = _sink->revocable_mem_size(_state); + for (const auto& op : _operators) { + total += op->revocable_mem_size(_state); + } + return total; } Status PipelineTask::revoke_memory(const std::shared_ptr& spill_context) { @@ -864,14 +997,16 @@ Status PipelineTask::revoke_memory(const std::shared_ptr& spill_co return Status::OK(); } - const auto revocable_size = _sink->revocable_mem_size(_state); - if (revocable_size >= SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + const auto revocable_size = get_revocable_size(); + if (revocable_size >= SpillFile::MIN_SPILL_WRITE_BATCH_MEM) { auto revokable_task = std::make_shared(shared_from_this(), spill_context); + // Submit a revocable task to run, the run method will call revoke memory. Currently the + // underline pipeline task is still blocked. RETURN_IF_ERROR(_state->get_query_ctx()->get_pipe_exec_scheduler()->submit(revokable_task)); } else { spill_context->on_task_finished(); - LOG(INFO) << "Query: " << print_id(_state->query_id()) << ", task: " << ((void*)this) - << " has not enough data to revoke: " << revocable_size; + VLOG_DEBUG << "Query: " << print_id(_state->query_id()) << ", task: " << ((void*)this) + << " has not enough data to revoke: " << revocable_size; } return Status::OK(); } diff --git a/be/src/exec/pipeline/pipeline_task.h b/be/src/exec/pipeline/pipeline_task.h index cb89673b2c04b1..e6fed38a5dbac7 100644 --- a/be/src/exec/pipeline/pipeline_task.h +++ b/be/src/exec/pipeline/pipeline_task.h @@ -199,6 +199,7 @@ class PipelineTask : public std::enable_shared_from_this { // Operator `op` try to reserve memory before executing. Return false if reserve failed // otherwise return true. bool _try_to_reserve_memory(const size_t reserve_size, OperatorBase* op); + bool _should_trigger_revoking(const size_t reserve_size) const; const TUniqueId _query_id; const uint32_t _index; diff --git a/be/src/exec/pipeline/revokable_task.h b/be/src/exec/pipeline/revokable_task.h index c2cc728f49b696..d3bf7c5d0dcc3f 100644 --- a/be/src/exec/pipeline/revokable_task.h +++ b/be/src/exec/pipeline/revokable_task.h @@ -63,6 +63,7 @@ class RevokableTask : public PipelineTask { Status execute(bool* done) override { return _task->do_revoke_memory(_spill_context); } + // Use blockable to make sure the task will be submiited to the blockable task queue and can be revoked when necessary. bool is_blockable() const override { return true; } private: diff --git a/be/src/exec/scan/split_source_connector.h b/be/src/exec/scan/split_source_connector.h index 5e61c16b9bcf04..9381e5564c2a92 100644 --- a/be/src/exec/scan/split_source_connector.h +++ b/be/src/exec/scan/split_source_connector.h @@ -18,6 +18,7 @@ #pragma once #include "common/config.h" +#include "core/custom_allocator.h" #include "runtime/runtime_state.h" #include "util/client_cache.h" @@ -46,10 +47,14 @@ class SplitSourceConnector { virtual TFileScanRangeParams* get_params() = 0; protected: - template - void _merge_ranges(std::vector& merged_ranges, const std::vector& scan_ranges) { + template , typename V2 = std::vector> + requires(std::is_same_v, + std::vector> && + std::is_same_v, + std::vector>) + void _merge_ranges(V1& merged_ranges, const V2& scan_ranges) { if (scan_ranges.size() <= _max_scanners) { - merged_ranges = scan_ranges; + merged_ranges.assign(scan_ranges.begin(), scan_ranges.end()); return; } @@ -98,7 +103,7 @@ class SplitSourceConnector { class LocalSplitSourceConnector : public SplitSourceConnector { private: std::mutex _range_lock; - std::vector _scan_ranges; + DorisVector _scan_ranges; int _scan_index = 0; int _range_index = 0; @@ -138,7 +143,7 @@ class RemoteSplitSourceConnector : public SplitSourceConnector { int64_t _split_source_id; int _num_splits; - std::vector _scan_ranges; + DorisVector _scan_ranges; bool _last_batch = false; int _scan_index = 0; int _range_index = 0; diff --git a/be/src/exec/sink/writer/iceberg/viceberg_sort_writer.cpp b/be/src/exec/sink/writer/iceberg/viceberg_sort_writer.cpp index 5be94279a3a945..dc1038e51915d0 100644 --- a/be/src/exec/sink/writer/iceberg/viceberg_sort_writer.cpp +++ b/be/src/exec/sink/writer/iceberg/viceberg_sort_writer.cpp @@ -17,8 +17,9 @@ #include "exec/sink/writer/iceberg/viceberg_sort_writer.h" -#include "exec/spill/spill_stream.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" #include "runtime/exec_env.h" #include "runtime/runtime_state.h" @@ -110,7 +111,7 @@ Status VIcebergSortWriter::close(const Status& status) { // Check if there is any remaining data in the sorter (either unsorted or already sorted blocks) if (!_sorter->merge_sort_state()->unsorted_block()->empty() || !_sorter->merge_sort_state()->get_sorted_block().empty()) { - if (_sorted_streams.empty()) { + if (_sorted_spill_files.empty()) { // No spill has occurred, all data is in memory. // Sort the remaining data, prepare for reading, and write to file. internal_status = _sorter->do_sort(); @@ -134,7 +135,7 @@ Status VIcebergSortWriter::close(const Status& status) { } // Merge all spilled streams using multi-way merge sort and output final sorted data to files - if (!_sorted_streams.empty()) { + if (!_sorted_spill_files.empty()) { internal_status = _combine_files_output(); if (!internal_status.ok()) { return internal_status; @@ -154,7 +155,7 @@ void VIcebergSortWriter::_update_spill_block_batch_row_count(const Block& block) // the optimal batch size for spill operations if (rows > 0 && 0 == _avg_row_bytes) { _avg_row_bytes = std::max(1UL, block.bytes() / rows); - int64_t spill_batch_bytes = _runtime_state->spill_sort_batch_bytes(); // default 8MB + int64_t spill_batch_bytes = _runtime_state->spill_buffer_size_bytes(); // default 8MB // Calculate how many rows fit in one spill batch (ceiling division) _spill_block_batch_row_count = (spill_batch_bytes + _avg_row_bytes - 1) / _avg_row_bytes; } @@ -225,14 +226,18 @@ Status VIcebergSortWriter::_do_spill() { // prepare_for_read(is_spill=true) adjusts limit/offset for spill mode // and builds the merge tree for reading sorted data RETURN_IF_ERROR(_sorter->prepare_for_read(true)); - int32_t batch_size = _get_spill_batch_size(); - // Register a new spill stream to store the sorted data on disk - SpillStreamSPtr spilling_stream; - RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - _runtime_state, spilling_stream, print_id(_runtime_state->query_id()), "iceberg-sort", - 1 /* node_id */, batch_size, _runtime_state->spill_sort_batch_bytes(), _profile)); - _sorted_streams.emplace_back(spilling_stream); + // Register a new spill file to store the sorted data on disk + SpillFileSPtr spilling_file; + auto relative_path = fmt::format("{}/{}-{}-{}-{}", print_id(_runtime_state->query_id()), + "MultiCastSender", 1 /* node_id */, _runtime_state->task_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + spilling_file)); + _sorted_spill_files.emplace_back(spilling_file); + + SpillFileWriterSPtr writer; + RETURN_IF_ERROR(spilling_file->create_writer(_runtime_state, _profile, writer)); // Read sorted data from the sorter in batches and write to the spill stream bool eos = false; @@ -242,7 +247,7 @@ Status VIcebergSortWriter::_do_spill() { // instead of C-style cast, which includes bounds checking RETURN_IF_ERROR(_sorter->merge_sort_read_for_spill(_runtime_state, &block, _get_spill_batch_size(), &eos)); - RETURN_IF_ERROR(spilling_stream->spill_block(_runtime_state, block, eos)); + RETURN_IF_ERROR(writer->write_block(_runtime_state, block)); block.clear_column_data(); } // Reset the sorter to free memory and accept new data @@ -251,13 +256,13 @@ Status VIcebergSortWriter::_do_spill() { } Status VIcebergSortWriter::_combine_files_output() { - // If there are too many spill streams to merge at once (limited by memory), - // perform intermediate merges to reduce the number of streams - while (_sorted_streams.size() > static_cast(_calc_max_merge_streams())) { + // If there are too many spill files to merge at once (limited by memory), + // perform intermediate merges to reduce the number of files + while (_sorted_spill_files.size() > static_cast(_calc_max_merge_streams())) { RETURN_IF_ERROR(_do_intermediate_merge()); } - // Create the final merger that combines all remaining spill streams + // Create the final merger that combines all remaining spill files RETURN_IF_ERROR(_create_final_merger()); // Read merged sorted data and write to Parquet/ORC files, @@ -287,36 +292,41 @@ Status VIcebergSortWriter::_do_intermediate_merge() { // Merge a subset of streams (non-final merge) to reduce total stream count RETURN_IF_ERROR(_create_merger(false, _spill_block_batch_row_count, max_stream_count)); - // Register a new spill stream for the merged output - int32_t batch_size = _get_spill_batch_size(); - SpillStreamSPtr tmp_stream; - RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - _runtime_state, tmp_stream, print_id(_runtime_state->query_id()), "iceberg-sort-merge", - 1 /* node_id */, batch_size, _runtime_state->spill_sort_batch_bytes(), _profile)); + // register new spill stream for merged output + SpillFileSPtr tmp_spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}-{}", print_id(_runtime_state->query_id()), + "MultiCastSender", 1 /* node_id */, _runtime_state->task_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + tmp_spill_file)); + + _sorted_spill_files.emplace_back(tmp_spill_file); - _sorted_streams.emplace_back(tmp_stream); + SpillFileWriterSPtr tmp_spill_writer; + RETURN_IF_ERROR(tmp_spill_file->create_writer(_runtime_state, _profile, tmp_spill_writer)); - // Merge the selected streams and write the result to the new spill stream + // Merge the selected files and write the result to the new spill file bool eos = false; Block merge_sorted_block; while (!eos && !_runtime_state->is_cancelled()) { merge_sorted_block.clear_column_data(); RETURN_IF_ERROR(_merger->get_next(&merge_sorted_block, &eos)); - RETURN_IF_ERROR(tmp_stream->spill_block(_runtime_state, merge_sorted_block, eos)); + RETURN_IF_ERROR(tmp_spill_writer->write_block(_runtime_state, merge_sorted_block)); } - // Clean up the streams that were consumed during this intermediate merge - for (auto& stream : _current_merging_streams) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + // Clean up the files that were consumed during this intermediate merge + for (auto& file : _current_merging_spill_files) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(file); } - _current_merging_streams.clear(); + _current_merging_spill_files.clear(); return Status::OK(); } int VIcebergSortWriter::_calc_max_merge_streams() const { // Calculate the maximum number of streams that can be merged simultaneously // based on the available memory limit and per-stream batch size - auto count = _runtime_state->spill_sort_mem_limit() / _runtime_state->spill_sort_batch_bytes(); + auto count = _runtime_state->spill_sort_merge_mem_limit_bytes() / + _runtime_state->spill_buffer_size_bytes(); if (count > std::numeric_limits::max()) { return std::numeric_limits::max(); } @@ -329,21 +339,19 @@ Status VIcebergSortWriter::_create_merger(bool is_final_merge, size_t batch_size std::vector child_block_suppliers; _merger = std::make_unique(_sorter->get_sort_description(), batch_size, -1, 0, _profile); - _current_merging_streams.clear(); + _current_merging_spill_files.clear(); // For final merge: merge all remaining streams // For intermediate merge: merge only num_streams streams - size_t streams_to_merge = is_final_merge ? _sorted_streams.size() : num_streams; - - for (size_t i = 0; i < streams_to_merge && !_sorted_streams.empty(); ++i) { - auto stream = _sorted_streams.front(); - stream->set_read_counters(_profile); - _current_merging_streams.emplace_back(stream); - // Create a block supplier lambda that reads the next block from the spill stream - child_block_suppliers.emplace_back([stream](Block* block, bool* eos) { - return stream->read_next_block_sync(block, eos); - }); - _sorted_streams.pop_front(); + size_t streams_to_merge = is_final_merge ? _sorted_spill_files.size() : num_streams; + + for (size_t i = 0; i < streams_to_merge && !_sorted_spill_files.empty(); ++i) { + auto spill_file = _sorted_spill_files.front(); + _current_merging_spill_files.emplace_back(spill_file); + SpillFileReaderSPtr reader = spill_file->create_reader(_runtime_state, _profile); + child_block_suppliers.emplace_back( + [reader](Block* block, bool* eos) { return reader->read(block, eos); }); + _sorted_spill_files.pop_front(); } RETURN_IF_ERROR(_merger->prepare(child_block_suppliers)); @@ -356,17 +364,17 @@ Status VIcebergSortWriter::_create_final_merger() { } void VIcebergSortWriter::_cleanup_spill_streams() { - // Clean up all remaining spill streams to release disk resources - for (auto& stream : _sorted_streams) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + // Clean up all remaining spill files to release disk resources + for (auto& file : _sorted_spill_files) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(file); } - _sorted_streams.clear(); + _sorted_spill_files.clear(); - // Also clean up any streams that are currently being merged - for (auto& stream : _current_merging_streams) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + // Also clean up any files that are currently being merged + for (auto& file : _current_merging_spill_files) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(file); } - _current_merging_streams.clear(); + _current_merging_spill_files.clear(); } #include "common/compile_check_end.h" diff --git a/be/src/exec/sink/writer/iceberg/viceberg_sort_writer.h b/be/src/exec/sink/writer/iceberg/viceberg_sort_writer.h index 0348a048d07a62..95ffc5a60e2faf 100644 --- a/be/src/exec/sink/writer/iceberg/viceberg_sort_writer.h +++ b/be/src/exec/sink/writer/iceberg/viceberg_sort_writer.h @@ -34,12 +34,10 @@ // in this header, causing heavy transitive includes for all files that include // viceberg_sort_writer.h. Moving implementations to .cpp allows us to forward-declare // these types and only include their headers in the .cpp file. -namespace doris { -class SpillStream; -using SpillStreamSPtr = std::shared_ptr; -} // namespace doris namespace doris { +class SpillFile; +using SpillFileSPtr = std::shared_ptr; class RuntimeState; class RuntimeProfile; @@ -169,10 +167,11 @@ class VIcebergSortWriter : public IPartitionWriterBase { // Sorter and merger for handling in-memory sorting and multi-way merge std::unique_ptr _sorter; std::unique_ptr _merger; - // Queue of spill streams waiting to be merged (FIFO order) - std::deque _sorted_streams; - // Streams currently being consumed by the merger - std::vector _current_merging_streams; + + // Queue of spill files waiting to be merged (FIFO order) + std::deque _sorted_spill_files; + // Files currently being consumed by the merger + std::vector _current_merging_spill_files; // Target file size in bytes; files are split when this threshold is exceeded // Default: config::iceberg_sink_max_file_size (1GB) diff --git a/be/src/exec/sort/sort_cursor.h b/be/src/exec/sort/sort_cursor.h index 2a0d85d0fd342a..be869f0e52fda9 100644 --- a/be/src/exec/sort/sort_cursor.h +++ b/be/src/exec/sort/sort_cursor.h @@ -20,6 +20,8 @@ #pragma once +#include + #include #include "core/block/block.h" diff --git a/be/src/exec/sort/sorter.h b/be/src/exec/sort/sorter.h index 42ef13792eb464..4d8777abeff96f 100644 --- a/be/src/exec/sort/sorter.h +++ b/be/src/exec/sort/sorter.h @@ -118,6 +118,8 @@ class Sorter { : _vsort_exec_exprs(mock_vsort_exec_exprs), _is_asc_order(mock_is_asc_order), _nulls_first(mock_nulls_first) {} + SortDescription& get_mutable_sort_description() { return _sort_description; } + const VSortExecExprs& get_vsort_exec_exprs() const { return _vsort_exec_exprs; } #endif virtual ~Sorter() = default; diff --git a/be/src/exec/spill/spill_file.cpp b/be/src/exec/spill/spill_file.cpp new file mode 100644 index 00000000000000..78347ea78ebf9e --- /dev/null +++ b/be/src/exec/spill/spill_file.cpp @@ -0,0 +1,95 @@ +// 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 "exec/spill/spill_file.h" + +#include + +#include +#include +#include + +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" +#include "io/fs/local_file_system.h" +#include "runtime/exec_env.h" +#include "runtime/query_context.h" +#include "runtime/runtime_profile.h" +#include "runtime/runtime_state.h" +#include "util/debug_points.h" + +namespace doris { +#include "common/compile_check_begin.h" +SpillFile::SpillFile(SpillDataDir* data_dir, std::string relative_path) + : _data_dir(data_dir), + _spill_dir(data_dir->get_spill_data_path() + "/" + std::move(relative_path)) {} + +SpillFile::~SpillFile() { + gc(); +} + +void SpillFile::gc() { + bool exists = false; + auto status = io::global_local_filesystem()->exists(_spill_dir, &exists); + if (status.ok() && exists) { + // Delete spill directory directly instead of moving it to a GC directory. + // This simplifies cleanup and avoids retaining spill data under a GC path. + status = io::global_local_filesystem()->delete_directory(_spill_dir); + DBUG_EXECUTE_IF("fault_inject::spill_file::gc", { + status = Status::Error("fault_inject spill_file gc failed"); + }); + if (!status.ok()) { + LOG_EVERY_T(WARNING, 1) << fmt::format("failed to delete spill data, dir {}, error: {}", + _spill_dir, status.to_string()); + } + } + // decrease spill data usage anyway, since in ~QueryContext() spill data of the query will be + // clean up as a last resort + _data_dir->update_spill_data_usage(-_total_written_bytes); + _total_written_bytes = 0; +} + +Status SpillFile::create_writer(RuntimeState* state, RuntimeProfile* profile, + SpillFileWriterSPtr& writer) { + writer = std::make_shared(shared_from_this(), state, profile, _data_dir, + _spill_dir); + // _active_writer is set in SpillFileWriter constructor via the shared_ptr + return Status::OK(); +} + +SpillFileReaderSPtr SpillFile::create_reader(RuntimeState* state, RuntimeProfile* profile) const { + // It's a programming error to create a reader while a writer is still active. + DCHECK(_active_writer == nullptr) << "create_reader() called while writer still active"; + return std::make_shared(state, profile, _spill_dir, _part_count); +} + +void SpillFile::finish_writing() { + _ready_for_reading = true; + // writer finished; clear active writer pointer + _active_writer = nullptr; +} + +void SpillFile::update_written_bytes(int64_t delta_bytes) { + _total_written_bytes += delta_bytes; +} + +void SpillFile::increment_part_count() { + ++_part_count; +} + +} // namespace doris diff --git a/be/src/exec/spill/spill_file.h b/be/src/exec/spill/spill_file.h new file mode 100644 index 00000000000000..aa9b2287b44372 --- /dev/null +++ b/be/src/exec/spill/spill_file.h @@ -0,0 +1,114 @@ +// 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 "common/status.h" + +namespace doris { +#include "common/compile_check_begin.h" +class RuntimeProfile; +class RuntimeState; + +class Block; +class SpillDataDir; +class SpillFileWriter; +class SpillFileReader; +using SpillFileWriterSPtr = std::shared_ptr; +using SpillFileReaderSPtr = std::shared_ptr; + +/// SpillFile represents a logical spill file that may consist of multiple +/// physical "part" files on disk. Parts are managed automatically by +/// SpillFileWriter when a part exceeds the configured size threshold. +/// +/// On-disk layout: +/// spill_dir/ (created lazily by SpillFileWriter on first write) +/// +-- 0 (part 0) +/// +-- 1 (part 1) +/// +-- ... +/// +/// Writing workflow: +/// SpillFileWriterSPtr writer; +/// RETURN_IF_ERROR(spill_file->create_writer(state, profile, writer)); +/// RETURN_IF_ERROR(writer->write_block(state, block)); // auto-rotates parts +/// RETURN_IF_ERROR(writer->close()); // finalizes all parts +/// +/// Reading workflow: +/// auto reader = spill_file->create_reader(state, profile); +/// RETURN_IF_ERROR(reader->open()); +/// while (!eos) { RETURN_IF_ERROR(reader->read(&block, &eos)); } +class SpillFile : public std::enable_shared_from_this { +public: + // to avoid too many small file writes + static constexpr size_t MIN_SPILL_WRITE_BATCH_MEM = 512 * 1024; + static constexpr size_t MAX_SPILL_WRITE_BATCH_MEM = 32 * 1024 * 1024; + + /// @param data_dir The spill storage directory (disk) selected by SpillFileManager. + /// @param relative_path Relative path under the spill root, formatted by the operator. + /// e.g. "query_id/sort-node_id-task_id-unique_id" + SpillFile(SpillDataDir* data_dir, std::string relative_path); + + SpillFile() = delete; + SpillFile(const SpillFile&) = delete; + SpillFile& operator=(const SpillFile&) = delete; + + ~SpillFile(); + + void gc(); + + /// Returns true after the writer has been closed (all data flushed). + bool ready_for_reading() const { return _ready_for_reading; } + + /// Create a SpillFileWriter that automatically manages multi-part rotation. + /// Only one writer should exist per SpillFile at a time. + /// Part size threshold is read from config::spill_file_part_size_bytes. + Status create_writer(RuntimeState* state, RuntimeProfile* profile, SpillFileWriterSPtr& writer); + + /// Create a SpillFileReader that reads sequentially across all parts. + /// The caller should call reader->open() before reading. + SpillFileReaderSPtr create_reader(RuntimeState* state, RuntimeProfile* profile) const; + +private: + friend class SpillFileWriter; + friend class SpillFileManager; + + /// Called by SpillFileWriter::close() to mark writing as complete. + void finish_writing(); + + /// Called by SpillFileWriter to incrementally track bytes written to disk. + /// This ensures SpillFile always knows the correct _total_written_bytes for + /// gc() accounting, even if the writer's close() is never properly called. + void update_written_bytes(int64_t delta_bytes); + + /// Called by SpillFileWriter when a part file is completed. + void increment_part_count(); + + SpillDataDir* _data_dir = nullptr; + // Absolute path: data_dir->get_spill_data_path() + "/" + relative_path + std::string _spill_dir; + int64_t _total_written_bytes = 0; + size_t _part_count = 0; + bool _ready_for_reading = false; + // Pointer to the currently-active writer. Mutable to allow checks from const + // methods like create_reader(). Only one writer may be active at a time. + mutable SpillFileWriter* _active_writer = nullptr; +}; +using SpillFileSPtr = std::shared_ptr; +} // namespace doris +#include "common/compile_check_end.h" diff --git a/be/src/exec/spill/spill_stream_manager.cpp b/be/src/exec/spill/spill_file_manager.cpp similarity index 84% rename from be/src/exec/spill/spill_stream_manager.cpp rename to be/src/exec/spill/spill_file_manager.cpp index 38d42daa6bd4c5..0946e3209b7bec 100644 --- a/be/src/exec/spill/spill_stream_manager.cpp +++ b/be/src/exec/spill/spill_file_manager.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include #include @@ -23,34 +23,30 @@ #include #include #include -#include -#include #include #include "common/logging.h" #include "common/metrics/doris_metrics.h" -#include "exec/spill/spill_stream.h" +#include "exec/spill/spill_file.h" #include "io/fs/file_system.h" #include "io/fs/local_file_system.h" -#include "runtime/runtime_profile.h" -#include "runtime/runtime_state.h" #include "storage/olap_define.h" #include "util/parse_util.h" #include "util/pretty_printer.h" #include "util/time.h" -#include "util/uid_util.h" namespace doris { #include "common/compile_check_begin.h" -SpillStreamManager::~SpillStreamManager() { +SpillFileManager::~SpillFileManager() { DorisMetrics::instance()->metric_registry()->deregister_entity(_entity); } -SpillStreamManager::SpillStreamManager( + +SpillFileManager::SpillFileManager( std::unordered_map>&& spill_store_map) : _spill_store_map(std::move(spill_store_map)), _stop_background_threads_latch(1) {} -Status SpillStreamManager::init() { +Status SpillFileManager::init() { LOG(INFO) << "init spill stream manager"; RETURN_IF_ERROR(_init_spill_store_map()); @@ -87,7 +83,7 @@ Status SpillStreamManager::init() { return Status::OK(); } -void SpillStreamManager::_init_metrics() { +void SpillFileManager::_init_metrics() { _entity = DorisMetrics::instance()->metric_registry()->register_entity("spill", {{"name", "spill"}}); @@ -103,7 +99,7 @@ void SpillStreamManager::_init_metrics() { } // clean up stale spilled files -void SpillStreamManager::_spill_gc_thread_callback() { +void SpillFileManager::_spill_gc_thread_callback() { while (!_stop_background_threads_latch.wait_for( std::chrono::milliseconds(config::spill_gc_interval_ms))) { gc(config::spill_gc_work_time_ms); @@ -113,7 +109,7 @@ void SpillStreamManager::_spill_gc_thread_callback() { } } -Status SpillStreamManager::_init_spill_store_map() { +Status SpillFileManager::_init_spill_store_map() { for (const auto& store : _spill_store_map) { RETURN_IF_ERROR(store.second->init()); } @@ -121,7 +117,7 @@ Status SpillStreamManager::_init_spill_store_map() { return Status::OK(); } -std::vector SpillStreamManager::_get_stores_for_spill( +std::vector SpillFileManager::_get_stores_for_spill( TStorageMedium::type storage_medium) { std::vector> stores_with_usage; for (auto& [_, store] : _spill_store_map) { @@ -133,8 +129,7 @@ std::vector SpillStreamManager::_get_stores_for_spill( return {}; } - std::sort(stores_with_usage.begin(), stores_with_usage.end(), - [](auto&& a, auto&& b) { return a.second < b.second; }); + std::ranges::sort(stores_with_usage, [](auto&& a, auto&& b) { return a.second < b.second; }); std::vector stores; for (const auto& [store, _] : stores_with_usage) { @@ -143,11 +138,8 @@ std::vector SpillStreamManager::_get_stores_for_spill( return stores; } -Status SpillStreamManager::register_spill_stream(RuntimeState* state, SpillStreamSPtr& spill_stream, - const std::string& query_id, - const std::string& operator_name, int32_t node_id, - int32_t batch_rows, size_t batch_bytes, - RuntimeProfile* operator_profile) { +Status SpillFileManager::create_spill_file(const std::string& relative_path, + SpillFileSPtr& spill_file) { auto data_dirs = _get_stores_for_spill(TStorageMedium::type::SSD); if (data_dirs.empty()) { data_dirs = _get_stores_for_spill(TStorageMedium::type::HDD); @@ -157,37 +149,21 @@ Status SpillStreamManager::register_spill_stream(RuntimeState* state, SpillStrea "no available disk can be used for spill."); } - uint64_t id = id_++; - std::string spill_dir; - SpillDataDir* data_dir = nullptr; - for (auto& dir : data_dirs) { - std::string spill_root_dir = dir->get_spill_data_path(); - // storage_root/spill/query_id/partitioned_hash_join-node_id-task_id-stream_id - 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()) { - std::cerr << "create spill dir failed: " << st.to_string(); - continue; - } - data_dir = dir; - 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, operator_profile); - RETURN_IF_ERROR(spill_stream->prepare()); + // Select the first available data dir (sorted by usage ascending) + SpillDataDir* data_dir = data_dirs.front(); + spill_file = std::make_shared(data_dir, relative_path); return Status::OK(); } -void SpillStreamManager::delete_spill_stream(SpillStreamSPtr stream) { - stream->gc(); +void SpillFileManager::delete_spill_file(SpillFileSPtr spill_file) { + if (!spill_file) { + LOG(WARNING) << "[spill][delete] null spill_file"; + return; + } + spill_file->gc(); } -void SpillStreamManager::gc(int32_t max_work_time_ms) { +void SpillFileManager::gc(int32_t max_work_time_ms) { bool exists = true; bool has_work = false; int64_t max_work_time_ns = max_work_time_ms * 1000L * 1000L; @@ -343,9 +319,7 @@ Status SpillDataDir::update_capacity() { _spill_data_limit_bytes = (int64_t)(_spill_data_limit_bytes * config::storage_flood_stage_usage_percent / 100); } - if (_spill_data_limit_bytes > disk_use_max_bytes) { - _spill_data_limit_bytes = disk_use_max_bytes; - } + _spill_data_limit_bytes = std::min(_spill_data_limit_bytes, disk_use_max_bytes); spill_disk_limit->set_value(_spill_data_limit_bytes); std::string spill_root_dir = get_spill_data_path(); @@ -374,7 +348,8 @@ bool SpillDataDir::reach_capacity_limit(int64_t incoming_data_size) { } if (_spill_data_bytes + incoming_data_size > _spill_data_limit_bytes) { LOG_EVERY_T(WARNING, 1) << fmt::format( - "spill data reach limit, path: {}, capacity: {}, limit: {}, used: {}, available: " + "spill data reach limit, path: {}, capacity: {}, limit: {}, used: {}, " + "available: " "{}, " "incoming " "bytes: {}", diff --git a/be/src/exec/spill/spill_stream_manager.h b/be/src/exec/spill/spill_file_manager.h similarity index 87% rename from be/src/exec/spill/spill_stream_manager.h rename to be/src/exec/spill/spill_file_manager.h index 4ecd93a700abd9..582df3f1f63c9b 100644 --- a/be/src/exec/spill/spill_stream_manager.h +++ b/be/src/exec/spill/spill_file_manager.h @@ -24,9 +24,10 @@ #include #include "common/metrics/metrics.h" -#include "exec/spill/spill_stream.h" +#include "exec/spill/spill_file.h" #include "storage/options.h" #include "util/threadpool.h" + namespace doris { #include "common/compile_check_begin.h" class RuntimeProfile; @@ -39,7 +40,7 @@ using UIntGauge = AtomicGauge; class MetricEntity; struct MetricPrototype; -class SpillStreamManager; +class SpillFileManager; class SpillDataDir { public: SpillDataDir(std::string path, int64_t capacity_bytes, @@ -88,7 +89,7 @@ class SpillDataDir { (double)_disk_capacity_bytes; } - friend class SpillStreamManager; + friend class SpillFileManager; std::string _path; // protect _disk_capacity_bytes, _available_bytes, _spill_data_limit_bytes, _spill_data_bytes @@ -110,10 +111,10 @@ class SpillDataDir { IntGauge* spill_disk_has_spill_data = nullptr; IntGauge* spill_disk_has_spill_gc_data = nullptr; }; -class SpillStreamManager { +class SpillFileManager { public: - ~SpillStreamManager(); - SpillStreamManager( + ~SpillFileManager(); + SpillFileManager( std::unordered_map>&& spill_store_map); Status init(); @@ -125,14 +126,16 @@ class SpillStreamManager { } } - // 创建SpillStream并登记 - Status register_spill_stream(RuntimeState* state, SpillStreamSPtr& spill_stream, - const std::string& query_id, const std::string& operator_name, - int32_t node_id, int32_t batch_rows, size_t batch_bytes, - RuntimeProfile* operator_profile); + // Create SpillFile and register it + // @param relative_path Operator-formatted path under the spill root, + // e.g. "query_id/sort-node_id-task_id-unique_id" + Status create_spill_file(const std::string& relative_path, SpillFileSPtr& spill_file); + + /// Get a unique ID for constructing spill file paths. + uint64_t next_id() { return id_++; } - // 标记SpillStream需要被删除,在GC线程中异步删除落盘文件 - void delete_spill_stream(SpillStreamSPtr spill_stream); + // Mark SpillFile for deletion; asynchronously delete spill files in the GC thread + void delete_spill_file(SpillFileSPtr spill_file); void gc(int32_t max_work_time_ms); diff --git a/be/src/exec/spill/spill_file_reader.cpp b/be/src/exec/spill/spill_file_reader.cpp new file mode 100644 index 00000000000000..50609d93f4253d --- /dev/null +++ b/be/src/exec/spill/spill_file_reader.cpp @@ -0,0 +1,237 @@ +// 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 "exec/spill/spill_file_reader.h" + +#include + +#include + +#include "common/cast_set.h" +#include "common/exception.h" +#include "core/block/block.h" +#include "exec/spill/spill_file_manager.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 "runtime/query_context.h" +#include "runtime/runtime_state.h" +#include "util/debug_points.h" +#include "util/slice.h" +namespace doris { +#include "common/compile_check_begin.h" +namespace io { +class FileSystem; +} // namespace io + +SpillFileReader::SpillFileReader(RuntimeState* state, RuntimeProfile* profile, + std::string spill_dir, size_t part_count) + : _spill_dir(std::move(spill_dir)), + _part_count(part_count), + _resource_ctx(state->get_query_ctx()->resource_ctx()) { + // Internalize counter setup + RuntimeProfile* custom_profile = profile->get_child("CustomCounters"); + DCHECK(custom_profile != nullptr); + _read_file_timer = custom_profile->get_counter("SpillReadFileTime"); + _deserialize_timer = custom_profile->get_counter("SpillReadDerializeBlockTime"); + _read_block_count = custom_profile->get_counter("SpillReadBlockCount"); + _read_block_data_size = custom_profile->get_counter("SpillReadBlockBytes"); + _read_file_size = custom_profile->get_counter("SpillReadFileBytes"); + _read_rows_count = custom_profile->get_counter("SpillReadRows"); + _read_file_count = custom_profile->get_counter("SpillReadFileCount"); +} + +Status SpillFileReader::open() { + if (_is_open || _part_count == 0) { + return Status::OK(); + } + RETURN_IF_ERROR(_open_part(0)); + _is_open = true; + return Status::OK(); +} + +Status SpillFileReader::_open_part(size_t part_index) { + _close_current_part(); + + _current_part_index = part_index; + _part_opened = true; + std::string part_path = _spill_dir + "/" + std::to_string(part_index); + + SCOPED_TIMER(_read_file_timer); + COUNTER_UPDATE(_read_file_count, 1); + RETURN_IF_ERROR(io::global_local_filesystem()->open_file(part_path, &_file_reader)); + + size_t file_size = _file_reader->size(); + DCHECK(file_size >= 16); // max_sub_block_size + block count + + Slice result((char*)&_part_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); + + // read max sub block size + bytes_read = 0; + result.data = (char*)&_part_max_sub_block_size; + RETURN_IF_ERROR(_file_reader->read_at(file_size - sizeof(size_t) * 2, result, &bytes_read)); + DCHECK(bytes_read == 8); + + // The buffer is used for two purposes: + // 1. Reading the block start offsets array (needs _part_block_count * sizeof(size_t) bytes) + // 2. Reading a single block's serialized data (needs up to _part_max_sub_block_size bytes) + // We must ensure the buffer is large enough for either case, so take the maximum. + size_t buff_size = std::max(_part_block_count * sizeof(size_t), _part_max_sub_block_size); + if (buff_size > _read_buff.size()) { + _read_buff.reserve(buff_size); + } + + // Read the block start offsets array from the end of the file. + // The file layout (from end backwards) is: + // [block count (size_t)] + // [max sub block size (size_t)] + // [block start offsets array (_part_block_count * size_t)] + // So the offsets array starts at: + // file_size - (_part_block_count + 2) * sizeof(size_t) + size_t read_offset = file_size - (_part_block_count + 2) * sizeof(size_t); + result.data = _read_buff.data(); + result.size = _part_block_count * sizeof(size_t); + + RETURN_IF_ERROR(_file_reader->read_at(read_offset, result, &bytes_read)); + DCHECK(bytes_read == _part_block_count * sizeof(size_t)); + + _block_start_offsets.resize(_part_block_count + 1); + for (size_t i = 0; i < _part_block_count; ++i) { + _block_start_offsets[i] = *(size_t*)(result.data + i * sizeof(size_t)); + } + _block_start_offsets[_part_block_count] = file_size - (_part_block_count + 2) * sizeof(size_t); + + _part_read_block_index = 0; + return Status::OK(); +} + +void SpillFileReader::_close_current_part() { + if (_file_reader) { + (void)_file_reader->close(); + _file_reader.reset(); + } + _part_block_count = 0; + _part_read_block_index = 0; + _part_max_sub_block_size = 0; + _block_start_offsets.clear(); + _part_opened = false; +} + +Status SpillFileReader::read(Block* block, bool* eos) { + DBUG_EXECUTE_IF("fault_inject::spill_file::read_next_block", { + return Status::InternalError("fault_inject spill_file read_next_block failed"); + }); + block->clear_column_data(); + + if (_part_count == 0) { + *eos = true; + return Status::OK(); + } + + // Advance to next part if current part is exhausted + while (_part_read_block_index >= _part_block_count) { + size_t next_part = _part_opened ? _current_part_index + 1 : 0; + if (next_part >= _part_count) { + *eos = true; + return Status::OK(); + } + RETURN_IF_ERROR(_open_part(next_part)); + } + + size_t bytes_to_read = _block_start_offsets[_part_read_block_index + 1] - + _block_start_offsets[_part_read_block_index]; + + if (bytes_to_read == 0) { + ++_part_read_block_index; + *eos = false; + return Status::OK(); + } + + Slice result(_read_buff.data(), bytes_to_read); + size_t bytes_read = 0; + { + SCOPED_TIMER(_read_file_timer); + RETURN_IF_ERROR(_file_reader->read_at(_block_start_offsets[_part_read_block_index], result, + &bytes_read)); + } + DCHECK(bytes_read == bytes_to_read); + + if (bytes_read > 0) { + COUNTER_UPDATE(_read_file_size, bytes_read); + ExecEnv::GetInstance()->spill_file_mgr()->update_spill_read_bytes(bytes_read); + if (_resource_ctx) { + _resource_ctx->io_context()->update_spill_read_bytes_from_local_storage(bytes_read); + } + COUNTER_UPDATE(_read_block_count, 1); + { + SCOPED_TIMER(_deserialize_timer); + if (!_pb_block.ParseFromArray(result.data, cast_set(result.size))) { + return Status::InternalError("Failed to read spilled block"); + } + size_t uncompressed_size = 0; + int64_t uncompressed_time = 0; + RETURN_IF_ERROR(block->deserialize(_pb_block, &uncompressed_size, &uncompressed_time)); + } + COUNTER_UPDATE(_read_block_data_size, block->bytes()); + COUNTER_UPDATE(_read_rows_count, block->rows()); + } else { + block->clear_column_data(); + } + + ++_part_read_block_index; + *eos = false; + return Status::OK(); +} + +Status SpillFileReader::seek(size_t block_index) { + return _seek_to_block(block_index); +} + +Status SpillFileReader::_seek_to_block(size_t block_index) { + if (_part_count == 0) { + return Status::OK(); + } + + size_t remaining = block_index; + for (size_t part_index = 0; part_index < _part_count; ++part_index) { + RETURN_IF_ERROR(_open_part(part_index)); + if (remaining < _part_block_count) { + _part_read_block_index = remaining; + return Status::OK(); + } + remaining -= _part_block_count; + } + + // block_index is out of range: position reader at EOS. + RETURN_IF_ERROR(_open_part(_part_count - 1)); + _part_read_block_index = _part_block_count; + return Status::OK(); +} + +Status SpillFileReader::close() { + _close_current_part(); + _is_open = false; + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/exec/spill/spill_file_reader.h b/be/src/exec/spill/spill_file_reader.h new file mode 100644 index 00000000000000..beca6cc7c44c22 --- /dev/null +++ b/be/src/exec/spill/spill_file_reader.h @@ -0,0 +1,111 @@ +// 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 "core/pod_array.h" +#include "core/pod_array_fwd.h" +#include "io/fs/file_reader_writer_fwd.h" +#include "runtime/runtime_profile.h" +#include "runtime/workload_management/resource_context.h" + +namespace doris { +#include "common/compile_check_begin.h" +class RuntimeState; +class Block; + +/// SpillFileReader reads blocks sequentially across all parts of a SpillFile. +/// +/// Usage: +/// auto reader = spill_file->create_reader(state, profile); +/// RETURN_IF_ERROR(reader->open()); +/// bool eos = false; +/// while (!eos) { RETURN_IF_ERROR(reader->read(&block, &eos)); } +/// +/// Part boundaries are transparent to the caller. When the current part is +/// exhausted, the reader automatically opens the next part. +class SpillFileReader { +public: + SpillFileReader(RuntimeState* state, RuntimeProfile* profile, std::string spill_dir, + size_t part_count); + + ~SpillFileReader() { (void)close(); } + + /// Open the first part and read its footer metadata. + Status open(); + + /// Read the next block. Automatically advances across part boundaries. + /// Sets *eos = true when all parts are exhausted. + Status read(Block* block, bool* eos); + + /// Seek to a global block index within the whole spill file. + /// block_index is 0-based across all parts. + /// If block_index is out of range, the reader is positioned at EOS. + Status seek(size_t block_index); + + Status close(); + +private: + /// Open a specific part file and read its footer. + Status _open_part(size_t part_index); + + /// Seek implementation with status propagation. + Status _seek_to_block(size_t block_index); + + /// Close the current part's file reader. + void _close_current_part(); + + // ── Configuration ── + std::string _spill_dir; + size_t _part_count; + + // ── Current part state ── + size_t _current_part_index = 0; + bool _is_open = false; + bool _part_opened = false; + io::FileReaderSPtr _file_reader; + size_t _part_block_count = 0; + size_t _part_read_block_index = 0; + size_t _part_max_sub_block_size = 0; + PaddedPODArray _read_buff; + std::vector _block_start_offsets; + + PBlock _pb_block; + + // ── Counters ── + RuntimeProfile::Counter* _read_file_timer = nullptr; + RuntimeProfile::Counter* _deserialize_timer = nullptr; + RuntimeProfile::Counter* _read_block_count = nullptr; + RuntimeProfile::Counter* _read_block_data_size = nullptr; + RuntimeProfile::Counter* _read_file_size = nullptr; + RuntimeProfile::Counter* _read_rows_count = nullptr; + RuntimeProfile::Counter* _read_file_count = nullptr; + + std::shared_ptr _resource_ctx = nullptr; +}; + +using SpillFileReaderSPtr = std::shared_ptr; + +} // namespace doris +#include "common/compile_check_end.h" diff --git a/be/src/exec/spill/spill_file_writer.cpp b/be/src/exec/spill/spill_file_writer.cpp new file mode 100644 index 00000000000000..ced813e51e3080 --- /dev/null +++ b/be/src/exec/spill/spill_file_writer.cpp @@ -0,0 +1,273 @@ +// 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 "exec/spill/spill_file_writer.h" + +#include "agent/be_exec_version_manager.h" +#include "common/config.h" +#include "common/status.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "io/fs/local_file_system.h" +#include "io/fs/local_file_writer.h" +#include "runtime/exec_env.h" +#include "runtime/query_context.h" +#include "runtime/runtime_state.h" +#include "runtime/thread_context.h" + +namespace doris { +#include "common/compile_check_begin.h" + +SpillFileWriter::SpillFileWriter(const std::shared_ptr& spill_file, RuntimeState* state, + RuntimeProfile* profile, SpillDataDir* data_dir, + const std::string& spill_dir) + : _spill_file_wptr(spill_file), + _data_dir(data_dir), + _spill_dir(spill_dir), + _max_part_size(config::spill_file_part_size_bytes), + _resource_ctx(state->get_query_ctx()->resource_ctx()) { + // Common counters + RuntimeProfile* common_profile = profile->get_child("CommonCounters"); + DCHECK(common_profile != nullptr); + _memory_used_counter = common_profile->get_counter("MemoryUsage"); + + // Register this writer as the active writer for the SpillFile. + spill_file->_active_writer = this; + + // Custom (spill-specific) counters + RuntimeProfile* custom_profile = profile->get_child("CustomCounters"); + _write_file_timer = custom_profile->get_counter("SpillWriteFileTime"); + _serialize_timer = custom_profile->get_counter("SpillWriteSerializeBlockTime"); + _write_block_counter = custom_profile->get_counter("SpillWriteBlockCount"); + _write_block_bytes_counter = custom_profile->get_counter("SpillWriteBlockBytes"); + _write_file_total_size = custom_profile->get_counter("SpillWriteFileBytes"); + _write_file_current_size = custom_profile->get_counter("SpillWriteFileCurrentBytes"); + _write_rows_counter = custom_profile->get_counter("SpillWriteRows"); + _total_file_count = custom_profile->get_counter("SpillWriteFileTotalCount"); +} + +SpillFileWriter::~SpillFileWriter() { + if (_closed) { + return; + } + Status st = close(); + if (!st.ok()) { + LOG(WARNING) << "SpillFileWriter::~SpillFileWriter() failed: " << st.to_string() + << ", spill_dir=" << _spill_dir; + } +} + +Status SpillFileWriter::_open_next_part() { + _current_part_path = _spill_dir + "/" + std::to_string(_current_part_index); + // Create the spill directory lazily on first part + if (_current_part_index == 0) { + RETURN_IF_ERROR(io::global_local_filesystem()->create_directory(_spill_dir)); + } + RETURN_IF_ERROR(io::global_local_filesystem()->create_file(_current_part_path, &_file_writer)); + COUNTER_UPDATE(_total_file_count, 1); + return Status::OK(); +} + +Status SpillFileWriter::_close_current_part(const std::shared_ptr& spill_file) { + if (!_file_writer) { + return Status::OK(); + } + + // Write footer: block offsets + max_sub_block_size + block_count + _part_meta.append((const char*)&_part_max_sub_block_size, sizeof(_part_max_sub_block_size)); + _part_meta.append((const char*)&_part_written_blocks, sizeof(_part_written_blocks)); + + { + SCOPED_TIMER(_write_file_timer); + RETURN_IF_ERROR(_file_writer->append(_part_meta)); + } + + int64_t meta_size = _part_meta.size(); + _part_written_bytes += meta_size; + _total_written_bytes += meta_size; + COUNTER_UPDATE(_write_file_total_size, meta_size); + if (_resource_ctx) { + _resource_ctx->io_context()->update_spill_write_bytes_to_local_storage(meta_size); + } + if (_write_file_current_size) { + COUNTER_UPDATE(_write_file_current_size, meta_size); + } + _data_dir->update_spill_data_usage(meta_size); + ExecEnv::GetInstance()->spill_file_mgr()->update_spill_write_bytes(meta_size); + // Incrementally update SpillFile's accounting so gc() can always + // decrement the correct amount, even if close() is never called. + if (spill_file) { + spill_file->update_written_bytes(meta_size); + } + + RETURN_IF_ERROR(_file_writer->close()); + _file_writer.reset(); + + // Advance to next part + ++_current_part_index; + ++_total_parts; + if (spill_file) { + spill_file->increment_part_count(); + } + _part_written_blocks = 0; + _part_written_bytes = 0; + _part_max_sub_block_size = 0; + _part_meta.clear(); + + return Status::OK(); +} + +Status SpillFileWriter::_rotate_if_needed(const std::shared_ptr& spill_file) { + if (_file_writer && _part_written_bytes >= _max_part_size) { + RETURN_IF_ERROR(_close_current_part(spill_file)); + } + return Status::OK(); +} + +Status SpillFileWriter::write_block(RuntimeState* state, const Block& block) { + DCHECK(!_closed); + + // Lock the SpillFile to ensure it is still alive. If it has already been + // destroyed (gc'd), we must not write any more data because the disk + // accounting would be out of sync. + auto spill_file = _spill_file_wptr.lock(); + if (!spill_file) { + return Status::Error( + "SpillFile has been destroyed, cannot write more data, spill_dir={}", _spill_dir); + } + + // Lazily open the first part + if (!_file_writer) { + RETURN_IF_ERROR(_open_next_part()); + } + + DBUG_EXECUTE_IF("fault_inject::spill_file::spill_block", { + return Status::Error("fault_inject spill_file spill_block failed"); + }); + + auto rows = block.rows(); + COUNTER_UPDATE(_write_rows_counter, rows); + COUNTER_UPDATE(_write_block_bytes_counter, block.bytes()); + + RETURN_IF_ERROR(_write_internal(block, spill_file)); + + // Auto-rotate if current part is full + return _rotate_if_needed(spill_file); +} + +Status SpillFileWriter::close() { + if (_closed) { + return Status::OK(); + } + _closed = true; + + DBUG_EXECUTE_IF("fault_inject::spill_file::spill_eof", { + return Status::Error("fault_inject spill_file spill_eof failed"); + }); + + auto spill_file = _spill_file_wptr.lock(); + RETURN_IF_ERROR(_close_current_part(spill_file)); + + if (spill_file) { + if (spill_file->_active_writer != this) { + return Status::Error( + "SpillFileWriter close() called but not registered as active writer, possible " + "double close or logic error"); + } + spill_file->finish_writing(); + } + + return Status::OK(); +} + +Status SpillFileWriter::_write_internal(const Block& block, + const std::shared_ptr& spill_file) { + size_t uncompressed_bytes = 0, compressed_bytes = 0; + + Status status; + std::string buff; + int64_t buff_size {0}; + + if (block.rows() > 0) { + { + PBlock pblock; + SCOPED_TIMER(_serialize_timer); + int64_t compressed_time = 0; + status = block.serialize( + BeExecVersionManager::get_newest_version(), &pblock, &uncompressed_bytes, + &compressed_bytes, &compressed_time, + segment_v2::CompressionTypePB::ZSTD); // ZSTD for better compression ratio + RETURN_IF_ERROR(status); + int64_t pblock_mem = pblock.ByteSizeLong(); + COUNTER_UPDATE(_memory_used_counter, pblock_mem); + Defer defer {[&]() { COUNTER_UPDATE(_memory_used_counter, -pblock_mem); }}; + if (!pblock.SerializeToString(&buff)) { + return Status::Error( + "serialize spill data error. [path={}]", _current_part_path); + } + buff_size = buff.size(); + COUNTER_UPDATE(_memory_used_counter, buff_size); + Defer defer2 {[&]() { COUNTER_UPDATE(_memory_used_counter, -buff_size); }}; + } + if (_data_dir->reach_capacity_limit(buff_size)) { + return Status::Error( + "spill data total size exceed limit, path: {}, size limit: {}, spill data " + "size: {}", + _data_dir->path(), + PrettyPrinter::print_bytes(_data_dir->get_spill_data_limit()), + PrettyPrinter::print_bytes(_data_dir->get_spill_data_bytes())); + } + + { + Defer defer {[&]() { + if (status.ok()) { + _data_dir->update_spill_data_usage(buff_size); + ExecEnv::GetInstance()->spill_file_mgr()->update_spill_write_bytes(buff_size); + + _part_max_sub_block_size = + std::max(_part_max_sub_block_size, (size_t)buff_size); + + _part_meta.append((const char*)&_part_written_bytes, sizeof(size_t)); + COUNTER_UPDATE(_write_file_total_size, buff_size); + if (_resource_ctx) { + _resource_ctx->io_context()->update_spill_write_bytes_to_local_storage( + buff_size); + } + if (_write_file_current_size) { + COUNTER_UPDATE(_write_file_current_size, buff_size); + } + COUNTER_UPDATE(_write_block_counter, 1); + _part_written_bytes += buff_size; + _total_written_bytes += buff_size; + ++_part_written_blocks; + // Incrementally update SpillFile so gc() can always + // decrement the correct amount from _data_dir. + spill_file->update_written_bytes(buff_size); + } + }}; + { + SCOPED_TIMER(_write_file_timer); + status = _file_writer->append(buff); + RETURN_IF_ERROR(status); + } + } + } + + return status; +} + +} // namespace doris \ No newline at end of file diff --git a/be/src/exec/spill/spill_file_writer.h b/be/src/exec/spill/spill_file_writer.h new file mode 100644 index 00000000000000..ba83ca8a5463b7 --- /dev/null +++ b/be/src/exec/spill/spill_file_writer.h @@ -0,0 +1,114 @@ +// 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 "core/block/block.h" +#include "io/fs/file_writer.h" +#include "runtime/runtime_profile.h" +#include "runtime/workload_management/resource_context.h" +namespace doris { +#include "common/compile_check_begin.h" +class RuntimeState; + +class SpillDataDir; +class SpillFile; + +/// SpillFileWriter writes blocks to a SpillFile, automatically managing +/// part-file rotation when a part exceeds the configured size threshold +/// (config::spill_file_part_size_bytes). +/// +/// Usage: +/// SpillFileWriterSPtr writer; +/// RETURN_IF_ERROR(spill_file->create_writer(state, profile, writer)); +/// RETURN_IF_ERROR(writer->write_block(state, block)); +/// RETURN_IF_ERROR(writer->close()); +/// +/// Part rotation is fully internal. Each part file has its own footer with +/// block offset metadata. Parts are named 0, 1, 2, ... within the SpillFile's +/// directory. +class SpillFileWriter { +public: + SpillFileWriter(const std::shared_ptr& spill_file, RuntimeState* state, + RuntimeProfile* profile, SpillDataDir* data_dir, const std::string& spill_dir); + + ~SpillFileWriter(); + + /// Write a block. Automatically opens the first part, splits large blocks, + /// and rotates to a new part when the current one exceeds max_part_size. + Status write_block(RuntimeState* state, const Block& block); + + /// Finalize: close the current part, record cumulative stats in SpillFile. + /// After close(), no more writes are allowed. + Status close(); + +private: + /// Open the next part file (spill_dir/{_current_part_index}). + Status _open_next_part(); + + /// Close the current part: write footer, close FileWriter, update stats. + Status _close_current_part(const std::shared_ptr& spill_file); + + /// If current part size >= _max_part_size, close it. + Status _rotate_if_needed(const std::shared_ptr& spill_file); + + /// Serialize and write a single block to the current part. + Status _write_internal(const Block& block, const std::shared_ptr& spill_file); + + // ── Back-reference ── + std::weak_ptr _spill_file_wptr; // weak ref; use lock() in close() + + // ── Configuration ── + SpillDataDir* _data_dir = nullptr; + std::string _spill_dir; + int64_t _max_part_size; + + // ── Current part state (reset on rotation) ── + size_t _current_part_index = 0; + std::string _current_part_path; + std::unique_ptr _file_writer; + size_t _part_written_blocks = 0; + int64_t _part_written_bytes = 0; + size_t _part_max_sub_block_size = 0; + std::string _part_meta; + + // ── Cumulative state ── + int64_t _total_written_bytes = 0; + size_t _total_parts = 0; + bool _closed = false; + + // ── Counters ── + RuntimeProfile::Counter* _write_file_timer = nullptr; + RuntimeProfile::Counter* _serialize_timer = nullptr; + RuntimeProfile::Counter* _write_block_counter = nullptr; + RuntimeProfile::Counter* _write_block_bytes_counter = nullptr; + RuntimeProfile::Counter* _write_file_total_size = nullptr; + RuntimeProfile::Counter* _write_file_current_size = nullptr; + RuntimeProfile::Counter* _write_rows_counter = nullptr; + RuntimeProfile::Counter* _memory_used_counter = nullptr; + RuntimeProfile::Counter* _total_file_count = nullptr; + + std::shared_ptr _resource_ctx = nullptr; +}; +using SpillFileWriterSPtr = std::shared_ptr; +} // namespace doris + +#include "common/compile_check_end.h" diff --git a/be/src/exec/spill/spill_reader.cpp b/be/src/exec/spill/spill_reader.cpp deleted file mode 100644 index 4604d93cdf13e3..00000000000000 --- a/be/src/exec/spill/spill_reader.cpp +++ /dev/null @@ -1,168 +0,0 @@ -// 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 "exec/spill/spill_reader.h" - -#include - -#include - -#include "common/cast_set.h" -#include "common/exception.h" -#include "core/block/block.h" -#include "exec/spill/spill_stream_manager.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" -namespace doris { -#include "common/compile_check_begin.h" -namespace io { -class FileSystem; -} // namespace io - -Status SpillReader::open() { - if (file_reader_) { - return Status::OK(); - } - - SCOPED_TIMER(_read_file_timer); - - COUNTER_UPDATE(_read_file_count, 1); - - RETURN_IF_ERROR(io::global_local_filesystem()->open_file(file_path_, &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)); - - size_t total_read_bytes = 0; - // 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 - total_read_bytes += bytes_read; - if (_resource_ctx) { - _resource_ctx->io_context()->update_spill_write_bytes_to_local_storage(bytes_read); - } - - // 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 - total_read_bytes += bytes_read; - if (_resource_ctx) { - _resource_ctx->io_context()->update_spill_write_bytes_to_local_storage(bytes_read); - } - - size_t buff_size = std::max(block_count_ * sizeof(size_t), max_sub_block_size_); - read_buff_.reserve(buff_size); - - // read block start offsets - size_t read_offset = file_size - (block_count_ + 2) * sizeof(size_t); - result.data = read_buff_.data(); - 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)); - total_read_bytes += bytes_read; - COUNTER_UPDATE(_read_file_size, total_read_bytes); - ExecEnv::GetInstance()->spill_stream_mgr()->update_spill_read_bytes(total_read_bytes); - if (_resource_ctx) { - _resource_ctx->io_context()->update_spill_read_bytes_from_local_storage(bytes_read); - } - - 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(); -} - -void SpillReader::seek(size_t block_index) { - DCHECK_LT(block_index, block_count_); - read_block_index_ = block_index; -} - -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_.data(), bytes_to_read); - size_t bytes_read = 0; - { - SCOPED_TIMER(_read_file_timer); - RETURN_IF_ERROR(file_reader_->read_at(block_start_offsets_[read_block_index_], result, - &bytes_read)); - } - DCHECK(bytes_read == bytes_to_read); - - if (bytes_read > 0) { - COUNTER_UPDATE(_read_file_size, bytes_read); - ExecEnv::GetInstance()->spill_stream_mgr()->update_spill_read_bytes(bytes_read); - if (_resource_ctx) { - _resource_ctx->io_context()->update_spill_read_bytes_from_local_storage(bytes_read); - } - COUNTER_UPDATE(_read_block_count, 1); - { - SCOPED_TIMER(_deserialize_timer); - if (!pb_block_.ParseFromArray(result.data, cast_set(result.size))) { - return Status::InternalError("Failed to read spilled block"); - } - size_t uncompressed_size = 0; - int64_t uncompressed_time = 0; - RETURN_IF_ERROR(block->deserialize(pb_block_, &uncompressed_size, &uncompressed_time)); - } - COUNTER_UPDATE(_read_block_data_size, block->bytes()); - COUNTER_UPDATE(_read_rows_count, block->rows()); - } 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 doris \ No newline at end of file diff --git a/be/src/exec/spill/spill_reader.h b/be/src/exec/spill/spill_reader.h deleted file mode 100644 index d787cf1735408b..00000000000000 --- a/be/src/exec/spill/spill_reader.h +++ /dev/null @@ -1,99 +0,0 @@ -// 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 "core/pod_array.h" -#include "core/pod_array_fwd.h" -#include "io/fs/file_reader_writer_fwd.h" -#include "runtime/runtime_profile.h" -#include "runtime/workload_management/resource_context.h" - -namespace doris { -#include "common/compile_check_begin.h" -class Block; -class SpillReader { -public: - SpillReader(std::shared_ptr resource_context, int64_t stream_id, - std::string file_path) - : stream_id_(stream_id), - file_path_(std::move(file_path)), - _resource_ctx(std::move(resource_context)) {} - - ~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* operator_profile) { - RuntimeProfile* custom_profile = operator_profile->get_child("CustomCounters"); - DCHECK(custom_profile != nullptr); - _read_file_timer = custom_profile->get_counter("SpillReadFileTime"); - _deserialize_timer = custom_profile->get_counter("SpillReadDeserializeBlockTime"); - _read_block_count = custom_profile->get_counter("SpillReadBlockCount"); - _read_block_data_size = custom_profile->get_counter("SpillReadBlockBytes"); - _read_file_size = custom_profile->get_counter("SpillReadFileBytes"); - _read_rows_count = custom_profile->get_counter("SpillReadRows"); - _read_file_count = custom_profile->get_counter("SpillReadFileCount"); - } - -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; - PaddedPODArray read_buff_; - std::vector block_start_offsets_; - - PBlock pb_block_; - - RuntimeProfile::Counter* _read_file_timer = nullptr; - RuntimeProfile::Counter* _deserialize_timer = nullptr; - RuntimeProfile::Counter* _read_block_count = nullptr; - RuntimeProfile::Counter* _read_block_data_size = nullptr; - RuntimeProfile::Counter* _read_file_size = nullptr; - RuntimeProfile::Counter* _read_rows_count = nullptr; - RuntimeProfile::Counter* _read_file_count = nullptr; - - std::shared_ptr _resource_ctx = nullptr; -}; - -using SpillReaderUPtr = std::unique_ptr; - -} // namespace doris -#include "common/compile_check_end.h" diff --git a/be/src/exec/spill/spill_repartitioner.cpp b/be/src/exec/spill/spill_repartitioner.cpp new file mode 100644 index 00000000000000..de7d225c69ecee --- /dev/null +++ b/be/src/exec/spill/spill_repartitioner.cpp @@ -0,0 +1,371 @@ +// 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 "exec/spill/spill_repartitioner.h" + +#include + +#include +#include +#include + +#include "core/block/block.h" +#include "core/column/column.h" +#include "exec/partitioner/partitioner.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_profile.h" +#include "runtime/runtime_state.h" +#include "util/uid_util.h" + +namespace doris { +#include "common/compile_check_begin.h" + +void SpillRepartitioner::init(std::unique_ptr partitioner, RuntimeProfile* profile, + int fanout, int repartition_level) { + _partitioner = std::move(partitioner); + _use_column_index_mode = false; + _fanout = fanout; + _repartition_level = repartition_level; + _operator_profile = profile; + _repartition_timer = ADD_TIMER_WITH_LEVEL(profile, "SpillRepartitionTime", 1); + _repartition_rows = ADD_COUNTER_WITH_LEVEL(profile, "SpillRepartitionRows", TUnit::UNIT, 1); +} + +void SpillRepartitioner::init_with_key_columns(std::vector key_column_indices, + std::vector key_data_types, + RuntimeProfile* profile, int fanout, + int repartition_level) { + _key_column_indices = std::move(key_column_indices); + _key_data_types = std::move(key_data_types); + _use_column_index_mode = true; + _partitioner.reset(); + _fanout = fanout; + _repartition_level = repartition_level; + _operator_profile = profile; + _repartition_timer = ADD_TIMER_WITH_LEVEL(profile, "SpillRepartitionTime", 1); + _repartition_rows = ADD_COUNTER_WITH_LEVEL(profile, "SpillRepartitionRows", TUnit::UNIT, 1); +} + +Status SpillRepartitioner::setup_output(RuntimeState* state, + std::vector& output_spill_files) { + DCHECK_EQ(output_spill_files.size(), _fanout); + _output_spill_files = &output_spill_files; + _output_writers.resize(_fanout); + for (int i = 0; i < _fanout; ++i) { + RETURN_IF_ERROR( + output_spill_files[i]->create_writer(state, _operator_profile, _output_writers[i])); + } + // Reset reader state from any previous repartition session + _input_reader.reset(); + _current_input_file.reset(); + return Status::OK(); +} + +Status SpillRepartitioner::repartition(RuntimeState* state, SpillFileSPtr& input_spill_file, + bool* done) { + DCHECK(_output_spill_files != nullptr) << "setup_output() must be called first"; + SCOPED_TIMER(_repartition_timer); + + *done = false; + size_t accumulated_bytes = 0; + + // Create or reuse input reader. If the input file changed, create a new reader. + if (_current_input_file != input_spill_file) { + _current_input_file = input_spill_file; + _input_reader = input_spill_file->create_reader(state, _operator_profile); + RETURN_IF_ERROR(_input_reader->open()); + } + + // Per-partition write buffers to batch small writes + std::vector> output_buffers(_fanout); + + bool eos = false; + while (!eos && !state->is_cancelled()) { + Block block; + RETURN_IF_ERROR(_input_reader->read(&block, &eos)); + + if (block.empty()) { + continue; + } + + accumulated_bytes += block.allocated_bytes(); + COUNTER_UPDATE(_repartition_rows, block.rows()); + + if (_use_column_index_mode) { + RETURN_IF_ERROR(_route_block_by_columns(state, block, output_buffers)); + } else { + RETURN_IF_ERROR(_route_block(state, block, output_buffers)); + } + + // Yield after processing MAX_BATCH_BYTES to let pipeline scheduler re-schedule + if (accumulated_bytes >= MAX_BATCH_BYTES && !eos) { + break; + } + } + + // Flush all remaining buffers + RETURN_IF_ERROR(_flush_all_buffers(state, output_buffers, /*force=*/true)); + + if (eos) { + *done = true; + // Reset reader for this input file + _input_reader.reset(); + _current_input_file.reset(); + } + + return Status::OK(); +} + +Status SpillRepartitioner::repartition(RuntimeState* state, SpillFileReaderSPtr& reader, + bool* done) { + DCHECK(_output_spill_files != nullptr) << "setup_output() must be called first"; + DCHECK(reader != nullptr) << "reader must not be null"; + SCOPED_TIMER(_repartition_timer); + + *done = false; + size_t accumulated_bytes = 0; + + // Per-partition write buffers to batch small writes + std::vector> output_buffers(_fanout); + + bool eos = false; + while (!eos && !state->is_cancelled()) { + Block block; + RETURN_IF_ERROR(reader->read(&block, &eos)); + + if (block.empty()) { + continue; + } + + accumulated_bytes += block.allocated_bytes(); + COUNTER_UPDATE(_repartition_rows, block.rows()); + + if (_use_column_index_mode) { + RETURN_IF_ERROR(_route_block_by_columns(state, block, output_buffers)); + } else { + RETURN_IF_ERROR(_route_block(state, block, output_buffers)); + } + + // Yield after processing MAX_BATCH_BYTES to let pipeline scheduler re-schedule + if (accumulated_bytes >= MAX_BATCH_BYTES && !eos) { + break; + } + } + + // Flush all remaining buffers + RETURN_IF_ERROR(_flush_all_buffers(state, output_buffers, /*force=*/true)); + + if (eos) { + *done = true; + reader.reset(); + } + + return Status::OK(); +} + +Status SpillRepartitioner::route_block(RuntimeState* state, Block& block) { + DCHECK(_output_spill_files != nullptr) << "setup_output() must be called first"; + if (UNLIKELY(_output_spill_files == nullptr)) { + return Status::InternalError("SpillRepartitioner::setup_output() must be called first"); + } + SCOPED_TIMER(_repartition_timer); + + if (block.empty()) { + return Status::OK(); + } + + COUNTER_UPDATE(_repartition_rows, block.rows()); + + std::vector> output_buffers(_fanout); + if (_use_column_index_mode) { + RETURN_IF_ERROR(_route_block_by_columns(state, block, output_buffers)); + } else { + RETURN_IF_ERROR(_route_block(state, block, output_buffers)); + } + RETURN_IF_ERROR(_flush_all_buffers(state, output_buffers, /*force=*/true)); + return Status::OK(); +} + +Status SpillRepartitioner::finalize() { + DCHECK(_output_spill_files != nullptr) << "setup_output() must be called first"; + if (UNLIKELY(_output_spill_files == nullptr)) { + return Status::InternalError("SpillRepartitioner::setup_output() must be called first"); + } + // Close all writers (Writer::close() automatically updates SpillFile stats) + for (int i = 0; i < _fanout; ++i) { + if (_output_writers[i]) { + RETURN_IF_ERROR(_output_writers[i]->close()); + } + } + _output_writers.clear(); + _output_spill_files = nullptr; + _input_reader.reset(); + _current_input_file.reset(); + return Status::OK(); +} + +Status SpillRepartitioner::create_output_spill_files( + RuntimeState* state, int node_id, const std::string& label_prefix, int fanout, + std::vector& output_spill_files) { + output_spill_files.resize(fanout); + for (int i = 0; i < fanout; ++i) { + auto relative_path = fmt::format("{}/{}_sub{}-{}-{}-{}", print_id(state->query_id()), + label_prefix, i, node_id, state->task_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + relative_path, output_spill_files[i])); + } + return Status::OK(); +} + +Status SpillRepartitioner::_route_block( + RuntimeState* state, Block& block, + std::vector>& output_buffers) { + // Compute raw hash values for every row in the block. + RETURN_IF_ERROR(_partitioner->do_partitioning(state, &block)); + const auto& hash_vals = _partitioner->get_channel_ids(); + const auto rows = block.rows(); + + // Build per-partition row index lists + std::vector> partition_row_indexes(_fanout); + for (uint32_t i = 0; i < rows; ++i) { + auto partition_idx = _map_hash_to_partition(hash_vals[i]); + partition_row_indexes[partition_idx].emplace_back(i); + } + + // Scatter rows into per-partition buffers + for (int p = 0; p < _fanout; ++p) { + if (partition_row_indexes[p].empty()) { + continue; + } + + // Lazily initialize the buffer + if (!output_buffers[p]) { + output_buffers[p] = MutableBlock::create_unique(block.clone_empty()); + } + + RETURN_IF_ERROR(output_buffers[p]->add_rows( + &block, partition_row_indexes[p].data(), + partition_row_indexes[p].data() + partition_row_indexes[p].size())); + + // Flush large buffers immediately to keep memory bounded + if (output_buffers[p]->allocated_bytes() >= MAX_BATCH_BYTES) { + RETURN_IF_ERROR(_flush_buffer(state, p, output_buffers[p])); + } + } + + return Status::OK(); +} + +Status SpillRepartitioner::_route_block_by_columns( + RuntimeState* state, Block& block, + std::vector>& output_buffers) { + const auto rows = block.rows(); + if (rows == 0) { + return Status::OK(); + } + + // Compute CRC32 hash on key columns + std::vector hash_vals(rows, 0); + auto* __restrict hashes = hash_vals.data(); + for (size_t j = 0; j < _key_column_indices.size(); ++j) { + auto col_idx = _key_column_indices[j]; + DCHECK_LT(col_idx, block.columns()); + const auto& column = block.get_by_position(col_idx).column; + column->update_crcs_with_value(hashes, _key_data_types[j]->get_primitive_type(), + static_cast(rows)); + } + + // Map hash values to output channels with level-aware mixing. + for (size_t i = 0; i < rows; ++i) { + hashes[i] = _map_hash_to_partition(hashes[i]); + } + + // Build per-partition row index lists + std::vector> partition_row_indexes(_fanout); + for (uint32_t i = 0; i < rows; ++i) { + partition_row_indexes[hashes[i]].emplace_back(i); + } + + // Scatter rows into per-partition buffers + for (int p = 0; p < _fanout; ++p) { + if (partition_row_indexes[p].empty()) { + continue; + } + + if (!output_buffers[p]) { + output_buffers[p] = MutableBlock::create_unique(block.clone_empty()); + } + + RETURN_IF_ERROR(output_buffers[p]->add_rows( + &block, partition_row_indexes[p].data(), + partition_row_indexes[p].data() + partition_row_indexes[p].size())); + + if (output_buffers[p]->allocated_bytes() >= MAX_BATCH_BYTES) { + RETURN_IF_ERROR(_flush_buffer(state, p, output_buffers[p])); + } + } + + return Status::OK(); +} + +Status SpillRepartitioner::_flush_buffer(RuntimeState* state, int partition_idx, + std::unique_ptr& buffer) { + if (!buffer || buffer->rows() == 0) { + return Status::OK(); + } + DCHECK(partition_idx < _fanout && _output_writers[partition_idx]); + if (UNLIKELY(partition_idx >= _fanout || !_output_writers[partition_idx])) { + return Status::InternalError( + "SpillRepartitioner output writer is not initialized for partition {}", + partition_idx); + } + auto out_block = buffer->to_block(); + buffer.reset(); + return _output_writers[partition_idx]->write_block(state, out_block); +} + +Status SpillRepartitioner::_flush_all_buffers( + RuntimeState* state, std::vector>& output_buffers, + bool force) { + for (int i = 0; i < _fanout; ++i) { + if (!output_buffers[i] || output_buffers[i]->rows() == 0) { + continue; + } + if (force || output_buffers[i]->allocated_bytes() >= MAX_BATCH_BYTES) { + RETURN_IF_ERROR(_flush_buffer(state, i, output_buffers[i])); + } + } + return Status::OK(); +} + +uint32_t SpillRepartitioner::_map_hash_to_partition(uint32_t hash) const { + DCHECK_GT(_fanout, 0); + // Use a level-dependent salt so each repartition level has a different + // projection from hash-space to partition-space. + constexpr uint32_t LEVEL_SALT_BASE = 0x9E3779B9U; + auto salt = static_cast(_repartition_level + 1) * LEVEL_SALT_BASE; + auto mixed = crc32c_shuffle_mix(hash ^ salt); + return ((mixed >> 16) | (mixed << 16)) % static_cast(_fanout); +} + +#include "common/compile_check_end.h" +} // namespace doris diff --git a/be/src/exec/spill/spill_repartitioner.h b/be/src/exec/spill/spill_repartitioner.h new file mode 100644 index 00000000000000..e100a2eeb957ff --- /dev/null +++ b/be/src/exec/spill/spill_repartitioner.h @@ -0,0 +1,161 @@ +// 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 "core/data_type/data_type.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" + +namespace doris { +class RuntimeState; +class RuntimeProfile; + +class Block; +class PartitionerBase; + +/// SpillRepartitioner reads data from an input SpillFile and redistributes it +/// into FANOUT output SpillFiles by computing hash on key columns. +/// +/// This is the core building block for multi-level spill partitioning used by both +/// Hash Join and Aggregation operators. +/// +/// Two modes of operation: +/// 1. Partitioner mode (init): Uses a PartitionerBase with expression contexts to +/// compute hash. Suitable for Hash Join where blocks match the child's row descriptor. +/// 2. Column-index mode (init_with_key_columns): Computes CRC32 hash directly on +/// specified column indices. Suitable for Aggregation where spill blocks have a +/// different schema (key columns at fixed positions 0..N-1). +/// +/// For repartitioning, hash computation and final channel mapping are separated: +/// - a partitioner can provide either direct channel ids or raw hash values +/// (e.g. SpillRePartitionChannelIds returns raw hash), +/// - SpillRepartitioner then applies the final channel mapping strategy. +/// This keeps repartition policy centralized and allows level-aware mapping. +/// +/// Processing is incremental: each call to repartition() processes up to MAX_BATCH_BYTES +/// (32 MB) of data and then returns, allowing the pipeline scheduler to yield and +/// re-schedule. The caller should loop calling repartition() until `done` is true. +/// +/// Usage pattern: +/// // 1. Initialize +/// repartitioner.init(...) or repartitioner.init_with_key_columns(...) +/// // 2. Create output files and set up writers +/// SpillRepartitioner::create_output_spill_files(state, ..., output_files, fanout); +/// repartitioner.setup_output(state, output_files); +/// // 3. Route blocks and/or repartition files +/// repartitioner.route_block(state, block); // from hash table +/// repartitioner.repartition(state, input_file, &done); // from spill file +/// // 4. Finalize +/// repartitioner.finalize(); +class SpillRepartitioner { +public: + static constexpr int MAX_DEPTH = 8; + static constexpr size_t MAX_BATCH_BYTES = 32 * 1024 * 1024; // 32 MB yield threshold + + SpillRepartitioner() = default; + ~SpillRepartitioner() = default; + + /// Initialize the repartitioner with a partitioner (for Hash Join). + void init(std::unique_ptr partitioner, RuntimeProfile* profile, int fanout, + int repartition_level); + + /// Initialize the repartitioner with explicit key column indices (for Aggregation). + void init_with_key_columns(std::vector key_column_indices, + std::vector key_data_types, RuntimeProfile* profile, + int fanout, int repartition_level); + + /// Set up output SpillFiles and create persistent writers for them. + /// Must be called before repartition() or route_block(). + Status setup_output(RuntimeState* state, std::vector& output_spill_files); + + /// Repartition data from input_spill_file into output files. + /// The input reader is created lazily and persists across yield calls. + /// Call repeatedly until done == true. + Status repartition(RuntimeState* state, SpillFileSPtr& input_spill_file, bool* done); + + /// Repartition data using an existing reader (continues from its current + /// position). Useful when the caller has already partially read the file + /// and wants to repartition only the remaining data without re-reading + /// from the beginning. Ownership of the reader is transferred on completion. + /// Call repeatedly until done == true. + Status repartition(RuntimeState* state, SpillFileReaderSPtr& reader, bool* done); + + /// Route a single in-memory block into output files via persistent writers. + Status route_block(RuntimeState* state, Block& block); + + /// Finalize: close all output writers and update SpillFile stats. + /// Also resets internal reader state. + Status finalize(); + + /// Create FANOUT output SpillFiles registered with the SpillFileManager. + static Status create_output_spill_files(RuntimeState* state, int node_id, + const std::string& label_prefix, int fanout, + std::vector& output_spill_files); + + int fanout() const { return _fanout; } + +private: + /// Route a block using the partitioner (Hash Join mode). + Status _route_block(RuntimeState* state, Block& block, + std::vector>& output_buffers); + + /// Route a block using direct column-index hashing (Aggregation mode). + Status _route_block_by_columns(RuntimeState* state, Block& block, + std::vector>& output_buffers); + + Status _flush_buffer(RuntimeState* state, int partition_idx, + std::unique_ptr& buffer); + + Status _flush_all_buffers(RuntimeState* state, + std::vector>& output_buffers, + bool force); + + uint32_t _map_hash_to_partition(uint32_t hash) const; + + // Partitioner mode (used by Hash Join) + std::unique_ptr _partitioner; + + // Column-index mode (used by Aggregation) + std::vector _key_column_indices; + std::vector _key_data_types; + bool _use_column_index_mode = false; + + RuntimeProfile::Counter* _repartition_timer = nullptr; + RuntimeProfile::Counter* _repartition_rows = nullptr; + RuntimeProfile* _operator_profile = nullptr; + int _fanout = 8; + int _repartition_level = 0; + + // ── Persistent state across repartition/route_block calls ────── + // Output writers (one per partition), created by setup_output() + std::vector _output_writers; + // Pointer to caller's output SpillFiles vector (for finalize) + std::vector* _output_spill_files = nullptr; + // Input reader for repartition(), persists across yield calls + SpillFileReaderSPtr _input_reader; + SpillFileSPtr _current_input_file; +}; + +} // namespace doris diff --git a/be/src/exec/spill/spill_stream.cpp b/be/src/exec/spill/spill_stream.cpp deleted file mode 100644 index 07a7d85b116b22..00000000000000 --- a/be/src/exec/spill/spill_stream.cpp +++ /dev/null @@ -1,173 +0,0 @@ -// 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 "exec/spill/spill_stream.h" - -#include - -#include -#include -#include - -#include "core/block/block.h" -#include "exec/spill/spill_reader.h" -#include "exec/spill/spill_stream_manager.h" -#include "exec/spill/spill_writer.h" -#include "io/fs/local_file_system.h" -#include "runtime/exec_env.h" -#include "runtime/query_context.h" -#include "runtime/runtime_profile.h" -#include "runtime/runtime_state.h" -#include "runtime/thread_context.h" -#include "util/debug_points.h" - -namespace doris { -#include "common/compile_check_begin.h" -SpillStream::SpillStream(RuntimeState* state, int64_t stream_id, SpillDataDir* data_dir, - std::string spill_dir, size_t batch_rows, size_t batch_bytes, - RuntimeProfile* operator_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), - query_id_(state->query_id()), - profile_(operator_profile) { - RuntimeProfile* custom_profile = operator_profile->get_child("CustomCounters"); - DCHECK(custom_profile != nullptr); - _total_file_count = custom_profile->get_counter("SpillWriteFileTotalCount"); - _current_file_count = custom_profile->get_counter("SpillWriteFileCurrentCount"); - _current_file_size = custom_profile->get_counter("SpillWriteFileCurrentBytes"); -} - -void SpillStream::update_shared_profiles(RuntimeProfile* source_op_profile) { - _current_file_count = source_op_profile->get_counter("SpillWriteFileCurrentCount"); - _current_file_size = source_op_profile->get_counter("SpillWriteFileCurrentBytes"); -} - -SpillStream::~SpillStream() { - gc(); -} - -void SpillStream::gc() { - if (_current_file_size) { - COUNTER_UPDATE(_current_file_size, -total_written_bytes_); - } - bool exists = false; - auto status = io::global_local_filesystem()->exists(spill_dir_, &exists); - if (status.ok() && exists) { - if (_current_file_count) { - COUNTER_UPDATE(_current_file_count, -1); - } - auto query_gc_dir = data_dir_->get_spill_data_gc_path(print_id(query_id_)); - status = io::global_local_filesystem()->create_directory(query_gc_dir); - DBUG_EXECUTE_IF("fault_inject::spill_stream::gc", { - status = Status::Error("fault_inject spill_stream gc failed"); - }); - if (status.ok()) { - auto gc_dir = fmt::format("{}/{}", query_gc_dir, - std::filesystem::path(spill_dir_).filename().string()); - status = io::global_local_filesystem()->rename(spill_dir_, gc_dir); - } - if (!status.ok()) { - LOG_EVERY_T(WARNING, 1) << fmt::format("failed to gc spill data, dir {}, error: {}", - query_gc_dir, status.to_string()); - } - } - // If QueryContext is destructed earlier than PipelineFragmentContext, - // spill_dir_ will be already moved to spill_gc directory. - - // decrease spill data usage anyway, since in ~QueryContext() spill data of the query will be - // clean up as a last resort - data_dir_->update_spill_data_usage(-total_written_bytes_); - total_written_bytes_ = 0; -} - -Status SpillStream::prepare() { - writer_ = std::make_unique(state_->get_query_ctx()->resource_ctx(), profile_, - stream_id_, batch_rows_, data_dir_, spill_dir_); - _set_write_counters(profile_); - - reader_ = std::make_unique(state_->get_query_ctx()->resource_ctx(), stream_id_, - writer_->get_file_path()); - - DBUG_EXECUTE_IF("fault_inject::spill_stream::prepare_spill", { - return Status::Error("fault_inject spill_stream prepare_spill failed"); - }); - COUNTER_UPDATE(_total_file_count, 1); - if (_current_file_count) { - COUNTER_UPDATE(_current_file_count, 1); - } - return writer_->open(); -} - -SpillReaderUPtr SpillStream::create_separate_reader() const { - return std::make_unique(state_->get_query_ctx()->resource_ctx(), stream_id_, - writer_->get_file_path()); -} - -const TUniqueId& SpillStream::query_id() const { - return query_id_; -} - -const std::string& SpillStream::get_spill_root_dir() const { - return data_dir_->path(); -} - -Status SpillStream::spill_block(RuntimeState* state, const Block& block, bool eof) { - size_t written_bytes = 0; - DBUG_EXECUTE_IF("fault_inject::spill_stream::spill_block", { - return Status::Error("fault_inject spill_stream spill_block failed"); - }); - RETURN_IF_ERROR(writer_->write(state, block, written_bytes)); - if (eof) { - RETURN_IF_ERROR(spill_eof()); - } else { - total_written_bytes_ = writer_->get_written_bytes(); - } - return Status::OK(); -} - -Status SpillStream::spill_eof() { - DBUG_EXECUTE_IF("fault_inject::spill_stream::spill_eof", { - return Status::Error("fault_inject spill_stream spill_eof failed"); - }); - auto status = writer_->close(); - total_written_bytes_ = writer_->get_written_bytes(); - writer_.reset(); - - if (status.ok()) { - _ready_for_reading = true; - } - return status; -} - -Status SpillStream::read_next_block_sync(Block* block, bool* eos) { - DCHECK(reader_ != nullptr); - DCHECK(!_is_reading); - _is_reading = true; - Defer defer([this] { _is_reading = false; }); - - DBUG_EXECUTE_IF("fault_inject::spill_stream::read_next_block", { - return Status::Error("fault_inject spill_stream read_next_block failed"); - }); - RETURN_IF_ERROR(reader_->open()); - return reader_->read(block, eos); -} - -} // namespace doris diff --git a/be/src/exec/spill/spill_stream.h b/be/src/exec/spill/spill_stream.h deleted file mode 100644 index 14420d2a6c9c14..00000000000000 --- a/be/src/exec/spill/spill_stream.h +++ /dev/null @@ -1,108 +0,0 @@ -// 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 "exec/spill/spill_reader.h" -#include "exec/spill/spill_writer.h" - -namespace doris { -#include "common/compile_check_begin.h" -class RuntimeProfile; -class ThreadPool; - -class Block; -class SpillDataDir; - -class SpillStream { -public: - // to avoid too many small file writes - static constexpr size_t MIN_SPILL_WRITE_BATCH_MEM = 32 * 1024; - static constexpr size_t MAX_SPILL_WRITE_BATCH_MEM = 32 * 1024 * 1024; - SpillStream(RuntimeState* state, int64_t stream_id, SpillDataDir* data_dir, - std::string spill_dir, size_t batch_rows, size_t batch_bytes, - RuntimeProfile* profile); - - SpillStream() = delete; - - ~SpillStream(); - - void gc(); - - int64_t id() const { return stream_id_; } - - SpillDataDir* get_data_dir() const { return data_dir_; } - const std::string& get_spill_root_dir() const; - - const std::string& get_spill_dir() const { return spill_dir_; } - - int64_t get_written_bytes() const { return total_written_bytes_; } - - Status spill_block(RuntimeState* state, const Block& block, bool eof); - - Status spill_eof(); - - Status read_next_block_sync(Block* block, bool* eos); - - void set_read_counters(RuntimeProfile* operator_profile) { - reader_->set_counters(operator_profile); - } - - void update_shared_profiles(RuntimeProfile* source_op_profile); - - SpillReaderUPtr create_separate_reader() const; - - const TUniqueId& query_id() const; - - bool ready_for_reading() const { return _ready_for_reading; } - -private: - friend class SpillStreamManager; - - Status prepare(); - - void _set_write_counters(RuntimeProfile* profile) { writer_->set_counters(profile); } - - RuntimeState* state_ = nullptr; - int64_t stream_id_; - SpillDataDir* data_dir_ = nullptr; - // Directory path format specified in SpillStreamManager::register_spill_stream: - // storage_root/spill/query_id/partitioned_hash_join-node_id-task_id-stream_id - std::string spill_dir_; - size_t batch_rows_; - size_t batch_bytes_; - int64_t total_written_bytes_ = 0; - - std::atomic_bool _ready_for_reading = false; - std::atomic_bool _is_reading = false; - - SpillWriterUPtr writer_; - SpillReaderUPtr reader_; - - TUniqueId query_id_; - - RuntimeProfile* profile_ = nullptr; - RuntimeProfile::Counter* _current_file_count = nullptr; - RuntimeProfile::Counter* _total_file_count = nullptr; - RuntimeProfile::Counter* _current_file_size = nullptr; -}; -using SpillStreamSPtr = std::shared_ptr; -} // namespace doris -#include "common/compile_check_end.h" diff --git a/be/src/exec/spill/spill_writer.cpp b/be/src/exec/spill/spill_writer.cpp deleted file mode 100644 index ce5bfa14215ef5..00000000000000 --- a/be/src/exec/spill/spill_writer.cpp +++ /dev/null @@ -1,178 +0,0 @@ -// 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 "exec/spill/spill_writer.h" - -#include "agent/be_exec_version_manager.h" -#include "common/status.h" -#include "exec/spill/spill_stream_manager.h" -#include "io/fs/local_file_system.h" -#include "io/fs/local_file_writer.h" -#include "runtime/exec_env.h" -#include "runtime/runtime_state.h" -#include "runtime/thread_context.h" - -namespace doris { -#include "common/compile_check_begin.h" -Status SpillWriter::open() { - if (file_writer_) { - return Status::OK(); - } - return io::global_local_filesystem()->create_file(file_path_, &file_writer_); -} - -Status SpillWriter::close() { - if (closed_ || !file_writer_) { - return Status::OK(); - } - closed_ = true; - - 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_file_timer); - RETURN_IF_ERROR(file_writer_->append(meta_)); - } - - total_written_bytes_ += meta_.size(); - COUNTER_UPDATE(_write_file_total_size, meta_.size()); - if (_resource_ctx) { - _resource_ctx->io_context()->update_spill_write_bytes_to_local_storage(meta_.size()); - } - if (_write_file_current_size) { - COUNTER_UPDATE(_write_file_current_size, meta_.size()); - } - data_dir_->update_spill_data_usage(meta_.size()); - ExecEnv::GetInstance()->spill_stream_mgr()->update_spill_write_bytes(meta_.size()); - - RETURN_IF_ERROR(file_writer_->close()); - - file_writer_.reset(); - return Status::OK(); -} - -Status SpillWriter::write(RuntimeState* state, const Block& block, size_t& written_bytes) { - written_bytes = 0; - DCHECK(file_writer_); - auto rows = block.rows(); - COUNTER_UPDATE(_write_rows_counter, rows); - COUNTER_UPDATE(_write_block_bytes_counter, block.bytes()); - // file format: block1, block2, ..., blockn, meta - if (rows <= batch_size_) { - return _write_internal(block, written_bytes); - } else { - auto 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 && !state->is_cancelled();) { - tmp_block.clear_column_data(); - - const 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); - } - }); - - int64_t tmp_blcok_mem = tmp_block.allocated_bytes(); - COUNTER_UPDATE(_memory_used_counter, tmp_blcok_mem); - Defer defer {[&]() { COUNTER_UPDATE(_memory_used_counter, -tmp_blcok_mem); }}; - 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; - int64_t buff_size {0}; - - if (block.rows() > 0) { - { - PBlock pblock; - SCOPED_TIMER(_serialize_timer); - int64_t compressed_time = 0; - status = block.serialize( - BeExecVersionManager::get_newest_version(), &pblock, &uncompressed_bytes, - &compressed_bytes, &compressed_time, - segment_v2::CompressionTypePB::ZSTD); // ZSTD for better compression ratio - RETURN_IF_ERROR(status); - int64_t pblock_mem = pblock.ByteSizeLong(); - COUNTER_UPDATE(_memory_used_counter, pblock_mem); - Defer defer {[&]() { COUNTER_UPDATE(_memory_used_counter, -pblock_mem); }}; - if (!pblock.SerializeToString(&buff)) { - return Status::Error( - "serialize spill data error. [path={}]", file_path_); - } - buff_size = buff.size(); - COUNTER_UPDATE(_memory_used_counter, buff_size); - Defer defer2 {[&]() { COUNTER_UPDATE(_memory_used_counter, -buff_size); }}; - } - if (data_dir_->reach_capacity_limit(buff_size)) { - return Status::Error( - "spill data total size exceed limit, path: {}, size limit: {}, spill data " - "size: {}", - data_dir_->path(), - PrettyPrinter::print_bytes(data_dir_->get_spill_data_limit()), - PrettyPrinter::print_bytes(data_dir_->get_spill_data_bytes())); - } - - { - Defer defer {[&]() { - if (status.ok()) { - data_dir_->update_spill_data_usage(buff_size); - ExecEnv::GetInstance()->spill_stream_mgr()->update_spill_write_bytes(buff_size); - - written_bytes += buff_size; - max_sub_block_size_ = std::max(max_sub_block_size_, (size_t)buff_size); - - meta_.append((const char*)&total_written_bytes_, sizeof(size_t)); - COUNTER_UPDATE(_write_file_total_size, buff_size); - if (_resource_ctx) { - _resource_ctx->io_context()->update_spill_write_bytes_to_local_storage( - buff_size); - } - if (_write_file_current_size) { - COUNTER_UPDATE(_write_file_current_size, buff_size); - } - COUNTER_UPDATE(_write_block_counter, 1); - total_written_bytes_ += buff_size; - ++written_blocks_; - } - }}; - { - SCOPED_TIMER(_write_file_timer); - status = file_writer_->append(buff); - RETURN_IF_ERROR(status); - } - } - } - - return status; -} - -} // namespace doris \ No newline at end of file diff --git a/be/src/exec/spill/spill_writer.h b/be/src/exec/spill/spill_writer.h deleted file mode 100644 index c78fcaea98be54..00000000000000 --- a/be/src/exec/spill/spill_writer.h +++ /dev/null @@ -1,103 +0,0 @@ -// 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 "core/block/block.h" -#include "io/fs/file_writer.h" -#include "runtime/runtime_profile.h" -#include "runtime/workload_management/resource_context.h" -namespace doris { -#include "common/compile_check_begin.h" -class RuntimeState; - -class SpillDataDir; -class SpillWriter { -public: - SpillWriter(std::shared_ptr resource_context, RuntimeProfile* profile, - int64_t id, size_t batch_size, SpillDataDir* data_dir, const std::string& dir) - : data_dir_(data_dir), - stream_id_(id), - batch_size_(batch_size), - _resource_ctx(std::move(resource_context)) { - // Directory path format specified in SpillStreamManager::register_spill_stream: - // storage_root/spill/query_id/partitioned_hash_join-node_id-task_id-stream_id/0 - file_path_ = dir + "/0"; - RuntimeProfile* common_profile = profile->get_child("CommonCounters"); - DCHECK(common_profile != nullptr); - _memory_used_counter = common_profile->get_counter("MemoryUsage"); - } - - Status open(); - - Status close(); - - Status write(RuntimeState* state, const Block& block, size_t& written_bytes); - - int64_t get_id() const { return stream_id_; } - - int64_t get_written_bytes() const { return total_written_bytes_; } - - const std::string& get_file_path() const { return file_path_; } - - void set_counters(RuntimeProfile* operator_profile) { - RuntimeProfile* custom_profile = operator_profile->get_child("CustomCounters"); - _write_file_timer = custom_profile->get_counter("SpillWriteFileTime"); - _serialize_timer = custom_profile->get_counter("SpillWriteSerializeBlockTime"); - _write_block_counter = custom_profile->get_counter("SpillWriteBlockCount"); - _write_block_bytes_counter = custom_profile->get_counter("SpillWriteBlockBytes"); - _write_file_total_size = custom_profile->get_counter("SpillWriteFileBytes"); - _write_file_current_size = custom_profile->get_counter("SpillWriteFileCurrentBytes"); - _write_rows_counter = custom_profile->get_counter("SpillWriteRows"); - } - -private: - 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. - SpillDataDir* data_dir_ = nullptr; - std::atomic_bool closed_ = false; - int64_t stream_id_; - size_t batch_size_; - size_t max_sub_block_size_ = 0; - std::string file_path_; - std::unique_ptr file_writer_; - - size_t written_blocks_ = 0; - int64_t total_written_bytes_ = 0; - std::string meta_; - - RuntimeProfile::Counter* _write_file_timer = nullptr; - RuntimeProfile::Counter* _serialize_timer = nullptr; - RuntimeProfile::Counter* _write_block_counter = nullptr; - RuntimeProfile::Counter* _write_block_bytes_counter = nullptr; - RuntimeProfile::Counter* _write_file_total_size = nullptr; - RuntimeProfile::Counter* _write_file_current_size = nullptr; - RuntimeProfile::Counter* _write_rows_counter = nullptr; - RuntimeProfile::Counter* _memory_used_counter = nullptr; - - std::shared_ptr _resource_ctx = nullptr; -}; -using SpillWriterUPtr = std::unique_ptr; -} // namespace doris - -#include "common/compile_check_end.h" diff --git a/be/src/format/parquet/byte_array_dict_decoder.cpp b/be/src/format/parquet/byte_array_dict_decoder.cpp index 9b685a075e242f..5441c07fce617e 100644 --- a/be/src/format/parquet/byte_array_dict_decoder.cpp +++ b/be/src/format/parquet/byte_array_dict_decoder.cpp @@ -23,6 +23,7 @@ #include "core/column/column.h" #include "core/column/column_dictionary.h" #include "core/column/column_string.h" +#include "core/custom_allocator.h" #include "core/data_type/data_type_nullable.h" #include "util/coding.h" #include "util/rle_encoding.h" @@ -81,7 +82,8 @@ Status ByteArrayDictDecoder::read_dict_values_to_column(MutableColumnPtr& doris_ MutableColumnPtr ByteArrayDictDecoder::convert_dict_column_to_string_column( const ColumnInt32* dict_column) { auto res = ColumnString::create(); - std::vector dict_values(dict_column->size()); + DorisVector dict_values(dict_column->size()); + const auto& data = dict_column->get_data(); for (size_t i = 0; i < dict_column->size(); ++i) { dict_values[i] = _dict_items[data[i]]; @@ -126,7 +128,7 @@ Status ByteArrayDictDecoder::_decode_values(MutableColumnPtr& doris_column, Data while (size_t run_length = select_vector.get_next_run(&read_type)) { switch (read_type) { case ColumnSelectVector::CONTENT: { - std::vector string_values; + DorisVector string_values; string_values.reserve(run_length); for (size_t i = 0; i < run_length; ++i) { string_values.emplace_back(_dict_items[_indexes[dict_index++]]); diff --git a/be/src/format/parquet/byte_array_dict_decoder.h b/be/src/format/parquet/byte_array_dict_decoder.h index 28d82f04ef09a2..89df5a2c8b51db 100644 --- a/be/src/format/parquet/byte_array_dict_decoder.h +++ b/be/src/format/parquet/byte_array_dict_decoder.h @@ -58,8 +58,8 @@ class ByteArrayDictDecoder final : public BaseDictDecoder { protected: // For dictionary encoding - std::vector _dict_items; - std::vector _dict_data; + DorisVector _dict_items; + DorisVector _dict_data; size_t _max_value_length; }; #include "common/compile_check_end.h" diff --git a/be/src/format/parquet/fix_length_dict_decoder.hpp b/be/src/format/parquet/fix_length_dict_decoder.hpp index ef84ecbfdbcc0a..37782cd7242e6f 100644 --- a/be/src/format/parquet/fix_length_dict_decoder.hpp +++ b/be/src/format/parquet/fix_length_dict_decoder.hpp @@ -223,7 +223,7 @@ class FixLengthDictDecoder final : public BaseDictDecoder { return res; } // For dictionary encoding - std::vector::CppType> _dict_items; + DorisVector::CppType> _dict_items; }; #include "common/compile_check_end.h" diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 502fb50bd91972..f5afa76aa9f317 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -50,7 +50,7 @@ class MemoryPool; namespace doris { class VDataStreamMgr; -class SpillStreamManager; +class SpillFileManager; class DeltaWriterV2Pool; class DictionaryFactory; class TaskScheduler; @@ -284,7 +284,9 @@ class ExecEnv { LoadStreamMgr* load_stream_mgr() { return _load_stream_mgr.get(); } NewLoadStreamMgr* new_load_stream_mgr() { return _new_load_stream_mgr.get(); } SmallFileMgr* small_file_mgr() { return _small_file_mgr; } - doris::SpillStreamManager* spill_stream_mgr() { return _spill_stream_mgr; } + + SpillFileManager* spill_file_mgr() { return _spill_file_mgr; } + GroupCommitMgr* group_commit_mgr() { return _group_commit_mgr; } CdcClientMgr* cdc_client_mgr() { return _cdc_client_mgr; } @@ -556,7 +558,8 @@ class ExecEnv { std::unique_ptr _pipeline_tracer_ctx; std::unique_ptr _tmp_file_dirs; - doris::SpillStreamManager* _spill_stream_mgr = nullptr; + + SpillFileManager* _spill_file_mgr = nullptr; orc::MemoryPool* _orc_memory_pool = nullptr; arrow::MemoryPool* _arrow_memory_pool = nullptr; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 68a8b6fed11958..d914ba1d73e5c9 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -52,7 +52,7 @@ #include "exec/scan/scanner_scheduler.h" #include "exec/sink/delta_writer_v2_pool.h" #include "exec/sink/load_stream_map_pool.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "exprs/function/dictionary_factory.h" #include "format/orc/orc_memory_pool.h" #include "format/parquet/arrow_memory_pool.h" @@ -124,6 +124,7 @@ #include "util/threadpool.h" #include "util/thrift_rpc_helper.h" #include "util/timezone_utils.h" + // clang-format off // this must after util/brpc_client_cache.h // /doris/thirdparty/installed/include/brpc/errno.pb.h:69:3: error: expected identifier @@ -350,7 +351,8 @@ Status ExecEnv::_init(const std::vector& store_paths, _wal_manager = WalManager::create_unique(this, config::group_commit_wal_path); _dns_cache = new DNSCache(); _write_cooldown_meta_executors = std::make_unique(); - _spill_stream_mgr = new SpillStreamManager(std::move(spill_store_map)); + + _spill_file_mgr = new SpillFileManager(std::move(spill_store_map)); _kerberos_ticket_mgr = new kerberos::KerberosTicketMgr(config::kerberos_ccache_path); _hdfs_mgr = new io::HdfsMgr(); _backend_client_cache->init_metrics("backend"); @@ -442,7 +444,7 @@ Status ExecEnv::_init(const std::vector& store_paths, }); } - RETURN_IF_ERROR(_spill_stream_mgr->init()); + RETURN_IF_ERROR(_spill_file_mgr->init()); RETURN_IF_ERROR(_runtime_query_statistics_mgr->start_report_thread()); _dict_factory = new doris::DictionaryFactory(); _s_ready = true; @@ -832,7 +834,7 @@ void ExecEnv::destroy() { SAFE_STOP(_storage_engine); _storage_engine.reset(); - SAFE_STOP(_spill_stream_mgr); + SAFE_STOP(_spill_file_mgr); if (_runtime_query_statistics_mgr) { _runtime_query_statistics_mgr->stop_report_thread(); } @@ -885,7 +887,7 @@ void ExecEnv::destroy() { SAFE_DELETE(_vstream_mgr); // When _vstream_mgr is deconstructed, it will try call query context's dctor and will // access spill stream mgr, so spill stream mgr should be deconstructed after data stream manager - SAFE_DELETE(_spill_stream_mgr); + SAFE_DELETE(_spill_file_mgr); SAFE_DELETE(_fragment_mgr); SAFE_DELETE(_workload_sched_mgr); SAFE_DELETE(_workload_group_manager); diff --git a/be/src/runtime/query_context.cpp b/be/src/runtime/query_context.cpp index dc7e28cf60c376..d641bad3c61b63 100644 --- a/be/src/runtime/query_context.cpp +++ b/be/src/runtime/query_context.cpp @@ -35,7 +35,7 @@ #include "exec/pipeline/dependency.h" #include "exec/pipeline/pipeline_fragment_context.h" #include "exec/runtime_filter/runtime_filter_definitions.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" #include "runtime/memory/heap_profiler.h" diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 74a0e7c34e5986..f233db496f2d8f 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -665,37 +665,91 @@ class RuntimeState { int64_t spill_min_revocable_mem() const { if (_query_options.__isset.min_revocable_mem) { - return std::max(_query_options.min_revocable_mem, (int64_t)1); + return std::max(_query_options.min_revocable_mem, (int64_t)1 << 20); } - return 1; + return 32 << 20; } - int64_t spill_sort_mem_limit() const { - if (_query_options.__isset.spill_sort_mem_limit) { - return std::max(_query_options.spill_sort_mem_limit, (int64_t)16777216); + int spill_aggregation_partition_count() const { + if (_query_options.__isset.spill_aggregation_partition_count) { + return std::min(std::max(2, _query_options.spill_aggregation_partition_count), 32); } - return 134217728; + return 8; } - int64_t spill_sort_batch_bytes() const { - if (_query_options.__isset.spill_sort_batch_bytes) { - return std::max(_query_options.spill_sort_batch_bytes, (int64_t)8388608); + int spill_hash_join_partition_count() const { + if (_query_options.__isset.spill_hash_join_partition_count) { + return std::min(std::max(2, _query_options.spill_hash_join_partition_count), 32); } - return 8388608; + return 8; } - int spill_aggregation_partition_count() const { - if (_query_options.__isset.spill_aggregation_partition_count) { - return std::min(std::max(_query_options.spill_aggregation_partition_count, 16), 8192); + int spill_repartition_max_depth() const { + if (_query_options.__isset.spill_repartition_max_depth) { + // Clamp to a reasonable range: [1, 128] + return std::max(1, std::min(_query_options.spill_repartition_max_depth, 128)); + } + return 8; + } + + int64_t spill_buffer_size_bytes() const { + // clamp to [1MB, 256MB] + constexpr int64_t kMin = 1LL * 1024 * 1024; + constexpr int64_t kMax = 256LL * 1024 * 1024; + if (_query_options.__isset.spill_buffer_size_bytes) { + int64_t v = _query_options.spill_buffer_size_bytes; + if (v < kMin) return kMin; + if (v > kMax) return kMax; + return v; + } + return 8LL * 1024 * 1024; + } + + // Per-sink memory limits: after spill is triggered, the sink proactively + // spills when its revocable memory exceeds this threshold. + // Clamped to [1MB, 4GB], default 64MB. + int64_t spill_join_build_sink_mem_limit_bytes() const { + constexpr int64_t kMin = 1LL * 1024 * 1024; + constexpr int64_t kMax = 4LL * 1024 * 1024 * 1024; + constexpr int64_t kDefault = 64LL * 1024 * 1024; + if (_query_options.__isset.spill_join_build_sink_mem_limit_bytes) { + int64_t v = _query_options.spill_join_build_sink_mem_limit_bytes; + return std::min(std::max(v, kMin), kMax); } - return 32; + return kDefault; } - int spill_hash_join_partition_count() const { - if (_query_options.__isset.spill_hash_join_partition_count) { - return std::min(std::max(_query_options.spill_hash_join_partition_count, 16), 8192); + int64_t spill_aggregation_sink_mem_limit_bytes() const { + constexpr int64_t kMin = 1LL * 1024 * 1024; + constexpr int64_t kMax = 4LL * 1024 * 1024 * 1024; + constexpr int64_t kDefault = 64LL * 1024 * 1024; + if (_query_options.__isset.spill_aggregation_sink_mem_limit_bytes) { + int64_t v = _query_options.spill_aggregation_sink_mem_limit_bytes; + return std::min(std::max(v, kMin), kMax); + } + return kDefault; + } + + int64_t spill_sort_sink_mem_limit_bytes() const { + constexpr int64_t kMin = 1LL * 1024 * 1024; + constexpr int64_t kMax = 4LL * 1024 * 1024 * 1024; + constexpr int64_t kDefault = 64LL * 1024 * 1024; + if (_query_options.__isset.spill_sort_sink_mem_limit_bytes) { + int64_t v = _query_options.spill_sort_sink_mem_limit_bytes; + return std::min(std::max(v, kMin), kMax); + } + return kDefault; + } + + int64_t spill_sort_merge_mem_limit_bytes() const { + constexpr int64_t kMin = 1LL * 1024 * 1024; + constexpr int64_t kMax = 4LL * 1024 * 1024 * 1024; + constexpr int64_t kDefault = 64LL * 1024 * 1024; + if (_query_options.__isset.spill_sort_merge_mem_limit_bytes) { + int64_t v = _query_options.spill_sort_merge_mem_limit_bytes; + return std::min(std::max(v, kMin), kMax); } - return 32; + return kDefault; } int64_t low_memory_mode_buffer_limit() const { @@ -722,7 +776,7 @@ class RuntimeState { return _query_options.minimum_operator_memory_required_kb * 1024; } else { // refer other database - return 100 * 1024; + return 4 * 1024 * 1024; } } diff --git a/be/test/exec/operator/agg_operator_test.cpp b/be/test/exec/operator/agg_operator_test.cpp index ff99bee4d44ea6..945fd0f9f1fc81 100644 --- a/be/test/exec/operator/agg_operator_test.cpp +++ b/be/test/exec/operator/agg_operator_test.cpp @@ -592,7 +592,7 @@ TEST_F(AggOperatorTestWithGroupBy, other_case_2) { local_state->_emplace_into_hash_table(local_state->_places.data(), key_columns, block.rows()); - EXPECT_EQ(local_state->_get_hash_table_size(), 4); // [1,2,3,null] + EXPECT_EQ(local_state->get_hash_table_size(), 4); // [1,2,3,null] } } diff --git a/be/test/exec/operator/join_test_helper.h b/be/test/exec/operator/join_test_helper.h index ff6ea8a59828ab..e507c0cbcd5a46 100644 --- a/be/test/exec/operator/join_test_helper.h +++ b/be/test/exec/operator/join_test_helper.h @@ -27,7 +27,7 @@ #include "common/object_pool.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" #include "testutil/mock/mock_runtime_state.h" diff --git a/be/test/exec/operator/partitioned_aggregation_sink_operator_test.cpp b/be/test/exec/operator/partitioned_aggregation_sink_operator_test.cpp index 8db5e7827780bc..42cc320f1832eb 100644 --- a/be/test/exec/operator/partitioned_aggregation_sink_operator_test.cpp +++ b/be/test/exec/operator/partitioned_aggregation_sink_operator_test.cpp @@ -29,7 +29,7 @@ #include "exec/operator/partitioned_hash_join_probe_operator.h" #include "exec/operator/partitioned_hash_join_sink_operator.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" #include "testutil/column_helper.h" @@ -222,16 +222,14 @@ TEST_F(PartitionedAggregationSinkOperatorTest, SinkWithSpill) { st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - ASSERT_GT(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); - auto* inner_sink_local_state = reinterpret_cast( local_state->_runtime_state->get_sink_local_state()); - ASSERT_GT(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_GT(inner_sink_local_state->get_hash_table_size(), 0); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); - ASSERT_EQ(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); st = sink_operator->sink(_helper.runtime_state.get(), &block, true); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); @@ -283,16 +281,14 @@ TEST_F(PartitionedAggregationSinkOperatorTest, SinkWithSpillAndEmptyEOS) { st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - ASSERT_GT(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); - auto* inner_sink_local_state = reinterpret_cast( local_state->_runtime_state->get_sink_local_state()); - ASSERT_GT(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_GT(inner_sink_local_state->get_hash_table_size(), 0); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); - ASSERT_EQ(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); block.clear_column_data(); st = sink_operator->sink(_helper.runtime_state.get(), &block, true); @@ -344,20 +340,18 @@ TEST_F(PartitionedAggregationSinkOperatorTest, SinkWithSpillLargeData) { st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - ASSERT_GT(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); - auto* inner_sink_local_state = reinterpret_cast( local_state->_runtime_state->get_sink_local_state()); - ASSERT_GT(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_GT(inner_sink_local_state->get_hash_table_size(), 0); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); auto* spill_write_rows_counter = local_state->custom_profile()->get_counter("SpillWriteRows"); ASSERT_TRUE(spill_write_rows_counter != nullptr); ASSERT_EQ(spill_write_rows_counter->value(), 4); - ASSERT_EQ(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); const size_t count = 1048576; std::vector data(count); @@ -416,15 +410,303 @@ TEST_F(PartitionedAggregationSinkOperatorTest, SinkWithSpilError) { st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - ASSERT_GT(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); - auto* inner_sink_local_state = reinterpret_cast( local_state->_runtime_state->get_sink_local_state()); - ASSERT_GT(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_GT(inner_sink_local_state->get_hash_table_size(), 0); - SpillableDebugPointHelper dp_helper("fault_inject::spill_stream::spill_block"); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + SpillableDebugPointHelper dp_helper("fault_inject::spill_file::spill_block"); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_FALSE(st.ok()) << "spilll status should be failed"; + + st = sink_operator->close(_helper.runtime_state.get(), st); + ASSERT_TRUE(st.ok()) << "close status should be successful even if spill failed: " + << st.to_string(); +} + +// Test multiple consecutive revoke_memory calls to verify repeated spilling works. +TEST_F(PartitionedAggregationSinkOperatorTest, SinkWithMultipleRevokes) { + auto [source_operator, sink_operator] = _helper.create_operators(); + ASSERT_TRUE(source_operator != nullptr); + ASSERT_TRUE(sink_operator != nullptr); + + const auto tnode = _helper.create_test_plan_node(); + auto st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), "PartitionedAggSinkTestDep"); + + LocalSinkStateInfo info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(local_state != nullptr); + + st = local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + auto* inner_sink_local_state = reinterpret_cast( + local_state->_runtime_state->get_sink_local_state()); + + // Round 1: sink → revoke + auto block = ColumnHelper::create_block({1, 2, 3, 4, 5}); + block.insert(ColumnHelper::create_column_with_name({1, 2, 3, 4, 5})); + st = sink_operator->sink(_helper.runtime_state.get(), &block, false); + ASSERT_TRUE(st.ok()) << "sink round 1 failed: " << st.to_string(); + ASSERT_GT(inner_sink_local_state->get_hash_table_size(), 0); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke round 1 failed: " << st.to_string(); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); + + // Round 2: sink more → revoke again + auto block2 = ColumnHelper::create_block({6, 7, 8, 9, 10}); + block2.insert(ColumnHelper::create_column_with_name({6, 7, 8, 9, 10})); + st = sink_operator->sink(_helper.runtime_state.get(), &block2, false); + ASSERT_TRUE(st.ok()) << "sink round 2 failed: " << st.to_string(); + ASSERT_GT(inner_sink_local_state->get_hash_table_size(), 0); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke round 2 failed: " << st.to_string(); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); + + ASSERT_TRUE(shared_state->_is_spilled); + + // Verify spill counters accumulated across rounds + auto* spill_write_rows_counter = local_state->custom_profile()->get_counter("SpillWriteRows"); + ASSERT_TRUE(spill_write_rows_counter != nullptr); + ASSERT_EQ(spill_write_rows_counter->value(), 10) << "SpillWriteRows should be 10 (5 per round)"; + + // Sink EOS + block.clear_column_data(); + st = sink_operator->sink(_helper.runtime_state.get(), &block, true); + ASSERT_TRUE(st.ok()) << "sink eos failed: " << st.to_string(); +} + +// Test revoke_memory when hash table is empty (no data sunk). +TEST_F(PartitionedAggregationSinkOperatorTest, RevokeMemoryEmpty) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + const auto tnode = _helper.create_test_plan_node(); + auto st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), "PartitionedAggSinkTestDep"); + + LocalSinkStateInfo info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()); + + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(local_state != nullptr); + st = local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + // Revoke with no data is a valid operation + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke_memory on empty should succeed: " << st.to_string(); + ASSERT_TRUE(shared_state->_is_spilled); + + auto* spill_write_rows_counter = local_state->custom_profile()->get_counter("SpillWriteRows"); + ASSERT_TRUE(spill_write_rows_counter != nullptr); + ASSERT_EQ(spill_write_rows_counter->value(), 0); +} + +// Test that AggSinkOperatorX::get_hash_table_size() correctly delegates to local state. +// This validates the new operator-level public interface introduced to decouple +// PartitionedAggSinkOperatorX from the internal aggregate_data_container. +TEST_F(PartitionedAggregationSinkOperatorTest, GetHashTableSizeViaAggSinkOperator) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + const auto tnode = _helper.create_test_plan_node(); + auto st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), "PartitionedAggSinkTestDep"); + + LocalSinkStateInfo info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(local_state != nullptr); + st = local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + auto* inner_sink_local_state = reinterpret_cast( + local_state->_runtime_state->get_sink_local_state()); + + // Hash table should be empty before any data is sinked + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); + + auto block = ColumnHelper::create_block({1, 2, 3, 4, 5}); + block.insert(ColumnHelper::create_column_with_name({1, 2, 3, 4, 5})); + st = sink_operator->sink(_helper.runtime_state.get(), &block, false); + ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); + + // Hash table should have entries after sinked data + ASSERT_GT(inner_sink_local_state->get_hash_table_size(), 0); + + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); + + // Hash table should be cleared after revoke + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); + + st = sink_operator->close(_helper.runtime_state.get(), st); + ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); +} + +} // namespace doris + +namespace doris::pipeline { + +// A test fixture that recreates the descriptor table with nullable=true. +// This is necessary to exercise the NULL key path in hash table aggregation, +// where DataWithNullKey::size() = Base::size() + has_null_key counts the NULL entry, +// but aggregate_data_container->total_count() does NOT (NULL key stored separately). +class PartitionedAggregationNullableKeySinkTest : public testing::Test { +protected: + void SetUp() override { + _helper.SetUp(); + // Recreate descriptor table with nullable=true so slot 0 (the GROUP BY key) is nullable. + DescriptorTbl* desc_tbl; + auto desc_table = _helper.create_test_table_descriptor(true); + auto st = DescriptorTbl::create(_helper.obj_pool.get(), desc_table, &desc_tbl); + ASSERT_TRUE(st.ok()) << "create nullable descriptor table failed: " << st.to_string(); + _helper.desc_tbl = desc_tbl; + _helper.runtime_state->set_desc_tbl(desc_tbl); + } + void TearDown() override { _helper.TearDown(); } + PartitionedAggregationTestHelper _helper; +}; + +// Test the core bug scenario: when only NULL key data is in the hash table at EOS, +// the old check (aggregate_data_container->total_count() > 0) returns 0 because +// NULL key data is stored outside the aggregate_data_container. This causes EOS to +// skip the final flush, losing the NULL key aggregated result. +// +// The fix uses get_hash_table_size() which calls DataWithNullKey::size(): +// size_t size() const { return Base::size() + has_null_key; } +// This correctly returns 1 when only a NULL key entry exists, triggering the flush. +TEST_F(PartitionedAggregationNullableKeySinkTest, SinkEOSFlushNullKeyOnly) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + // Use a nullable key expression so the aggregation selects a DataWithNullKey hash method. + auto tnode = _helper.create_test_plan_node(); + tnode.agg_node.grouping_exprs[0].nodes[0].__set_is_nullable(true); + + auto st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + auto* dep = shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), + "PartitionedAggSinkTestDep"); + + LocalSinkStateInfo info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(local_state != nullptr); + st = local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + auto* inner_sink_local_state = reinterpret_cast( + local_state->_runtime_state->get_sink_local_state()); + + // Sink first batch: key=NULL (null_map={1}), value=42. + // All rows share the same NULL key → aggregated into hash_table.null_key_data. + // aggregate_data_container->total_count() = 0 (NULL key not in container) + // get_hash_table_size() = 1 (DataWithNullKey::size() = 0 + has_null_key) + auto block1 = ColumnHelper::create_nullable_block( + {0}, {1}); // placeholder value=0, null_map[0]=1 → key is NULL + block1.insert(ColumnHelper::create_column_with_name({42})); + st = sink_operator->sink(_helper.runtime_state.get(), &block1, false); + ASSERT_TRUE(st.ok()) << "first sink failed: " << st.to_string(); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 1); + + // Spill to disk and mark as spilled. + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); + ASSERT_TRUE(shared_state->_is_spilled); + + auto* spill_write_rows_counter = local_state->custom_profile()->get_counter("SpillWriteRows"); + ASSERT_TRUE(spill_write_rows_counter != nullptr); + ASSERT_EQ(spill_write_rows_counter->value(), 1); + + // Sink second batch: again key=NULL, value=10. + // Now _is_spilled=true; the NULL key row goes into hash_table.null_key_data again. + // aggregate_data_container->total_count() = 0 (NULL key not in container) + // get_hash_table_size() = 1 (DataWithNullKey::size() = 0 + has_null_key) + auto block2 = ColumnHelper::create_nullable_block( + {0}, {1}); // key=NULL, value placeholder + block2.insert(ColumnHelper::create_column_with_name({10})); + st = sink_operator->sink(_helper.runtime_state.get(), &block2, false); + ASSERT_TRUE(st.ok()) << "second sink failed: " << st.to_string(); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 1); + + // EOS: send an empty block with eos=true. + // Old code: aggregate_data_container->total_count() = 0 → SKIP flush → NULL key row LOST! + // New code: get_hash_table_size() = 1 → trigger flush → NULL key row saved. + Block empty_block; + st = sink_operator->sink(_helper.runtime_state.get(), &empty_block, true); + ASSERT_TRUE(st.ok()) << "EOS sink failed: " << st.to_string(); + + // Hash table must be empty after EOS flush. + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); + ASSERT_FALSE(dep->is_blocked_by()); + + // Two NULL key aggregated rows were spilled (one per revoke/flush cycle). + ASSERT_EQ(spill_write_rows_counter->value(), 2); + + st = sink_operator->close(_helper.runtime_state.get(), st); + ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); } -} // namespace doris \ No newline at end of file +} // namespace doris::pipeline \ No newline at end of file diff --git a/be/test/exec/operator/partitioned_aggregation_source_operator_test.cpp b/be/test/exec/operator/partitioned_aggregation_source_operator_test.cpp index a0de1f9521d60e..f331b529ce5d5d 100644 --- a/be/test/exec/operator/partitioned_aggregation_source_operator_test.cpp +++ b/be/test/exec/operator/partitioned_aggregation_source_operator_test.cpp @@ -24,6 +24,7 @@ #include "common/config.h" #include "core/block/block.h" #include "core/data_type/data_type_number.h" +#include "exec/common/agg_utils.h" #include "exec/operator/aggregation_sink_operator.h" #include "exec/operator/operator.h" #include "exec/operator/partitioned_aggregation_sink_operator.h" @@ -60,9 +61,9 @@ TEST_F(PartitionedAggregationSourceOperatorTest, Init) { std::shared_ptr shared_state = MockPartitionedAggSharedState::create_shared(); - shared_state->in_mem_shared_state_sptr = std::make_shared(); - shared_state->in_mem_shared_state = - reinterpret_cast(shared_state->in_mem_shared_state_sptr.get()); + shared_state->_in_mem_shared_state_sptr = std::make_shared(); + shared_state->_in_mem_shared_state = + reinterpret_cast(shared_state->_in_mem_shared_state_sptr.get()); LocalStateInfo info { .parent_profile = _helper.operator_profile.get(), @@ -138,8 +139,6 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockEmpty) { _helper.runtime_state->get_local_state(source_operator->operator_id())); ASSERT_TRUE(local_state != nullptr); - local_state->_copy_shared_spill_profile = false; - st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); @@ -207,11 +206,9 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlock) { st = sink_operator->sink(_helper.runtime_state.get(), &block, true); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - ASSERT_GT(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); - auto* inner_sink_local_state = reinterpret_cast( sink_local_state->_runtime_state->get_sink_local_state()); - ASSERT_GT(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_GT(inner_sink_local_state->get_hash_table_size(), 0); LocalStateInfo info { .parent_profile = _helper.operator_profile.get(), @@ -227,8 +224,6 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlock) { _helper.runtime_state->get_local_state(source_operator->operator_id())); ASSERT_TRUE(local_state != nullptr); - local_state->_copy_shared_spill_profile = false; - st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); @@ -295,10 +290,10 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithSpill) { st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); - ASSERT_TRUE(shared_state->is_spilled); + ASSERT_TRUE(shared_state->_is_spilled); st = sink_operator->sink(_helper.runtime_state.get(), &block, true); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); @@ -307,7 +302,7 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithSpill) { auto* inner_sink_local_state = reinterpret_cast( sink_local_state->_runtime_state->get_sink_local_state()); - ASSERT_EQ(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); LocalStateInfo info { .parent_profile = _helper.operator_profile.get(), @@ -323,8 +318,6 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithSpill) { _helper.runtime_state->get_local_state(source_operator->operator_id())); ASSERT_TRUE(local_state != nullptr); - local_state->_copy_shared_spill_profile = false; - st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); @@ -396,10 +389,10 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithSpillError) { st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); - ASSERT_TRUE(shared_state->is_spilled); + ASSERT_TRUE(shared_state->_is_spilled); st = sink_operator->sink(_helper.runtime_state.get(), &block, true); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); @@ -408,7 +401,7 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithSpillError) { auto* inner_sink_local_state = reinterpret_cast( sink_local_state->_runtime_state->get_sink_local_state()); - ASSERT_EQ(inner_sink_local_state->_get_hash_table_size(), 0); + ASSERT_EQ(inner_sink_local_state->get_hash_table_size(), 0); LocalStateInfo info { .parent_profile = _helper.operator_profile.get(), @@ -424,12 +417,10 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithSpillError) { _helper.runtime_state->get_local_state(source_operator->operator_id())); ASSERT_TRUE(local_state != nullptr); - local_state->_copy_shared_spill_profile = false; - st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); - SpillableDebugPointHelper dp_helper("fault_inject::spill_stream::read_next_block"); + SpillableDebugPointHelper dp_helper("fault_inject::spill_file::read_next_block"); block.clear(); bool eos = false; @@ -441,4 +432,617 @@ TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithSpillError) { ASSERT_FALSE(st.ok()); } + +// Test spill → recover cycle with large data to verify all rows come back. +TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithSpillLargeData) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + const auto tnode = _helper.create_test_plan_node(); + auto st = source_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = source_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), "PartitionedAggSinkTestDep"); + + LocalSinkStateInfo sink_info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), sink_info); + ASSERT_TRUE(st.ok()); + + auto* sink_local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(sink_local_state != nullptr); + st = sink_local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + // Create data with many distinct values to test larger spill sizes + const size_t count = 10000; + std::vector data(count); + std::iota(data.begin(), data.end(), 0); + auto block = ColumnHelper::create_block(data); + block.insert(ColumnHelper::create_column_with_name(data)); + + st = sink_operator->sink(_helper.runtime_state.get(), &block, false); + ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); + + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); + ASSERT_TRUE(shared_state->_is_spilled); + + // Sink EOS + block.clear_column_data(); + st = sink_operator->sink(_helper.runtime_state.get(), &block, true); + ASSERT_TRUE(st.ok()); + + // Now read back via source + LocalStateInfo info {.parent_profile = _helper.operator_profile.get(), + .scan_ranges = {}, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .task_idx = 0}; + st = source_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()); + + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_local_state(source_operator->operator_id())); + ASSERT_TRUE(local_state != nullptr); + st = local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + block.clear(); + bool eos = false; + size_t rows = 0; + while (!eos) { + st = source_operator->get_block(_helper.runtime_state.get(), &block, &eos); + ASSERT_TRUE(st.ok()) << "get_block failed: " << st.to_string(); + rows += block.rows(); + block.clear_column_data(); + } + + ASSERT_TRUE(eos); + // With GROUP BY, all distinct keys should come back + ASSERT_EQ(rows, count) << "Expected " << count << " distinct rows"; + + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); +} + +// Test multiple spill+recover cycles: sink → revoke → sink more → revoke → eos → source. +TEST_F(PartitionedAggregationSourceOperatorTest, GetBlockWithMultipleSpills) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + const auto tnode = _helper.create_test_plan_node(); + auto st = source_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = source_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), "PartitionedAggSinkTestDep"); + + LocalSinkStateInfo sink_info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), sink_info); + ASSERT_TRUE(st.ok()); + + auto* sink_local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(sink_local_state != nullptr); + st = sink_local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + // Round 1: sink {1,2,3,4} → revoke + auto block = ColumnHelper::create_block({1, 2, 3, 4}); + block.insert(ColumnHelper::create_column_with_name({1, 2, 3, 4})); + st = sink_operator->sink(_helper.runtime_state.get(), &block, false); + ASSERT_TRUE(st.ok()); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + ASSERT_TRUE(shared_state->_is_spilled); + + // Round 2: sink {5,6,7,8} → revoke + block = ColumnHelper::create_block({5, 6, 7, 8}); + block.insert(ColumnHelper::create_column_with_name({5, 6, 7, 8})); + st = sink_operator->sink(_helper.runtime_state.get(), &block, false); + ASSERT_TRUE(st.ok()); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + // Sink EOS + block.clear_column_data(); + st = sink_operator->sink(_helper.runtime_state.get(), &block, true); + ASSERT_TRUE(st.ok()); + + // Read back via source + LocalStateInfo info {.parent_profile = _helper.operator_profile.get(), + .scan_ranges = {}, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .task_idx = 0}; + st = source_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()); + + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_local_state(source_operator->operator_id())); + ASSERT_TRUE(local_state != nullptr); + st = local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + block.clear(); + bool eos = false; + size_t rows = 0; + while (!eos) { + st = source_operator->get_block(_helper.runtime_state.get(), &block, &eos); + ASSERT_TRUE(st.ok()) << "get_block failed: " << st.to_string(); + rows += block.rows(); + block.clear_column_data(); + } + + ASSERT_TRUE(eos); + ASSERT_EQ(rows, 8) << "Should recover all 8 distinct rows from 2 spill rounds"; + + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); +} +// --- Tests for PartitionedAggSourceOperatorX::revocable_mem_size --- +// The function checks two early-exit conditions then sums three memory sources: +// (1) !_is_spilled → return 0 +// (2) !_current_partition.spill_file → return 0 +// bytes = sum(_blocks.allocated_bytes) + hash_table_bytes + container.memory_usage() +// return bytes > spill_min_revocable_mem() ? bytes : 0 +// +// create_operators() sets min_revocable_mem=0, so effective threshold = max(0, 1MB) = 1MB. + +// Condition 1: not spilled → immediate 0. +TEST_F(PartitionedAggregationSourceOperatorTest, RevocableMemSizeNotSpilledReturnsZero) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + + std::shared_ptr shared_state; + _helper.create_source_local_state(_helper.runtime_state.get(), source_operator.get(), + shared_state); + + shared_state->_is_spilled = false; + EXPECT_EQ(source_operator->revocable_mem_size(_helper.runtime_state.get()), 0UL); +} + +// Condition 2: spilled but _current_partition.spill_file is null → 0. +TEST_F(PartitionedAggregationSourceOperatorTest, RevocableMemSizeSpilledNoSpillFileReturnsZero) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + + std::shared_ptr shared_state; + auto* local_state = _helper.create_source_local_state(_helper.runtime_state.get(), + source_operator.get(), shared_state); + + // create_source_local_state sets _is_spilled = true + ASSERT_TRUE(shared_state->_is_spilled.load()); + // _current_partition.spill_file defaults to nullptr + ASSERT_EQ(local_state->_current_partition.spill_file, nullptr); + EXPECT_EQ(source_operator->revocable_mem_size(_helper.runtime_state.get()), 0UL); +} + +// Spilled + valid spill_file, but total bytes < 1MB threshold → 0. +TEST_F(PartitionedAggregationSourceOperatorTest, + RevocableMemSizeWithSmallBlocksBelowThresholdReturnsZero) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + + std::shared_ptr shared_state; + auto* local_state = _helper.create_source_local_state(_helper.runtime_state.get(), + source_operator.get(), shared_state); + + auto small_block = ColumnHelper::create_block({1, 2, 3}); + ASSERT_LT(small_block.allocated_bytes(), 1UL << 20); // < 1MB + local_state->_blocks.push_back(std::move(small_block)); + + SpillFileSPtr spill_file; + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file("ut/revocable_small", spill_file) + .ok()); + local_state->_current_partition.spill_file = spill_file; + + EXPECT_EQ(source_operator->revocable_mem_size(_helper.runtime_state.get()), 0UL); +} + +// Spilled + valid spill_file + large blocks (>1MB) → returns block bytes. +TEST_F(PartitionedAggregationSourceOperatorTest, RevocableMemSizeWithLargeBlocksReturnsBytes) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + + std::shared_ptr shared_state; + auto* local_state = _helper.create_source_local_state(_helper.runtime_state.get(), + source_operator.get(), shared_state); + + std::vector large_data(300000); + std::iota(large_data.begin(), large_data.end(), 0); + auto large_block = ColumnHelper::create_block(large_data); + const size_t block_bytes = large_block.allocated_bytes(); + ASSERT_GT(block_bytes, 1UL << 20); // > 1MB threshold + local_state->_blocks.push_back(std::move(large_block)); + + SpillFileSPtr spill_file; + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file("ut/revocable_blocks", spill_file) + .ok()); + local_state->_current_partition.spill_file = spill_file; + + EXPECT_EQ(source_operator->revocable_mem_size(_helper.runtime_state.get()), block_bytes); +} + +// Spilled + valid spill_file + aggregate_data_container (>1MB) → returns container bytes. +TEST_F(PartitionedAggregationSourceOperatorTest, RevocableMemSizeWithAggContainerCountsMemory) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + + std::shared_ptr shared_state; + auto* local_state = _helper.create_source_local_state(_helper.runtime_state.get(), + source_operator.get(), shared_state); + + auto agg_sptr = std::make_shared(); + shared_state->_in_mem_shared_state_sptr = agg_sptr; + shared_state->_in_mem_shared_state = agg_sptr.get(); + agg_sptr->aggregate_data_container = + std::make_unique(sizeof(uint32_t), 8); + // ~13 sub-containers of 8192 entries each ≈ 1.28 MB → exceeds 1MB threshold + for (uint32_t i = 0; i < 100000; ++i) { + agg_sptr->aggregate_data_container->append_data(i); + } + const size_t container_bytes = agg_sptr->aggregate_data_container->memory_usage(); + ASSERT_GT(container_bytes, 1UL << 20); + + SpillFileSPtr spill_file; + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file("ut/revocable_container", spill_file) + .ok()); + local_state->_current_partition.spill_file = spill_file; + + EXPECT_EQ(source_operator->revocable_mem_size(_helper.runtime_state.get()), container_bytes); +} + +// --- Tests for PartitionedAggSourceOperatorX::revoke_memory --- +// revoke_memory: +// if (!_is_spilled) return OK (no-op) +// else: _flush_and_repartition → reset _current_partition + _need_to_setup_partition + +// Path 1: not spilled → immediate OK with no state change. +TEST_F(PartitionedAggregationSourceOperatorTest, RevokeMemoryNotSpilledIsNoOp) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + + std::shared_ptr shared_state; + auto* local_state = _helper.create_source_local_state(_helper.runtime_state.get(), + source_operator.get(), shared_state); + shared_state->_is_spilled = false; + + const bool partition_before = local_state->_need_to_setup_partition; + auto st = source_operator->revoke_memory(_helper.runtime_state.get()); + EXPECT_TRUE(st.ok()) << st.to_string(); + // State must be unchanged + EXPECT_EQ(local_state->_current_partition.spill_file, nullptr); + EXPECT_EQ(local_state->_need_to_setup_partition, partition_before); +} + +// Path 2: spilled → _flush_and_repartition runs, current partition is reset. +TEST_F(PartitionedAggregationSourceOperatorTest, RevokeMemorySpilledResetsPartitionState) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + ASSERT_TRUE(sink_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(sink_operator->prepare(_helper.runtime_state.get()).ok()); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), "RevokeMemoryTestDep"); + + LocalSinkStateInfo sink_info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + ASSERT_TRUE(sink_operator->setup_local_state(_helper.runtime_state.get(), sink_info).ok()); + auto* sink_local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(sink_local_state->open(_helper.runtime_state.get()).ok()); + + auto block = ColumnHelper::create_block({1, 2, 3, 4}); + block.insert(ColumnHelper::create_column_with_name({1, 2, 3, 4})); + ASSERT_TRUE(sink_operator->sink(_helper.runtime_state.get(), &block, false).ok()); + ASSERT_TRUE(sink_operator->revoke_memory(_helper.runtime_state.get()).ok()); + ASSERT_TRUE(shared_state->_is_spilled); + ASSERT_TRUE(sink_operator->sink(_helper.runtime_state.get(), &block, true).ok()); + + LocalStateInfo info {.parent_profile = _helper.operator_profile.get(), + .scan_ranges = {}, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .task_idx = 0}; + ASSERT_TRUE(source_operator->setup_local_state(_helper.runtime_state.get(), info).ok()); + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_local_state(source_operator->operator_id())); + ASSERT_TRUE(local_state != nullptr); + ASSERT_TRUE(local_state->open(_helper.runtime_state.get()).ok()); + + // First get_block: sets up partition queue and recovers blocks from disk (yields without merging) + Block out; + bool eos = false; + ASSERT_TRUE(source_operator->get_block(_helper.runtime_state.get(), &out, &eos).ok()); + ASSERT_FALSE(eos); + + // revoke_memory flushes the (empty) hash table and repartitions recovered _blocks + auto st = source_operator->revoke_memory(_helper.runtime_state.get()); + EXPECT_TRUE(st.ok()) << st.to_string(); + + // Current partition must be reset; _need_to_setup_partition set back to true + EXPECT_EQ(local_state->_current_partition.spill_file, nullptr); + EXPECT_TRUE(local_state->_need_to_setup_partition); + + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); +} + +// Path 3: spilled but partition level has reached max depth → InternalError. +// Uses -fno-access-control to set the private level field directly. +TEST_F(PartitionedAggregationSourceOperatorTest, RevokeMemoryAtMaxDepthReturnsError) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + + std::shared_ptr shared_state; + auto* local_state = _helper.create_source_local_state(_helper.runtime_state.get(), + source_operator.get(), shared_state); + // _is_spilled = true is already set by create_source_local_state. + + // _flush_and_repartition checks: new_level = level + 1 >= _repartition_max_depth. + // SpillRepartitioner::MAX_DEPTH = 8; set level = 7 so new_level = 8 >= 8 → error. + local_state->_current_partition.level = + static_cast(source_operator->_repartition_max_depth) - 1; + + auto st = source_operator->revoke_memory(_helper.runtime_state.get()); + EXPECT_FALSE(st.ok()); + EXPECT_EQ(st.code(), ErrorCode::INTERNAL_ERROR); +} + +// --- Tests for PartitionedAggLocalState::_flush_hash_table_to_sub_spill_files --- +// The function: +// 1. Calls aggregate_data_container->init_once() (requires non-null container). +// 2. Loops calling _agg_source_operator->get_serialized_block(); non-empty blocks are +// routed via _repartitioner.route_block() (repartitioner must be set up first). +// 3. Calls _agg_source_operator->reset_hash_table() to clear the hash table. +// +// Tests use the full pipeline path (sink → revoke → source open) to ensure +// _runtime_state and _in_mem_shared_state are valid. + +// Test 1: Hash table has merged data → blocks are serialised and routed successfully. +// Repartitioner is initialised manually before calling the function. +TEST_F(PartitionedAggregationSourceOperatorTest, FlushHashTableToSubSpillFilesSucceeds) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + ASSERT_TRUE(sink_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(sink_operator->prepare(_helper.runtime_state.get()).ok()); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), "FlushHtTestDep"); + + LocalSinkStateInfo sink_info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + ASSERT_TRUE(sink_operator->setup_local_state(_helper.runtime_state.get(), sink_info).ok()); + auto* sink_local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(sink_local_state->open(_helper.runtime_state.get()).ok()); + + // Sink data then spill so the source path is exercised. + auto block = ColumnHelper::create_block({1, 2, 3, 4, 2, 3, 4, 3, 4, 4}); + block.insert( + ColumnHelper::create_column_with_name({1, 2, 3, 4, 2, 3, 4, 3, 4, 4})); + ASSERT_TRUE(sink_operator->sink(_helper.runtime_state.get(), &block, false).ok()); + ASSERT_TRUE(sink_operator->revoke_memory(_helper.runtime_state.get()).ok()); + ASSERT_TRUE(shared_state->_is_spilled); + block.clear_column_data(); + ASSERT_TRUE(sink_operator->sink(_helper.runtime_state.get(), &block, true).ok()); + + // Open source operator to set up _runtime_state and _in_mem_shared_state. + LocalStateInfo info {.parent_profile = _helper.operator_profile.get(), + .scan_ranges = {}, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .task_idx = 0}; + ASSERT_TRUE(source_operator->setup_local_state(_helper.runtime_state.get(), info).ok()); + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_local_state(source_operator->operator_id())); + ASSERT_TRUE(local_state != nullptr); + ASSERT_TRUE(local_state->open(_helper.runtime_state.get()).ok()); + + // Phase 2: first get_block recovers serialised blocks from disk into _blocks. + Block out; + bool eos = false; + ASSERT_TRUE(source_operator->get_block(_helper.runtime_state.get(), &out, &eos).ok()); + ASSERT_FALSE(eos); + + // Phase 3: second get_block merges _blocks into the hash table. + // After this call aggregate_data_container is non-null and the hash table has data. + out.clear(); + ASSERT_TRUE(source_operator->get_block(_helper.runtime_state.get(), &out, &eos).ok()); + ASSERT_FALSE(eos); + + auto* in_mem_state = shared_state->_in_mem_shared_state; + ASSERT_NE(in_mem_state, nullptr); + ASSERT_NE(in_mem_state->aggregate_data_container, nullptr); + + // Set up the repartitioner the same way _flush_and_repartition does. + const int new_level = local_state->_current_partition.level + 1; + const int fanout = static_cast(source_operator->_partition_count); + size_t num_keys = in_mem_state->probe_expr_ctxs.size(); + std::vector key_column_indices(num_keys); + std::vector key_data_types(num_keys); + for (size_t i = 0; i < num_keys; ++i) { + key_column_indices[i] = i; + key_data_types[i] = in_mem_state->probe_expr_ctxs[i]->root()->data_type(); + } + std::vector output_spill_files; + ASSERT_TRUE(SpillRepartitioner::create_output_spill_files( + _helper.runtime_state.get(), source_operator->node_id(), "ut/flush_ht_test", + fanout, output_spill_files) + .ok()); + local_state->_repartitioner.init_with_key_columns( + key_column_indices, key_data_types, local_state->operator_profile(), fanout, new_level); + ASSERT_TRUE(local_state->_repartitioner + .setup_output(_helper.runtime_state.get(), output_spill_files) + .ok()); + + // Call the function under test. + auto st = local_state->_flush_hash_table_to_sub_spill_files(_helper.runtime_state.get()); + EXPECT_TRUE(st.ok()) << st.to_string(); + + // Flush writers and release file resources. + ASSERT_TRUE(local_state->_repartitioner.finalize().ok()); + + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); +} + +// Test 2: No merge phase — hash table is in monostate (no rows aggregated yet). +// get_serialized_block returns inner_eos=true immediately → route_block is never called, +// so the repartitioner does not need setup_output. reset_hash_table is still called. +TEST_F(PartitionedAggregationSourceOperatorTest, + FlushHashTableToSubSpillFilesEmptyHashTableSucceeds) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + ASSERT_TRUE(sink_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(sink_operator->prepare(_helper.runtime_state.get()).ok()); + + auto shared_state = std::dynamic_pointer_cast( + sink_operator->create_shared_state()); + shared_state->create_source_dependency(source_operator->operator_id(), + source_operator->node_id(), "FlushHtEmptyDep"); + + LocalSinkStateInfo sink_info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = TDataSink()}; + ASSERT_TRUE(sink_operator->setup_local_state(_helper.runtime_state.get(), sink_info).ok()); + auto* sink_local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(sink_local_state->open(_helper.runtime_state.get()).ok()); + + // Spill data to disk so the source enters the spilled path on open. + auto block = ColumnHelper::create_block({1, 2, 3, 4}); + block.insert(ColumnHelper::create_column_with_name({1, 2, 3, 4})); + ASSERT_TRUE(sink_operator->sink(_helper.runtime_state.get(), &block, false).ok()); + ASSERT_TRUE(sink_operator->revoke_memory(_helper.runtime_state.get()).ok()); + ASSERT_TRUE(shared_state->_is_spilled); + block.clear_column_data(); + ASSERT_TRUE(sink_operator->sink(_helper.runtime_state.get(), &block, true).ok()); + + // Open source operator: _runtime_state and _in_mem_shared_state are now valid + // with a freshly created, empty hash table (no data merged yet). + LocalStateInfo info {.parent_profile = _helper.operator_profile.get(), + .scan_ranges = {}, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .task_idx = 0}; + ASSERT_TRUE(source_operator->setup_local_state(_helper.runtime_state.get(), info).ok()); + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_local_state(source_operator->operator_id())); + ASSERT_TRUE(local_state != nullptr); + ASSERT_TRUE(local_state->open(_helper.runtime_state.get()).ok()); + + // aggregate_data_container is initialised by the sink's open phase (not null). + // The hash table is in monostate (no data merged via merge_with_serialized_key_helper), + // so get_serialized_block returns inner_eos=true immediately and route_block is + // never reached — the repartitioner does not need setup_output. + auto* in_mem_state = shared_state->_in_mem_shared_state; + ASSERT_NE(in_mem_state, nullptr); + ASSERT_NE(in_mem_state->aggregate_data_container, nullptr); + auto st = local_state->_flush_hash_table_to_sub_spill_files(_helper.runtime_state.get()); + EXPECT_TRUE(st.ok()) << st.to_string(); + + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); +} + +// Null _in_mem_shared_state: only block bytes contribute, hash table + container skipped. +TEST_F(PartitionedAggregationSourceOperatorTest, RevocableMemSizeNullInMemStateSkipsHashTable) { + auto [source_operator, sink_operator] = _helper.create_operators(); + const auto tnode = _helper.create_test_plan_node(); + ASSERT_TRUE(source_operator->init(tnode, _helper.runtime_state.get()).ok()); + ASSERT_TRUE(source_operator->prepare(_helper.runtime_state.get()).ok()); + + std::shared_ptr shared_state; + auto* local_state = _helper.create_source_local_state(_helper.runtime_state.get(), + source_operator.get(), shared_state); + + std::vector large_data(300000); + std::iota(large_data.begin(), large_data.end(), 0); + auto large_block = ColumnHelper::create_block(large_data); + const size_t block_bytes = large_block.allocated_bytes(); + local_state->_blocks.push_back(std::move(large_block)); + + SpillFileSPtr spill_file; + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file("ut/revocable_null_state", spill_file) + .ok()); + local_state->_current_partition.spill_file = spill_file; + + // _in_mem_shared_state is null → hash table and container contribute 0 bytes + ASSERT_EQ(shared_state->_in_mem_shared_state, nullptr); + EXPECT_EQ(source_operator->revocable_mem_size(_helper.runtime_state.get()), block_bytes); +} + } // namespace doris diff --git a/be/test/exec/operator/partitioned_aggregation_test_helper.cpp b/be/test/exec/operator/partitioned_aggregation_test_helper.cpp index 8fdeeadbd5d34b..eae2f22b0608f0 100644 --- a/be/test/exec/operator/partitioned_aggregation_test_helper.cpp +++ b/be/test/exec/operator/partitioned_aggregation_test_helper.cpp @@ -160,6 +160,10 @@ std::tuple, std::shared_ptr> PartitionedAggregationTestHelper::create_operators() { TPlanNode tnode = create_test_plan_node(); + TQueryOptions query_options = runtime_state->query_options(); + query_options.__set_min_revocable_mem(0); + runtime_state->set_query_options(query_options); + auto desc_tbl = runtime_state->desc_tbl(); EXPECT_EQ(desc_tbl.get_tuple_descs().size(), 3); @@ -196,13 +200,12 @@ PartitionedAggLocalState* PartitionedAggregationTestHelper::create_source_local_ auto* local_state = local_state_uptr.get(); shared_state = std::make_shared(); local_state->_shared_state = shared_state.get(); - shared_state->is_spilled = true; + shared_state->_is_spilled = true; ADD_TIMER(local_state->common_profile(), "ExecTime"); local_state->common_profile()->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 0); local_state->init_spill_read_counters(); local_state->init_spill_write_counters(); - local_state->_copy_shared_spill_profile = false; local_state->_internal_runtime_profile = std::make_unique("inner_test"); state->emplace_local_state(probe_operator->operator_id(), std::move(local_state_uptr)); @@ -225,8 +228,6 @@ PartitionedAggSinkLocalState* PartitionedAggregationTestHelper::create_sink_loca sink_operator->dests_id().front(), sink_operator->operator_id(), "PartitionedHashJoinTestDep"); - shared_state->setup_shared_profile(local_state->custom_profile()); - state->emplace_sink_local_state(sink_operator->operator_id(), std::move(local_state_uptr)); return local_state; } diff --git a/be/test/exec/operator/partitioned_aggregation_test_helper.h b/be/test/exec/operator/partitioned_aggregation_test_helper.h index ee94a7dd4b4b9a..5ecfe8dd29771f 100644 --- a/be/test/exec/operator/partitioned_aggregation_test_helper.h +++ b/be/test/exec/operator/partitioned_aggregation_test_helper.h @@ -35,7 +35,7 @@ #include "exec/operator/partitioned_aggregation_source_operator.h" #include "exec/operator/spillable_operator_test_helper.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" @@ -48,7 +48,7 @@ class MockPartitionedAggSharedState : public PartitionedAggSharedState { ENABLE_FACTORY_CREATOR(MockPartitionedAggSharedState); public: - MockPartitionedAggSharedState() { is_spilled = false; } + MockPartitionedAggSharedState() { _is_spilled = false; } }; class MockPartitionedAggSinkLocalState : public PartitionedAggSinkLocalState { diff --git a/be/test/exec/operator/partitioned_hash_join_probe_operator_test.cpp b/be/test/exec/operator/partitioned_hash_join_probe_operator_test.cpp index aceade5473f2a1..a29f44bb533f6e 100644 --- a/be/test/exec/operator/partitioned_hash_join_probe_operator_test.cpp +++ b/be/test/exec/operator/partitioned_hash_join_probe_operator_test.cpp @@ -28,7 +28,7 @@ #include "exec/operator/partitioned_hash_join_sink_operator.h" #include "exec/operator/partitioned_hash_join_test_helper.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" @@ -47,6 +47,83 @@ class PartitionedHashJoinProbeOperatorTest : public testing::Test { PartitionedHashJoinTestHelper _helper; }; +namespace { + +SpillFileSPtr create_probe_test_spill_file(RuntimeState* state, RuntimeProfile* profile, + int node_id, const std::string& prefix, + const std::vector>& batches) { + SpillFileSPtr spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}", print_id(state->query_id()), prefix, node_id, + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + auto st = + ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, spill_file); + EXPECT_TRUE(st.ok()) << "create spill file failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(state, profile, writer); + EXPECT_TRUE(st.ok()) << "create writer failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + for (const auto& batch : batches) { + Block block = ColumnHelper::create_block(batch); + st = writer->write_block(state, block); + EXPECT_TRUE(st.ok()) << "write block failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + } + + st = writer->close(); + EXPECT_TRUE(st.ok()) << "close writer failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + return spill_file; +} + +int64_t count_spill_rows(RuntimeState* state, RuntimeProfile* profile, + const SpillFileSPtr& spill_file) { + auto reader = spill_file->create_reader(state, profile); + auto st = reader->open(); + EXPECT_TRUE(st.ok()) << "open reader failed: " << st.to_string(); + if (!st.ok()) { + return 0; + } + + int64_t rows = 0; + bool eos = false; + while (!eos) { + Block block; + st = reader->read(&block, &eos); + EXPECT_TRUE(st.ok()) << "read block failed: " << st.to_string(); + if (!st.ok()) { + return rows; + } + rows += block.rows(); + } + st = reader->close(); + EXPECT_TRUE(st.ok()) << "close reader failed: " << st.to_string(); + return rows; +} + +Status prepare_probe_local_state_for_repartition(PartitionedHashJoinProbeOperatorX* probe_operator, + PartitionedHashJoinProbeLocalState* local_state, + RuntimeState* state) { + RETURN_IF_ERROR(probe_operator->init(probe_operator->_tnode, state)); + probe_operator->_inner_sink_operator->_child = nullptr; + probe_operator->_inner_probe_operator->_child = nullptr; + probe_operator->_inner_probe_operator->_build_side_child = nullptr; + RETURN_IF_ERROR(probe_operator->prepare(state)); + return local_state->open(state); +} + +} // namespace + TEST_F(PartitionedHashJoinProbeOperatorTest, debug_string) { auto [probe_operator, sink_operator] = _helper.create_operators(); @@ -57,7 +134,7 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, debug_string) { auto debug_string = local_state->debug_string(0); std::cout << "debug string: " << debug_string << std::endl; - shared_state->is_spilled = false; + shared_state->_is_spilled = false; debug_string = local_state->debug_string(0); std::cout << "debug string: " << debug_string << std::endl; @@ -99,11 +176,11 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, InitAndOpen) { st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st) << "open failed: " << st.to_string(); - local_state->_shared_state->inner_shared_state = std::make_shared(); - local_state->_shared_state->inner_runtime_state = std::make_unique(); - local_state->_shared_state->inner_runtime_state->set_desc_tbl( + local_state->_shared_state->_inner_shared_state = std::make_shared(); + local_state->_shared_state->_inner_runtime_state = std::make_unique(); + local_state->_shared_state->_inner_runtime_state->set_desc_tbl( &(_helper.runtime_state->desc_tbl())); - local_state->_shared_state->inner_runtime_state->resize_op_id_to_local_state(-100); + local_state->_shared_state->_inner_runtime_state->resize_op_id_to_local_state(-100); auto mock_inner_sink_operator = probe_operator->_inner_sink_operator; probe_operator->_inner_sink_operator = std::make_shared( @@ -116,32 +193,32 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, InitAndOpen) { auto inner_probe_state = std::make_unique( _helper.runtime_state.get(), probe_operator->_inner_probe_operator.get()); - st = inner_probe_state->init(local_state->_shared_state->inner_runtime_state.get(), info); + st = inner_probe_state->init(local_state->_shared_state->_inner_runtime_state.get(), info); ASSERT_TRUE(st) << "init failed: " << st.to_string(); - local_state->_shared_state->inner_runtime_state->emplace_local_state( + local_state->_shared_state->_inner_runtime_state->emplace_local_state( probe_operator->_inner_probe_operator->operator_id(), std::move(inner_probe_state)); auto inner_sink_state = std::make_unique( probe_operator->_inner_sink_operator.get(), _helper.runtime_state.get()); LocalSinkStateInfo sink_info {0, _helper.operator_profile.get(), - -1, local_state->_shared_state->inner_shared_state.get(), + -1, local_state->_shared_state->_inner_shared_state.get(), {}, {}}; st = probe_operator->_inner_sink_operator->prepare( - local_state->_shared_state->inner_runtime_state.get()); + local_state->_shared_state->_inner_runtime_state.get()); ASSERT_TRUE(st) << "prepare failed: " << st.to_string(); - st = inner_sink_state->init(local_state->_shared_state->inner_runtime_state.get(), sink_info); + st = inner_sink_state->init(local_state->_shared_state->_inner_runtime_state.get(), sink_info); ASSERT_TRUE(st) << "init failed: " << st.to_string(); - local_state->_shared_state->inner_runtime_state->emplace_sink_local_state( + local_state->_shared_state->_inner_runtime_state->emplace_sink_local_state( 0, std::move(inner_sink_state)); - local_state->_shared_state->is_spilled = false; + local_state->_shared_state->_is_spilled = false; local_state->update_profile_from_inner(); - local_state->_shared_state->is_spilled = true; + local_state->_shared_state->_is_spilled = true; local_state->update_profile_from_inner(); st = local_state->close(_helper.runtime_state.get()); @@ -158,6 +235,410 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, InitAndOpen) { ASSERT_TRUE(st) << "close failed: " << st.to_string(); } +TEST_F(PartitionedHashJoinProbeOperatorTest, CloseReleasesSpillResources) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + auto create_spill_file = [&](const std::string& prefix, + std::initializer_list values) -> SpillFileSPtr { + SpillFileSPtr spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}", print_id(_helper.runtime_state->query_id()), + prefix, probe_operator->node_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + spill_file); + EXPECT_TRUE(st.ok()) << "create spill file failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_helper.runtime_state.get(), local_state->operator_profile(), + writer); + EXPECT_TRUE(st.ok()) << "create spill writer failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + Block block = ColumnHelper::create_block(values); + st = writer->write_block(_helper.runtime_state.get(), block); + EXPECT_TRUE(st.ok()) << "write spill block failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + st = writer->close(); + EXPECT_TRUE(st.ok()) << "close spill writer failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + return spill_file; + }; + auto expect_spill_file_deleted = [&](const SpillFileSPtr& spill_file) { + auto reader = spill_file->create_reader(_helper.runtime_state.get(), + local_state->operator_profile()); + auto st = reader->open(); + EXPECT_FALSE(st.ok()) << "spill file should have been deleted"; + }; + + auto queued_build_file = create_spill_file("hash_build_close_queue", {1, 2, 3}); + auto queued_probe_file = create_spill_file("hash_probe_close_queue", {4, 5, 6}); + auto current_build_file = create_spill_file("hash_build_close_current", {7, 8, 9}); + auto current_probe_file = create_spill_file("hash_probe_close_current", {10, 11, 12}); + + ASSERT_TRUE(queued_build_file != nullptr); + ASSERT_TRUE(queued_probe_file != nullptr); + ASSERT_TRUE(current_build_file != nullptr); + ASSERT_TRUE(current_probe_file != nullptr); + + SpillFileWriterSPtr writer; + auto st = local_state->acquire_spill_writer(_helper.runtime_state.get(), 0, writer); + ASSERT_TRUE(st.ok()) << "acquire spill writer failed: " << st.to_string(); + Block writer_block = ColumnHelper::create_block({13, 14, 15}); + st = writer->write_block(_helper.runtime_state.get(), writer_block); + ASSERT_TRUE(st.ok()) << "write spill block failed: " << st.to_string(); + + local_state->_spill_partition_queue.emplace_back(queued_build_file, queued_probe_file, 2); + local_state->_current_partition = + JoinSpillPartitionInfo(current_build_file, current_probe_file, 1); + local_state->_queue_probe_blocks.emplace_back( + ColumnHelper::create_block({16, 17, 18})); + + local_state->_current_build_reader = current_build_file->create_reader( + _helper.runtime_state.get(), local_state->operator_profile()); + st = local_state->_current_build_reader->open(); + ASSERT_TRUE(st.ok()) << "open current build reader failed: " << st.to_string(); + + local_state->_current_probe_reader = current_probe_file->create_reader( + _helper.runtime_state.get(), local_state->operator_profile()); + st = local_state->_current_probe_reader->open(); + ASSERT_TRUE(st.ok()) << "open current probe reader failed: " << st.to_string(); + + st = local_state->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); + + ASSERT_TRUE(local_state->_probe_writers.empty()); + ASSERT_TRUE(local_state->_probe_spilling_groups[0] != nullptr); + ASSERT_TRUE(local_state->_probe_spilling_groups[0]->ready_for_reading()); + ASSERT_EQ(local_state->_current_build_reader, nullptr); + ASSERT_EQ(local_state->_current_probe_reader, nullptr); + ASSERT_TRUE(local_state->_spill_partition_queue.empty()); + ASSERT_FALSE(local_state->_current_partition.is_valid()); + ASSERT_EQ(local_state->_current_partition.build_file, nullptr); + ASSERT_EQ(local_state->_current_partition.probe_file, nullptr); + ASSERT_TRUE(local_state->_queue_probe_blocks.empty()); + ASSERT_TRUE(local_state->_closed); + + expect_spill_file_deleted(queued_build_file); + expect_spill_file_deleted(queued_probe_file); + expect_spill_file_deleted(current_build_file); + expect_spill_file_deleted(current_probe_file); + + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file( + local_state->_probe_spilling_groups[0]); + local_state->_probe_spilling_groups[0].reset(); +} + +TEST_F(PartitionedHashJoinProbeOperatorTest, CloseReturnsWriterCloseError) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + SpillFileWriterSPtr writer; + auto st = local_state->acquire_spill_writer(_helper.runtime_state.get(), 0, writer); + ASSERT_TRUE(st.ok()) << "acquire spill writer failed: " << st.to_string(); + + local_state->_spill_partition_queue.emplace_back(JoinSpillPartitionInfo(nullptr, nullptr, 0)); + local_state->_current_partition = JoinSpillPartitionInfo(nullptr, nullptr, 1); + local_state->_queue_probe_blocks.emplace_back(ColumnHelper::create_block({1})); + + { + SpillableDebugPointHelper dp_helper("fault_inject::spill_file::spill_eof"); + st = local_state->close(_helper.runtime_state.get()); + } + ASSERT_FALSE(st.ok()); + ASSERT_TRUE(st.to_string().find("fault_inject spill_file spill_eof failed") != + std::string::npos) + << "unexpected error: " << st.to_string(); + + ASSERT_TRUE(local_state->_probe_writers.empty()); + ASSERT_TRUE(local_state->_spill_partition_queue.empty()); + ASSERT_FALSE(local_state->_current_partition.is_valid()); + ASSERT_TRUE(local_state->_queue_probe_blocks.empty()); + ASSERT_TRUE(local_state->_closed); + + st = local_state->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "reentrant close failed: " << st.to_string(); + + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file( + local_state->_probe_spilling_groups[0]); + local_state->_probe_spilling_groups[0].reset(); +} + +TEST_F(PartitionedHashJoinProbeOperatorTest, RepartitionCurrentPartition) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + auto st = probe_operator->init(probe_operator->_tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + probe_operator->_inner_sink_operator->_child = nullptr; + probe_operator->_inner_probe_operator->_child = nullptr; + probe_operator->_inner_probe_operator->_build_side_child = nullptr; + + st = probe_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + st = local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + auto create_spill_file = + [&](const std::string& prefix, + const std::vector>& batches) -> SpillFileSPtr { + SpillFileSPtr spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}", print_id(_helper.runtime_state->query_id()), + prefix, probe_operator->node_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + auto status = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + spill_file); + EXPECT_TRUE(status.ok()) << "create spill file failed: " << status.to_string(); + if (!status.ok()) { + return nullptr; + } + + SpillFileWriterSPtr writer; + status = spill_file->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer); + EXPECT_TRUE(status.ok()) << "create writer failed: " << status.to_string(); + if (!status.ok()) { + return nullptr; + } + + for (const auto& batch : batches) { + Block block = ColumnHelper::create_block(batch); + status = writer->write_block(_helper.runtime_state.get(), block); + EXPECT_TRUE(status.ok()) << "write block failed: " << status.to_string(); + if (!status.ok()) { + return nullptr; + } + } + + status = writer->close(); + EXPECT_TRUE(status.ok()) << "close writer failed: " << status.to_string(); + if (!status.ok()) { + return nullptr; + } + return spill_file; + }; + auto count_rows = [&](const SpillFileSPtr& spill_file) -> int64_t { + auto reader = spill_file->create_reader(_helper.runtime_state.get(), + local_state->operator_profile()); + auto status = reader->open(); + EXPECT_TRUE(status.ok()) << "open reader failed: " << status.to_string(); + if (!status.ok()) { + return 0; + } + + int64_t rows = 0; + bool eos = false; + while (!eos) { + Block block; + status = reader->read(&block, &eos); + EXPECT_TRUE(status.ok()) << "read block failed: " << status.to_string(); + if (!status.ok()) { + return rows; + } + rows += block.rows(); + } + status = reader->close(); + EXPECT_TRUE(status.ok()) << "close reader failed: " << status.to_string(); + return rows; + }; + + auto build_file = create_spill_file("hash_build_repartition_test", {{1, 2, 3}, {4, 5}}); + auto probe_file = create_spill_file("hash_probe_repartition_test", {{6, 7}, {8, 9, 10}}); + ASSERT_TRUE(build_file != nullptr); + ASSERT_TRUE(probe_file != nullptr); + + JoinSpillPartitionInfo partition(build_file, probe_file, 0); + local_state->_current_build_reader = + build_file->create_reader(_helper.runtime_state.get(), local_state->operator_profile()); + st = local_state->_current_build_reader->open(); + ASSERT_TRUE(st.ok()) << "open current build reader failed: " << st.to_string(); + + Block recovered_block; + bool eos = false; + st = local_state->_current_build_reader->read(&recovered_block, &eos); + ASSERT_TRUE(st.ok()) << "read recovered build block failed: " << st.to_string(); + ASSERT_FALSE(eos); + local_state->_recovered_build_block = MutableBlock::create_unique(std::move(recovered_block)); + + st = local_state->repartition_current_partition(_helper.runtime_state.get(), partition); + ASSERT_TRUE(st.ok()) << "repartition current partition failed: " << st.to_string(); + + ASSERT_EQ(partition.build_file, nullptr); + ASSERT_EQ(partition.probe_file, nullptr); + ASSERT_EQ(local_state->_recovered_build_block, nullptr); + ASSERT_EQ(local_state->_current_build_reader, nullptr); + ASSERT_EQ(local_state->_current_probe_reader, nullptr); + ASSERT_EQ(local_state->_spill_partition_queue.size(), + PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT); + ASSERT_EQ(local_state->_total_partition_spills->value(), + PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT); + ASSERT_EQ(local_state->_max_partition_level_seen, 1); + ASSERT_EQ(local_state->_max_partition_level->value(), 1); + + int64_t repartitioned_build_rows = 0; + int64_t repartitioned_probe_rows = 0; + for (auto& queue_partition : local_state->_spill_partition_queue) { + ASSERT_TRUE(queue_partition.is_valid()); + ASSERT_EQ(queue_partition.level, 1); + repartitioned_build_rows += count_rows(queue_partition.build_file); + repartitioned_probe_rows += count_rows(queue_partition.probe_file); + } + ASSERT_EQ(repartitioned_build_rows, 5); + ASSERT_EQ(repartitioned_probe_rows, 5); + + for (auto& queue_partition : local_state->_spill_partition_queue) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(queue_partition.build_file); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(queue_partition.probe_file); + } + local_state->_spill_partition_queue.clear(); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(build_file); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(probe_file); +} + +TEST_F(PartitionedHashJoinProbeOperatorTest, RepartitionCurrentPartitionExceedsMaxDepth) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + auto st = probe_operator->init(probe_operator->_tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + probe_operator->_repartition_max_depth = 1; + JoinSpillPartitionInfo partition(nullptr, nullptr, 0); + + st = local_state->repartition_current_partition(_helper.runtime_state.get(), partition); + ASSERT_FALSE(st.ok()); + ASSERT_TRUE(st.to_string().find("exceeded max depth 1") != std::string::npos) + << "unexpected error: " << st.to_string(); + ASSERT_TRUE(local_state->_spill_partition_queue.empty()); +} + +TEST_F(PartitionedHashJoinProbeOperatorTest, RevokeBuildData) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + auto st = prepare_probe_local_state_for_repartition(probe_operator.get(), local_state, + _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare probe local state failed: " << st.to_string(); + + auto build_file = create_probe_test_spill_file( + _helper.runtime_state.get(), local_state->operator_profile(), probe_operator->node_id(), + "hash_build_revoke_test", {{1, 2, 3}, {4, 5}}); + auto probe_file = create_probe_test_spill_file( + _helper.runtime_state.get(), local_state->operator_profile(), probe_operator->node_id(), + "hash_probe_revoke_test", {{6, 7}, {8, 9, 10}}); + ASSERT_TRUE(build_file != nullptr); + ASSERT_TRUE(probe_file != nullptr); + + local_state->_child_eos = true; + local_state->_spill_queue_initialized = true; + local_state->_need_to_setup_queue_partition = false; + local_state->_current_partition = JoinSpillPartitionInfo(build_file, probe_file, 0); + local_state->_queue_probe_blocks.emplace_back( + ColumnHelper::create_block({11, 12})); + + local_state->_current_build_reader = + build_file->create_reader(_helper.runtime_state.get(), local_state->operator_profile()); + st = local_state->_current_build_reader->open(); + ASSERT_TRUE(st.ok()) << "open current build reader failed: " << st.to_string(); + + Block recovered_block; + bool eos = false; + st = local_state->_current_build_reader->read(&recovered_block, &eos); + ASSERT_TRUE(st.ok()) << "read recovered build block failed: " << st.to_string(); + ASSERT_FALSE(eos); + local_state->_recovered_build_block = MutableBlock::create_unique(std::move(recovered_block)); + + st = local_state->revoke_build_data(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke build data failed: " << st.to_string(); + + ASSERT_FALSE(local_state->_current_partition.is_valid()); + ASSERT_TRUE(local_state->_need_to_setup_queue_partition); + ASSERT_TRUE(local_state->_queue_probe_blocks.empty()); + ASSERT_EQ(local_state->_recovered_build_block, nullptr); + ASSERT_EQ(local_state->_current_build_reader, nullptr); + ASSERT_EQ(local_state->_current_probe_reader, nullptr); + ASSERT_EQ(local_state->_spill_partition_queue.size(), + PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT); + ASSERT_EQ(local_state->_total_partition_spills->value(), + PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT); + ASSERT_EQ(local_state->_max_partition_level_seen, 1); + ASSERT_EQ(local_state->_max_partition_level->value(), 1); + + int64_t repartitioned_build_rows = 0; + int64_t repartitioned_probe_rows = 0; + for (auto& queue_partition : local_state->_spill_partition_queue) { + ASSERT_TRUE(queue_partition.is_valid()); + ASSERT_EQ(queue_partition.level, 1); + repartitioned_build_rows += + count_spill_rows(_helper.runtime_state.get(), local_state->operator_profile(), + queue_partition.build_file); + repartitioned_probe_rows += + count_spill_rows(_helper.runtime_state.get(), local_state->operator_profile(), + queue_partition.probe_file); + } + ASSERT_EQ(repartitioned_build_rows, 5); + ASSERT_EQ(repartitioned_probe_rows, 5); + + for (auto& queue_partition : local_state->_spill_partition_queue) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(queue_partition.build_file); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(queue_partition.probe_file); + } + local_state->_spill_partition_queue.clear(); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(build_file); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(probe_file); +} + +TEST_F(PartitionedHashJoinProbeOperatorTest, RevokeBuildDataPropagatesRepartitionError) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + probe_operator->_repartition_max_depth = 1; + local_state->_child_eos = true; + local_state->_spill_queue_initialized = true; + local_state->_need_to_setup_queue_partition = false; + local_state->_current_partition = JoinSpillPartitionInfo(nullptr, nullptr, 0); + local_state->_queue_probe_blocks.emplace_back(ColumnHelper::create_block({1})); + + auto st = local_state->revoke_build_data(_helper.runtime_state.get()); + ASSERT_FALSE(st.ok()); + ASSERT_TRUE(st.to_string().find("exceeded max depth 1") != std::string::npos) + << "unexpected error: " << st.to_string(); + ASSERT_TRUE(local_state->_current_partition.is_valid()); + ASSERT_FALSE(local_state->_need_to_setup_queue_partition); + ASSERT_EQ(local_state->_queue_probe_blocks.size(), 1); + ASSERT_TRUE(local_state->_spill_partition_queue.empty()); +} + TEST_F(PartitionedHashJoinProbeOperatorTest, spill_probe_blocks) { auto [probe_operator, sink_operator] = _helper.create_operators(); @@ -171,14 +652,14 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, spill_probe_blocks) { _helper.runtime_state.get(), PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT, {tnode.hash_join_node.eq_join_conjuncts[0].left}, row_desc); - // create probe blocks + // create probe blocks in _partitioned_blocks for (int32_t i = 0; i != PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT; ++i) { if (i % 2 == 0) { continue; } Block block = ColumnHelper::create_block({1 * i, 2 * i, 3 * i}); - local_state->_probe_blocks[i].emplace_back(std::move(block)); + local_state->_partitioned_blocks[i] = MutableBlock::create_unique(std::move(block)); } std::vector large_data(3 * 1024 * 1024); @@ -189,18 +670,18 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, spill_probe_blocks) { std::iota(small_data.begin(), small_data.end(), 3 * 1024 * 1024); Block small_block = ColumnHelper::create_block(small_data); - // add a large block to the last partition + // add a large block to the last partition (overwrite if needed) local_state->_partitioned_blocks[PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT - 1] = MutableBlock::create_unique(std::move(large_block)); - // add a small block to the first partition + // add a small block to the first partition (overwrite; first partition is even so was skipped above) local_state->_partitioned_blocks[0] = MutableBlock::create_unique(std::move(small_block)); - local_state->_shared_state->is_spilled = false; + local_state->_shared_state->_is_spilled = false; local_state->update_profile_from_inner(); - local_state->_shared_state->is_spilled = true; - auto st = local_state->spill_probe_blocks(_helper.runtime_state.get()); + local_state->_shared_state->_is_spilled = true; + auto st = local_state->spill_probe_blocks(_helper.runtime_state.get(), true); ASSERT_TRUE(st.ok()) << "spill probe blocks failed: " << st.to_string(); local_state->update_profile_from_inner(); @@ -208,17 +689,20 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, spill_probe_blocks) { std::cout << "profile: " << local_state->custom_profile()->pretty_print() << std::endl; for (int32_t i = 0; i != PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT; ++i) { - if (!local_state->_probe_spilling_streams[i]) { + if (!local_state->_probe_spilling_groups[i]) { continue; } - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream( - local_state->_probe_spilling_streams[i]); - local_state->_probe_spilling_streams[i].reset(); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file( + local_state->_probe_spilling_groups[i]); + local_state->_probe_spilling_groups[i].reset(); } - auto* write_rows_counter = local_state->custom_profile()->get_counter("SpillWriteRows"); + auto* write_rows_counter = local_state->custom_profile()->get_counter("SpillProbeRows"); + // Odd partitions 1,3,5 each have 3 rows; partition 7 (odd) was overwritten by large_block. + // Partition 0 has small_block (3*1024 rows), partition 7 has large_block (3*1024*1024 rows). ASSERT_EQ(write_rows_counter->value(), - (PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT / 2) * 3 + 3 * 1024 * 1024); + (PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT / 2 - 1) * 3 + 3 * 1024 * 1024 + + 3 * 1024); } TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDisk) { @@ -228,37 +712,39 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDisk) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // Create and register a spill stream for testing - const uint32_t test_partition = 0; - auto& spill_stream = local_state->_probe_spilling_streams[test_partition]; + // Create and register a spill file for testing + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_probe-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); ASSERT_TRUE(ExecEnv::GetInstance() - ->spill_stream_mgr() - ->register_spill_stream( - _helper.runtime_state.get(), spill_stream, - print_id(_helper.runtime_state->query_id()), "hash_probe", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()) + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) .ok()); - // Write some test data to spill stream + // Write some test data to spill file { Block block = ColumnHelper::create_block({1, 2, 3}); - ASSERT_TRUE(spill_stream->spill_block(_helper.runtime_state.get(), block, false).ok()); - ASSERT_TRUE(spill_stream->spill_eof().ok()); + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), block).ok()); + ASSERT_TRUE(writer->close().ok()); } - // Test recovery - bool has_data = false; + // Test recovery using JoinSpillPartitionInfo + JoinSpillPartitionInfo partition_info(nullptr, spill_file, 0); ASSERT_TRUE(local_state - ->recover_probe_blocks_from_disk(_helper.runtime_state.get(), - test_partition, has_data) + ->recover_probe_blocks_from_partition(_helper.runtime_state.get(), + partition_info) .ok()); - ASSERT_TRUE(has_data); std::cout << "profile: " << local_state->custom_profile()->pretty_print() << std::endl; - // Verify recovered data - auto& probe_blocks = local_state->_probe_blocks[test_partition]; + // Verify recovered data (now in _queue_probe_blocks) + auto& probe_blocks = local_state->_queue_probe_blocks; ASSERT_FALSE(probe_blocks.empty()); ASSERT_EQ(probe_blocks[0].rows(), 3); @@ -267,11 +753,11 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDisk) { local_state->custom_profile()->get_counter("SpillRecoveryProbeRows"); ASSERT_EQ(recovery_rows_counter->value(), 3); auto* recovery_blocks_counter = - local_state->custom_profile()->get_counter("SpillReadBlockCount"); + local_state->custom_profile()->get_counter("SpillRecoveryProbeBlocks"); ASSERT_EQ(recovery_blocks_counter->value(), 1); // Verify stream cleanup - ASSERT_EQ(local_state->_probe_spilling_streams[test_partition], nullptr); + ASSERT_EQ(partition_info.probe_file, nullptr); } TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDiskLargeData) { @@ -281,60 +767,67 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDiskLargeData auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // Create and register a spill stream for testing - const uint32_t test_partition = 0; - auto& spill_stream = local_state->_probe_spilling_streams[test_partition]; + // Create and register a spill file for testing + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_probe-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); ASSERT_TRUE(ExecEnv::GetInstance() - ->spill_stream_mgr() - ->register_spill_stream( - _helper.runtime_state.get(), spill_stream, - print_id(_helper.runtime_state->query_id()), "hash_probe", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()) + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) .ok()); - // Write some test data to spill stream + // Write some test data to spill file { // create block larger than 32MB(4 * (8 * 1024 * 1024 + 10)) std::vector large_data(8 * 1024 * 1024 + 10); std::iota(large_data.begin(), large_data.end(), 0); Block large_block = ColumnHelper::create_block(large_data); - ASSERT_TRUE( - spill_stream->spill_block(_helper.runtime_state.get(), large_block, false).ok()); + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), large_block).ok()); Block block = ColumnHelper::create_block({1, 2, 3}); - ASSERT_TRUE(spill_stream->spill_block(_helper.runtime_state.get(), block, false).ok()); - ASSERT_TRUE(spill_stream->spill_eof().ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), block).ok()); + ASSERT_TRUE(writer->close().ok()); } - // Test recovery - bool has_data = true; - while (has_data) { + // Test recovery using JoinSpillPartitionInfo + JoinSpillPartitionInfo partition_info(nullptr, spill_file, 0); + while (partition_info.probe_file) { ASSERT_TRUE(local_state - ->recover_probe_blocks_from_disk(_helper.runtime_state.get(), - test_partition, has_data) + ->recover_probe_blocks_from_partition(_helper.runtime_state.get(), + partition_info) .ok()); } std::cout << "profile: " << local_state->custom_profile()->pretty_print() << std::endl; - // Verify recovered data - auto& probe_blocks = local_state->_probe_blocks[test_partition]; + // Verify recovered data (now in _queue_probe_blocks) + auto& probe_blocks = local_state->_queue_probe_blocks; ASSERT_FALSE(probe_blocks.empty()); - ASSERT_EQ(probe_blocks[0].rows(), 8 * 1024 * 1024 + 10); - ASSERT_EQ(probe_blocks[1].rows(), 3); + + // Count total recovered rows + int64_t total_rows = 0; + for (const auto& block : probe_blocks) { + total_rows += block.rows(); + } + ASSERT_EQ(total_rows, 8 * 1024 * 1024 + 10 + 3); // Verify counters auto* recovery_rows_counter = local_state->custom_profile()->get_counter("SpillRecoveryProbeRows"); ASSERT_EQ(recovery_rows_counter->value(), 3 + 8 * 1024 * 1024 + 10); auto* recovery_blocks_counter = - local_state->custom_profile()->get_counter("SpillReadBlockCount"); + local_state->custom_profile()->get_counter("SpillRecoveryProbeBlocks"); ASSERT_EQ(recovery_blocks_counter->value(), 2); // Verify stream cleanup - ASSERT_EQ(local_state->_probe_spilling_streams[test_partition], nullptr); + ASSERT_EQ(partition_info.probe_file, nullptr); } TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDiskEmpty) { @@ -344,31 +837,35 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDiskEmpty) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // Test multiple cases - const uint32_t test_partition = 0; - - auto& spilled_stream = local_state->_probe_spilling_streams[test_partition]; + // Create an empty spill file + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_probe-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); ASSERT_TRUE(ExecEnv::GetInstance() - ->spill_stream_mgr() - ->register_spill_stream( - _helper.runtime_state.get(), spilled_stream, - print_id(_helper.runtime_state->query_id()), "hash_probe", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()) + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) .ok()); - ASSERT_TRUE(spilled_stream->spill_eof().ok()); + // Write nothing, just close the writer + { + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->close().ok()); + } - bool has_data = false; + JoinSpillPartitionInfo partition_info(nullptr, spill_file, 0); ASSERT_TRUE(local_state - ->recover_probe_blocks_from_disk(_helper.runtime_state.get(), - test_partition, has_data) + ->recover_probe_blocks_from_partition(_helper.runtime_state.get(), + partition_info) .ok()); - ASSERT_TRUE(has_data); - ASSERT_TRUE(local_state->_probe_blocks[test_partition].empty()) - << "probe blocks not empty: " << local_state->_probe_blocks[test_partition].size(); + ASSERT_TRUE(local_state->_queue_probe_blocks.empty()) + << "probe blocks not empty: " << local_state->_queue_probe_blocks.size(); - ASSERT_TRUE(spilled_stream == nullptr); + ASSERT_TRUE(partition_info.probe_file == nullptr); } TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDiskError) { @@ -378,37 +875,38 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeBlocksFromDiskError) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // Test multiple cases - const uint32_t test_partition = 0; - - auto& spilling_stream = local_state->_probe_spilling_streams[test_partition]; + // Create a spill file and write data + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_probe-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); ASSERT_TRUE(ExecEnv::GetInstance() - ->spill_stream_mgr() - ->register_spill_stream( - _helper.runtime_state.get(), spilling_stream, - print_id(_helper.runtime_state->query_id()), "hash_probe", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()) + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) .ok()); - // Write some test data to spill stream + // Write some test data to spill file { Block block = ColumnHelper::create_block({1, 2, 3}); - ASSERT_TRUE(spilling_stream->spill_block(_helper.runtime_state.get(), block, false).ok()); - ASSERT_TRUE(spilling_stream->spill_eof().ok()); + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), block).ok()); + ASSERT_TRUE(writer->close().ok()); } - SpillableDebugPointHelper dp_helper("fault_inject::spill_stream::read_next_block"); - bool has_data = false; - auto status = local_state->recover_probe_blocks_from_disk(_helper.runtime_state.get(), - test_partition, has_data); + SpillableDebugPointHelper dp_helper("fault_inject::spill_file::read_next_block"); + JoinSpillPartitionInfo partition_info(nullptr, spill_file, 0); + auto status = local_state->recover_probe_blocks_from_partition(_helper.runtime_state.get(), + partition_info); - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(spilling_stream); - spilling_stream.reset(); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(spill_file); + spill_file.reset(); ASSERT_FALSE(status.ok()); - ASSERT_TRUE(status.to_string().find("fault_inject spill_stream read_next_block") != - std::string::npos) + ASSERT_TRUE(status.to_string().find("spill_file read_next_block failed") != std::string::npos) << "unexpected error: " << status.to_string(); } @@ -421,32 +919,34 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildBlocksFromDisk) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // Create and register spill stream with test data - const uint32_t test_partition = 0; - auto& spilled_stream = local_state->_shared_state->spilled_streams[test_partition]; + // Create and register spill file with test data + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_build-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); ASSERT_TRUE(ExecEnv::GetInstance() - ->spill_stream_mgr() - ->register_spill_stream( - _helper.runtime_state.get(), spilled_stream, - print_id(_helper.runtime_state->query_id()), "hash_build", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()) + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) .ok()); // Write test data { Block block = ColumnHelper::create_block({1, 2, 3}); - ASSERT_TRUE(spilled_stream->spill_block(_helper.runtime_state.get(), block, false).ok()); - ASSERT_TRUE(spilled_stream->spill_eof().ok()); + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), block).ok()); + ASSERT_TRUE(writer->close().ok()); } - // Test recovery - bool has_data = false; + // Test recovery using JoinSpillPartitionInfo + JoinSpillPartitionInfo partition_info(spill_file, nullptr, 0); ASSERT_TRUE(local_state - ->recover_build_blocks_from_disk(_helper.runtime_state.get(), - test_partition, has_data) + ->recover_build_blocks_from_partition(_helper.runtime_state.get(), + partition_info) .ok()); - ASSERT_TRUE(has_data); // Verify recovered data ASSERT_TRUE(local_state->_recovered_build_block != nullptr); @@ -457,11 +957,11 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildBlocksFromDisk) { local_state->custom_profile()->get_counter("SpillRecoveryBuildRows"); ASSERT_EQ(recovery_rows_counter->value(), 3); auto* recovery_blocks_counter = - local_state->custom_profile()->get_counter("SpillReadBlockCount"); + local_state->custom_profile()->get_counter("SpillRecoveryBuildBlocks"); ASSERT_EQ(recovery_blocks_counter->value(), 1); // Verify stream cleanup - ASSERT_EQ(local_state->_shared_state->spilled_streams[test_partition], nullptr); + ASSERT_EQ(partition_info.build_file, nullptr); } TEST_F(PartitionedHashJoinProbeOperatorTest, need_more_input_data) { @@ -473,7 +973,7 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, need_more_input_data) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - local_state->_shared_state->is_spilled = true; + local_state->_shared_state->_is_spilled = true; local_state->_child_eos = false; ASSERT_EQ(probe_operator->need_more_input_data(_helper.runtime_state.get()), !local_state->_child_eos); @@ -482,7 +982,7 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, need_more_input_data) { ASSERT_EQ(probe_operator->need_more_input_data(_helper.runtime_state.get()), !local_state->_child_eos); - local_state->_shared_state->is_spilled = false; + local_state->_shared_state->_is_spilled = false; auto inner_operator = std::dynamic_pointer_cast( probe_operator->_inner_probe_operator); @@ -504,27 +1004,24 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, revocable_mem_size) { local_state->_child_eos = true; ASSERT_EQ(probe_operator->revocable_mem_size(_helper.runtime_state.get()), 0); + // revocable_mem_size requires _is_spilled = true to report non-zero memory + local_state->_shared_state->_is_spilled = true; local_state->_child_eos = false; auto block1 = ColumnHelper::create_block({1, 2, 3}); - local_state->_probe_blocks[0].emplace_back(block1); - ASSERT_EQ(probe_operator->revocable_mem_size(_helper.runtime_state.get()), - block1.allocated_bytes()); - auto block2 = ColumnHelper::create_block({1, 2, 3, 5, 6, 7}); - local_state->_partitioned_blocks[0] = MutableBlock::create_unique(std::move(block2)); - - // block2 is small, so it should not be counted - ASSERT_EQ(probe_operator->revocable_mem_size(_helper.runtime_state.get()), - block1.allocated_bytes()); - - // Create large input block (> 32k) - std::vector large_data(9 * 1024); + local_state->_partitioned_blocks[0] = MutableBlock::create_unique(std::move(block1)); + // Small blocks (< MIN_SPILL_WRITE_BATCH_MEM) are not counted as revocable + ASSERT_EQ(probe_operator->revocable_mem_size(_helper.runtime_state.get()), 0); + + // Create large input block (>= MIN_SPILL_WRITE_BATCH_MEM = 512KB) + std::vector large_data(256 * 1024); // 1MB of int32 std::iota(large_data.begin(), large_data.end(), 0); Block large_block = ColumnHelper::create_block(large_data); const auto large_size = large_block.allocated_bytes(); + + ASSERT_GE(large_size, SpillFile::MIN_SPILL_WRITE_BATCH_MEM); local_state->_partitioned_blocks[0] = MutableBlock::create_unique(std::move(large_block)); - ASSERT_EQ(probe_operator->revocable_mem_size(_helper.runtime_state.get()), - block1.allocated_bytes() + large_size); + ASSERT_EQ(probe_operator->revocable_mem_size(_helper.runtime_state.get()), large_size); local_state->_child_eos = true; ASSERT_EQ(probe_operator->revocable_mem_size(_helper.runtime_state.get()), 0); @@ -539,28 +1036,24 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, get_reserve_mem_size) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - local_state->_shared_state->is_spilled = true; + local_state->_shared_state->_is_spilled = true; local_state->_child_eos = false; - local_state->_need_to_setup_internal_operators = false; + // When child_eos is false, only minimum_operator_memory_required_bytes is reserved + local_state->_need_to_setup_queue_partition = false; ASSERT_EQ(probe_operator->get_reserve_mem_size(_helper.runtime_state.get()), - SpillStream::MAX_SPILL_WRITE_BATCH_MEM); + _helper.runtime_state->minimum_operator_memory_required_bytes()); - local_state->_need_to_setup_internal_operators = true; - ASSERT_GT(probe_operator->get_reserve_mem_size(_helper.runtime_state.get()), - SpillStream::MAX_SPILL_WRITE_BATCH_MEM); + // When not spilled, delegates to base class which returns minimum_operator_memory_required_bytes + local_state->_shared_state->_is_spilled = false; + ASSERT_GE(probe_operator->get_reserve_mem_size(_helper.runtime_state.get()), + _helper.runtime_state->minimum_operator_memory_required_bytes()); - const auto default_reserve_size = - _helper.runtime_state->minimum_operator_memory_required_bytes() + - probe_operator->get_child()->get_reserve_mem_size(_helper.runtime_state.get()); - local_state->_shared_state->is_spilled = false; - ASSERT_EQ(probe_operator->get_reserve_mem_size(_helper.runtime_state.get()), - default_reserve_size); - - local_state->_shared_state->is_spilled = true; + // When spilled and child_eos, no active partition: only baseline reservation + local_state->_shared_state->_is_spilled = true; local_state->_child_eos = true; ASSERT_EQ(probe_operator->get_reserve_mem_size(_helper.runtime_state.get()), - default_reserve_size); + _helper.runtime_state->minimum_operator_memory_required_bytes()); } TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildBlocksFromDiskEmpty) { @@ -571,28 +1064,33 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildBlocksFromDiskEmpty) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // Test empty stream - const uint32_t test_partition = 0; - auto& spilled_stream = local_state->_shared_state->spilled_streams[test_partition]; + // Create an empty spill file + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_build-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); ASSERT_TRUE(ExecEnv::GetInstance() - ->spill_stream_mgr() - ->register_spill_stream( - _helper.runtime_state.get(), spilled_stream, - print_id(_helper.runtime_state->query_id()), "hash_build", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()) + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) .ok()); - ASSERT_TRUE(spilled_stream->spill_eof().ok()); + // Write nothing, just close the writer + { + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->close().ok()); + } - bool has_data = false; + JoinSpillPartitionInfo partition_info(spill_file, nullptr, 0); ASSERT_TRUE(local_state - ->recover_build_blocks_from_disk(_helper.runtime_state.get(), - test_partition, has_data) + ->recover_build_blocks_from_partition(_helper.runtime_state.get(), + partition_info) .ok()); - ASSERT_TRUE(has_data); - ASSERT_EQ(spilled_stream, nullptr); + ASSERT_EQ(partition_info.build_file, nullptr); ASSERT_TRUE(local_state->_recovered_build_block == nullptr); } @@ -604,44 +1102,44 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildBlocksFromDiskLargeData auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // Test empty stream - const uint32_t test_partition = 0; - auto& spilled_stream = local_state->_shared_state->spilled_streams[test_partition]; + // Create spill file for large data test + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_build-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); ASSERT_TRUE(ExecEnv::GetInstance() - ->spill_stream_mgr() - ->register_spill_stream( - _helper.runtime_state.get(), spilled_stream, - print_id(_helper.runtime_state->query_id()), "hash_build", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()) + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) .ok()); - // Write some test data to spill stream + // Write some test data to spill file { // create block larger than 32MB(4 * (8 * 1024 * 1024 + 10)) std::vector large_data(8 * 1024 * 1024 + 10); std::iota(large_data.begin(), large_data.end(), 0); Block large_block = ColumnHelper::create_block(large_data); - ASSERT_TRUE( - spilled_stream->spill_block(_helper.runtime_state.get(), large_block, false).ok()); + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), large_block).ok()); Block block = ColumnHelper::create_block({1, 2, 3}); - ASSERT_TRUE(spilled_stream->spill_block(_helper.runtime_state.get(), block, false).ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), block).ok()); + ASSERT_TRUE(writer->close().ok()); } - ASSERT_TRUE(spilled_stream->spill_eof().ok()); - bool has_data = false; - do { + JoinSpillPartitionInfo partition_info(spill_file, nullptr, 0); + while (partition_info.build_file) { ASSERT_TRUE(local_state - ->recover_build_blocks_from_disk(_helper.runtime_state.get(), - test_partition, has_data) + ->recover_build_blocks_from_partition(_helper.runtime_state.get(), + partition_info) .ok()); ASSERT_TRUE(local_state->_recovered_build_block); - } while (has_data); - - ASSERT_EQ(spilled_stream, nullptr); + } // Verify recovered data ASSERT_EQ(local_state->_recovered_build_block->rows(), 8 * 1024 * 1024 + 10 + 3); @@ -651,12 +1149,11 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildBlocksFromDiskLargeData local_state->custom_profile()->get_counter("SpillRecoveryBuildRows"); ASSERT_EQ(recovery_rows_counter->value(), 8 * 1024 * 1024 + 10 + 3); auto* recovery_blocks_counter = - local_state->custom_profile()->get_counter("SpillReadBlockCount"); + local_state->custom_profile()->get_counter("SpillRecoveryBuildBlocks"); ASSERT_EQ(recovery_blocks_counter->value(), 2); } TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildBlocksFromDiskError) { - // Similar setup code as above... // Similar setup as above... auto [probe_operator, sink_operator] = _helper.create_operators(); @@ -664,28 +1161,34 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildBlocksFromDiskError) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // Test empty stream - const uint32_t test_partition = 0; - auto& spilled_stream = local_state->_shared_state->spilled_streams[test_partition]; + // Create an empty spill file + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_build-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); ASSERT_TRUE(ExecEnv::GetInstance() - ->spill_stream_mgr() - ->register_spill_stream( - _helper.runtime_state.get(), spilled_stream, - print_id(_helper.runtime_state->query_id()), "hash_build", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()) + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) .ok()); - ASSERT_TRUE(spilled_stream->spill_eof().ok()); + // Write nothing, just close + { + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->close().ok()); + } ASSERT_TRUE(local_state->_recovered_build_block == nullptr); // Test error handling with fault injection SpillableDebugPointHelper dp_helper( "fault_inject::partitioned_hash_join_probe::recover_build_blocks"); - bool has_data = false; - auto status = local_state->recover_build_blocks_from_disk(_helper.runtime_state.get(), - test_partition, has_data); + JoinSpillPartitionInfo partition_info(spill_file, nullptr, 0); + auto status = local_state->recover_build_blocks_from_partition(_helper.runtime_state.get(), + partition_info); ASSERT_FALSE(status.ok()); ASSERT_TRUE(status.to_string().find("fault_inject partitioned_hash_join_probe " @@ -710,7 +1213,7 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, GetBlockTestNonSpill) { std::dynamic_pointer_cast(probe_operator->get_child()); probe_side_source_operator->set_block(std::move(*input_block)); - local_state->_shared_state->is_spilled = false; + local_state->_shared_state->_is_spilled = false; // Test non empty input block path { @@ -828,19 +1331,14 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, PushWithEOS) { st = probe_operator->push(_helper.runtime_state.get(), &input_block, true); ASSERT_TRUE(st.ok()) << "Push failed: " << st.to_string(); - // Verify all data is moved to probe blocks due to EOS - int64_t total_probe_block_rows = 0; + // Verify all data is still in partitioned blocks (will be flushed at spill time) + int64_t total_partitioned_rows = 0; for (uint32_t i = 0; i < probe_operator->_partition_count; ++i) { - for (const auto& block : local_state->_probe_blocks[i]) { - total_probe_block_rows += block.rows(); + if (local_state->_partitioned_blocks[i]) { + total_partitioned_rows += local_state->_partitioned_blocks[i]->rows(); } } - ASSERT_EQ(total_probe_block_rows, 3); // All rows should be in probe blocks - - // Verify partitioned blocks are cleared - for (uint32_t i = 0; i < probe_operator->_partition_count; ++i) { - ASSERT_EQ(local_state->_partitioned_blocks[i], nullptr); - } + ASSERT_EQ(total_partitioned_rows, 3); // All rows should be in partitioned blocks } TEST_F(PartitionedHashJoinProbeOperatorTest, PushLargeBlock) { @@ -868,30 +1366,69 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, PushLargeBlock) { auto st = probe_operator->push(_helper.runtime_state.get(), &large_block, false); ASSERT_TRUE(st.ok()); - // Verify some partitions have blocks moved to probe_blocks due to size threshold - bool found_probe_blocks = false; + // Large blocks may be spilled during push (blocks >= MIN_SPILL_WRITE_BATCH_MEM are + // written to disk). Verify total rows = in-memory + spilled. size_t partitioned_rows_count = 0; for (uint32_t i = 0; i < probe_operator->_partition_count; ++i) { - if (!local_state->_probe_blocks[i].empty()) { - for (auto& block : local_state->_probe_blocks[i]) { - if (!block.empty()) { - partitioned_rows_count += block.rows(); - found_probe_blocks = true; - } - } - } if (local_state->_partitioned_blocks[i] && !local_state->_partitioned_blocks[i]->empty()) { partitioned_rows_count += local_state->_partitioned_blocks[i]->rows(); - found_probe_blocks = true; } } - ASSERT_EQ(partitioned_rows_count, large_block.rows()); - ASSERT_TRUE(found_probe_blocks); + auto* spill_probe_rows = local_state->custom_profile()->get_counter("SpillProbeRows"); + ASSERT_TRUE(spill_probe_rows != nullptr); + size_t total_rows = partitioned_rows_count + spill_probe_rows->value(); + ASSERT_EQ(total_rows, large_data.size()); +} + +TEST_F(PartitionedHashJoinProbeOperatorTest, PullInitializesSpillQueueFromLevel0Spills) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); - // Verify bytes counter - auto* probe_blocks_bytes = local_state->custom_profile()->get_counter("ProbeBloksBytesInMem"); - ASSERT_GT(probe_blocks_bytes->value(), 0); + auto build_file = create_probe_test_spill_file( + _helper.runtime_state.get(), local_state->operator_profile(), probe_operator->node_id(), + "hash_build_pull_init", {{1, 2, 3}}); + ASSERT_TRUE(build_file != nullptr); + + shared_state->_spilled_build_groups[0] = build_file; + local_state->_partitioned_blocks[0] = + MutableBlock::create_unique(ColumnHelper::create_block({4, 5, 6})); + local_state->_shared_state->_is_spilled = true; + local_state->_child_eos = true; + local_state->_need_to_setup_queue_partition = true; + + Block output_block; + bool eos = false; + auto st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "pull failed: " << st.to_string(); + + ASSERT_FALSE(eos); + ASSERT_TRUE(local_state->_spill_queue_initialized); + ASSERT_TRUE(local_state->_current_partition.is_valid()); + ASSERT_EQ(local_state->_current_partition.level, 0); + ASSERT_TRUE(local_state->_current_partition.probe_file != nullptr); + ASSERT_EQ(local_state->_spill_partition_queue.size(), + PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT - 1); + ASSERT_TRUE(local_state->_recovered_build_block != nullptr); + ASSERT_EQ(local_state->_recovered_build_block->rows(), 3); + ASSERT_EQ(shared_state->_spilled_build_groups[0], nullptr); + ASSERT_EQ(local_state->_probe_spilling_groups[0], nullptr); + ASSERT_EQ(local_state->_total_partition_spills->value(), + PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT); + ASSERT_EQ(local_state->_max_partition_level->value(), 0); + + auto* spill_probe_rows = local_state->custom_profile()->get_counter("SpillProbeRows"); + ASSERT_TRUE(spill_probe_rows != nullptr); + ASSERT_EQ(spill_probe_rows->value(), 3); + + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(build_file); + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file( + local_state->_current_partition.probe_file); + local_state->_current_partition = JoinSpillPartitionInfo {}; + local_state->_spill_partition_queue.clear(); } TEST_F(PartitionedHashJoinProbeOperatorTest, PullBasic) { @@ -901,17 +1438,20 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, PullBasic) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - local_state->_need_to_setup_internal_operators = true; - local_state->_partition_cursor = 0; + // Pre-initialize the spill queue with one empty partition (no build/probe files) + local_state->_spill_queue_initialized = true; + local_state->_need_to_setup_queue_partition = true; + local_state->_spill_partition_queue.emplace_back(JoinSpillPartitionInfo(nullptr, nullptr, 0)); Block test_block; bool eos = false; auto st = probe_operator->pull(_helper.runtime_state.get(), &test_block, &eos); ASSERT_TRUE(st.ok()) << "Pull failed: " << st.to_string(); - ASSERT_FALSE(eos) << "First pull should not be eos"; - ASSERT_EQ(1, local_state->_partition_cursor) << "Partition cursor should be 1"; + // After processing setup, _need_to_setup_queue_partition should be false + ASSERT_FALSE(local_state->_need_to_setup_queue_partition) + << "Partition setup should have been completed"; } TEST_F(PartitionedHashJoinProbeOperatorTest, PullMultiplePartitions) { @@ -920,27 +1460,29 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, PullMultiplePartitions) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); + // Pre-initialize the spill queue with multiple empty partitions + local_state->_spill_queue_initialized = true; for (uint32_t i = 0; i < PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT; i++) { - auto& probe_blocks = local_state->_probe_blocks[i]; - probe_blocks.emplace_back(ColumnHelper::create_block({1, 2, 3})); + local_state->_spill_partition_queue.emplace_back( + JoinSpillPartitionInfo(nullptr, nullptr, 0)); } Block output_block; bool eos = false; - for (uint32_t i = 0; i < PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT; i++) { - local_state->_partition_cursor = i; - local_state->_need_to_setup_internal_operators = true; - + // Process all partitions through the queue. + // Each partition requires two pulls: one for setup (build), one for probe. + // Only set _need_to_setup_queue_partition on the first call; after that the + // natural state machine transitions handle it. + local_state->_need_to_setup_queue_partition = true; + int processed = 0; + const int max_iterations = (int)PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT * 2 + 1; + while (!eos && processed < max_iterations) { auto st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); - ASSERT_TRUE(st.ok()) << "Pull failed for partition " << i; - - if (i == PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT - 1) { - ASSERT_TRUE(eos) << "Last partition should be eos"; - } else { - ASSERT_FALSE(eos) << "Non-last partition should not be eos"; - } + ASSERT_TRUE(st.ok()) << "Pull failed for iteration " << processed; + processed++; } + ASSERT_TRUE(eos) << "Should reach eos after all partitions are processed"; } TEST_F(PartitionedHashJoinProbeOperatorTest, PullWithDiskRecovery) { @@ -949,48 +1491,138 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, PullWithDiskRecovery) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - local_state->_shared_state->is_spilled = true; - - const uint32_t test_partition = 0; - auto& spilled_stream = local_state->_shared_state->spilled_streams[test_partition]; - auto& spilling_stream = local_state->_probe_spilling_streams[test_partition]; + local_state->_shared_state->_is_spilled = true; - local_state->_need_to_setup_internal_operators = true; + // Create build and probe spill files + SpillFileSPtr build_file; + auto build_path = fmt::format( + "{}/hash_build-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file(build_path, build_file) + .ok()); - auto st = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - _helper.runtime_state.get(), spilled_stream, - print_id(_helper.runtime_state->query_id()), "hash_probe_spilled", - probe_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), local_state->operator_profile()); + SpillFileSPtr probe_file; + auto probe_path = fmt::format( + "{}/hash_probe-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file(probe_path, probe_file) + .ok()); - ASSERT_TRUE(st) << "Register spill stream failed: " << st.to_string(); - st = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - _helper.runtime_state.get(), spilling_stream, - print_id(_helper.runtime_state->query_id()), "hash_probe", probe_operator->node_id(), - std::numeric_limits::max(), std::numeric_limits::max(), - local_state->operator_profile()); + // Write test data to build file + { + Block spill_block = ColumnHelper::create_block({1, 2, 3}); + SpillFileWriterSPtr writer; + ASSERT_TRUE(build_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), spill_block).ok()); + ASSERT_TRUE(writer->close().ok()); + } - ASSERT_TRUE(st) << "Register spill stream failed: " << st.to_string(); + // Write test data to probe file + { + Block spill_block = ColumnHelper::create_block({4, 5, 6}); + SpillFileWriterSPtr writer; + ASSERT_TRUE(probe_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), spill_block).ok()); + ASSERT_TRUE(writer->close().ok()); + } - Block spill_block = ColumnHelper::create_block({1, 2, 3}); - st = spilled_stream->spill_block(_helper.runtime_state.get(), spill_block, true); - ASSERT_TRUE(st) << "Spill block failed: " << st.to_string(); - st = spilling_stream->spill_block(_helper.runtime_state.get(), spill_block, false); - ASSERT_TRUE(st) << "Spill block failed: " << st.to_string(); + // Pre-initialize queue with one partition + local_state->_spill_queue_initialized = true; + local_state->_need_to_setup_queue_partition = true; + local_state->_spill_partition_queue.emplace_back( + JoinSpillPartitionInfo(build_file, probe_file, 0)); Block output_block; bool eos = false; - st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); + // First pull should recover build data + auto st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); ASSERT_TRUE(st.ok()) << "Pull failed: " << st.to_string(); + ASSERT_FALSE(eos) << "Should not be eos during disk recovery"; - st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); + ASSERT_GT(local_state->_recovery_build_rows->value(), 0) + << "Should have recovered some build rows from disk"; +} - ASSERT_TRUE(st.ok()) << "Pull failed: " << st.to_string(); - ASSERT_FALSE(eos) << "Should not be eos during disk recovery"; +TEST_F(PartitionedHashJoinProbeOperatorTest, PullRecoversProbeBlocksFromPartition) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + auto probe_file = create_probe_test_spill_file( + _helper.runtime_state.get(), local_state->operator_profile(), probe_operator->node_id(), + "hash_probe_pull_recover", {{1, 2, 3}, {4, 5}}); + ASSERT_TRUE(probe_file != nullptr); + + local_state->_shared_state->_is_spilled = true; + local_state->_spill_queue_initialized = true; + local_state->_need_to_setup_queue_partition = false; + local_state->_current_partition = JoinSpillPartitionInfo(nullptr, probe_file, 1); + local_state->_current_partition.build_finished = true; + + Block output_block; + bool eos = false; + auto st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "pull failed: " << st.to_string(); + + ASSERT_FALSE(eos); + ASSERT_EQ(local_state->_queue_probe_blocks.size(), 2); + ASSERT_EQ(local_state->_recovery_probe_rows->value(), 5); + ASSERT_EQ(local_state->_recovery_probe_blocks->value(), 2); + ASSERT_EQ(local_state->_current_partition.probe_file, nullptr); + ASSERT_EQ(local_state->_current_probe_reader, nullptr); + + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(probe_file); + local_state->_queue_probe_blocks.clear(); + local_state->_current_partition = JoinSpillPartitionInfo {}; +} + +TEST_F(PartitionedHashJoinProbeOperatorTest, PullFinishesPartitionAfterRecoveredProbeBlocks) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + auto probe_file = create_probe_test_spill_file( + _helper.runtime_state.get(), local_state->operator_profile(), probe_operator->node_id(), + "hash_probe_pull_finish", {{6, 7, 8}}); + ASSERT_TRUE(probe_file != nullptr); - ASSERT_GT(local_state->_recovery_probe_rows->value(), 0) - << "Should have recovered some rows from disk"; + local_state->_shared_state->_is_spilled = true; + local_state->_spill_queue_initialized = true; + local_state->_need_to_setup_queue_partition = false; + local_state->_current_partition = JoinSpillPartitionInfo(nullptr, probe_file, 1); + local_state->_current_partition.build_finished = true; + + Block output_block; + bool eos = false; + auto st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "first pull failed: " << st.to_string(); + ASSERT_FALSE(eos); + ASSERT_EQ(local_state->_queue_probe_blocks.size(), 1); + + st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "second pull failed: " << st.to_string(); + ASSERT_TRUE(eos); + ASSERT_FALSE(local_state->_current_partition.is_valid()); + ASSERT_TRUE(local_state->_need_to_setup_queue_partition); + ASSERT_TRUE(local_state->_queue_probe_blocks.empty()); + ASSERT_TRUE(local_state->_spill_partition_queue.empty()); + + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(probe_file); } TEST_F(PartitionedHashJoinProbeOperatorTest, PullWithEmptyPartition) { @@ -999,20 +1631,22 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, PullWithEmptyPartition) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - // 设置空分区 - local_state->_partition_cursor = 0; - local_state->_need_to_setup_internal_operators = true; + // Set up queue with an empty partition followed by another + local_state->_spill_queue_initialized = true; + local_state->_need_to_setup_queue_partition = true; + local_state->_spill_partition_queue.emplace_back(JoinSpillPartitionInfo(nullptr, nullptr, 0)); + local_state->_spill_partition_queue.emplace_back(JoinSpillPartitionInfo(nullptr, nullptr, 0)); Block output_block; bool eos = false; auto st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); ASSERT_TRUE(st.ok()) << "Pull failed for empty partition"; - ASSERT_FALSE(eos) << "Should not be eos for first empty partition"; + ASSERT_FALSE(eos) << "Should not be eos since more partitions remain in queue"; - // 验证分区游标已更新 - ASSERT_EQ(1, local_state->_partition_cursor) - << "Partition cursor should move to next after empty partition"; + // The first partition should have been popped from the queue + ASSERT_EQ(local_state->_spill_partition_queue.size(), 1u) + << "One partition should remain in queue after processing empty one"; } TEST_F(PartitionedHashJoinProbeOperatorTest, Other) { @@ -1022,11 +1656,174 @@ TEST_F(PartitionedHashJoinProbeOperatorTest, Other) { auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), probe_operator.get(), shared_state); - local_state->_shared_state->is_spilled = true; - ASSERT_FALSE(probe_operator->_should_revoke_memory(_helper.runtime_state.get())); + local_state->_shared_state->_is_spilled = true; - auto st = probe_operator->_revoke_memory(_helper.runtime_state.get()); + auto st = probe_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "Revoke memory failed: " << st.to_string(); } -} // namespace doris \ No newline at end of file +// Test RecoverProbeBlocks with multiple blocks in one spill file. +TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverProbeMultipleBlocks) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + // Create spill file with 3 blocks + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_probe-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) + .ok()); + + { + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + + for (int batch = 0; batch < 3; ++batch) { + Block block = ColumnHelper::create_block( + {batch * 10 + 1, batch * 10 + 2, batch * 10 + 3}); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), block).ok()); + } + ASSERT_TRUE(writer->close().ok()); + } + + // Recover all blocks + JoinSpillPartitionInfo partition_info(nullptr, spill_file, 0); + while (partition_info.probe_file) { + ASSERT_TRUE(local_state + ->recover_probe_blocks_from_partition(_helper.runtime_state.get(), + partition_info) + .ok()); + } + + // Verify all data recovered + int64_t total_rows = 0; + for (const auto& block : local_state->_queue_probe_blocks) { + total_rows += block.rows(); + } + ASSERT_EQ(total_rows, 9); + + auto* recovery_rows = local_state->custom_profile()->get_counter("SpillRecoveryProbeRows"); + ASSERT_EQ(recovery_rows->value(), 9); + auto* recovery_blocks = local_state->custom_profile()->get_counter("SpillRecoveryProbeBlocks"); + ASSERT_EQ(recovery_blocks->value(), 3); +} + +// Test RecoverBuildBlocks with multiple blocks in one spill file. +TEST_F(PartitionedHashJoinProbeOperatorTest, RecoverBuildMultipleBlocks) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + // Create spill file with 3 blocks + SpillFileSPtr spill_file; + auto relative_path = fmt::format( + "{}/hash_build-{}-{}", print_id(_helper.runtime_state->query_id()), + probe_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file(relative_path, spill_file) + .ok()); + + { + SpillFileWriterSPtr writer; + ASSERT_TRUE(spill_file + ->create_writer(_helper.runtime_state.get(), + local_state->operator_profile(), writer) + .ok()); + + for (int batch = 0; batch < 3; ++batch) { + Block block = + ColumnHelper::create_block({batch * 100 + 1, batch * 100 + 2}); + ASSERT_TRUE(writer->write_block(_helper.runtime_state.get(), block).ok()); + } + ASSERT_TRUE(writer->close().ok()); + } + + // Recover all blocks + JoinSpillPartitionInfo partition_info(spill_file, nullptr, 0); + while (partition_info.build_file) { + ASSERT_TRUE(local_state + ->recover_build_blocks_from_partition(_helper.runtime_state.get(), + partition_info) + .ok()); + } + + // Verify all data recovered + ASSERT_TRUE(local_state->_recovered_build_block != nullptr); + ASSERT_EQ(local_state->_recovered_build_block->rows(), 6); + + auto* recovery_rows = local_state->custom_profile()->get_counter("SpillRecoveryBuildRows"); + ASSERT_EQ(recovery_rows->value(), 6); + auto* recovery_blocks = local_state->custom_profile()->get_counter("SpillRecoveryBuildBlocks"); + ASSERT_EQ(recovery_blocks->value(), 3); +} + +// Test queue with all empty partitions reaches EOS. +TEST_F(PartitionedHashJoinProbeOperatorTest, PullAllEmptyPartitions) { + auto [probe_operator, sink_operator] = _helper.create_operators(); + std::shared_ptr shared_state; + auto local_state = _helper.create_probe_local_state(_helper.runtime_state.get(), + probe_operator.get(), shared_state); + + // Initialize queue with 3 empty partitions + local_state->_spill_queue_initialized = true; + for (int i = 0; i < 3; ++i) { + local_state->_spill_partition_queue.emplace_back( + JoinSpillPartitionInfo(nullptr, nullptr, 0)); + } + + Block output_block; + bool eos = false; + int iterations = 0; + + // Each partition requires two pulls: one for setup, one for probe. + local_state->_need_to_setup_queue_partition = true; + while (!eos && iterations < 10) { + auto st = probe_operator->pull(_helper.runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "Pull failed at iteration " << iterations; + iterations++; + } + + ASSERT_TRUE(eos) << "Should reach EOS after processing all empty partitions"; + ASSERT_TRUE(local_state->_spill_partition_queue.empty()) + << "Queue should be empty after processing all partitions"; +} + +// Test JoinSpillPartitionInfo validity. +TEST_F(PartitionedHashJoinProbeOperatorTest, JoinSpillPartitionInfoValidation) { + // Default constructed should be invalid + JoinSpillPartitionInfo default_info; + ASSERT_FALSE(default_info.is_valid()); + + // Constructed with files should be valid + SpillFileSPtr build_file; + auto relative_path = + fmt::format("{}/hash_build-test-{}", print_id(_helper.runtime_state->query_id()), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + ASSERT_TRUE(ExecEnv::GetInstance() + ->spill_file_mgr() + ->create_spill_file(relative_path, build_file) + .ok()); + + JoinSpillPartitionInfo valid_info(build_file, nullptr, 1); + ASSERT_TRUE(valid_info.is_valid()); + ASSERT_EQ(valid_info.level, 1); + ASSERT_TRUE(valid_info.build_file != nullptr); + ASSERT_TRUE(valid_info.probe_file == nullptr); + + // Null files + initialized should still be valid + JoinSpillPartitionInfo null_files_info(nullptr, nullptr, 0); + ASSERT_TRUE(null_files_info.is_valid()); +} +} // namespace doris diff --git a/be/test/exec/operator/partitioned_hash_join_sink_operator_test.cpp b/be/test/exec/operator/partitioned_hash_join_sink_operator_test.cpp index 9c2d092b1fb560..92479589603507 100644 --- a/be/test/exec/operator/partitioned_hash_join_sink_operator_test.cpp +++ b/be/test/exec/operator/partitioned_hash_join_sink_operator_test.cpp @@ -36,7 +36,7 @@ #include "exec/operator/partitioned_hash_join_probe_operator.h" #include "exec/operator/partitioned_hash_join_test_helper.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "exprs/vexpr_context.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" @@ -69,9 +69,7 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, Init) { tnode.row_tuples.push_back(desc_tbl.get_tuple_descs().front()->id()); tnode.nullable_tuples.push_back(false); - PartitionedHashJoinSinkOperatorX operator_x( - _helper.obj_pool.get(), 0, 0, tnode, desc_tbl, - PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT); + PartitionedHashJoinSinkOperatorX operator_x(_helper.obj_pool.get(), 0, 0, tnode, desc_tbl); auto child = std::make_shared(); child->_row_descriptor = RowDescriptor(_helper.runtime_state->desc_tbl(), {1}, {false}); @@ -133,18 +131,17 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, InitLocalState) { local_state->update_memory_usage(); - shared_state->is_spilled = false; + shared_state->_is_spilled = false; auto reserve_size = local_state->get_reserve_mem_size(_helper.runtime_state.get(), false); - shared_state->is_spilled = true; + shared_state->_is_spilled = true; reserve_size = local_state->get_reserve_mem_size(_helper.runtime_state.get(), false); - ASSERT_EQ(reserve_size, - sink_operator->_partition_count * SpillStream::MIN_SPILL_WRITE_BATCH_MEM); + ASSERT_EQ(reserve_size, sink_operator->_partition_count * SpillFile::MIN_SPILL_WRITE_BATCH_MEM); auto* finish_dep = local_state->finishdependency(); ASSERT_TRUE(finish_dep != nullptr); - shared_state->is_spilled = false; + shared_state->_is_spilled = false; st = local_state->close(_helper.runtime_state.get(), Status::OK()); ASSERT_TRUE(st) << "close failed: " << st.to_string(); @@ -164,9 +161,8 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, InitBuildExprs) { } DescriptorTbl desc_tbl; - PartitionedHashJoinSinkOperatorX operator_x( - _helper.obj_pool.get(), 0, 0, tnode, desc_tbl, - PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT); + PartitionedHashJoinSinkOperatorX operator_x(_helper.obj_pool.get(), 0, 0, tnode, desc_tbl); + operator_x._partition_count = PartitionedHashJoinTestHelper::TEST_PARTITION_COUNT; ASSERT_TRUE(operator_x.init(tnode, _helper.runtime_state.get())); ASSERT_EQ(operator_x._build_exprs.size(), 4); // 1个初始 + 3个新增 @@ -203,7 +199,7 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, Sink) { auto read_dependency = Dependency::create_shared(sink_operator->operator_id(), sink_operator->node_id(), "HashJoinBuildReadDependency", false); - sink_local_state->_shared_state->is_spilled = false; + sink_local_state->_shared_state->_is_spilled = false; shared_state->source_deps.emplace_back(read_dependency); @@ -221,6 +217,7 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, SinkEosAndSpill) { auto [_, sink_operator] = _helper.create_operators(); auto shared_state = std::make_shared(); + shared_state->init(sink_operator->_partition_count); LocalSinkStateInfo sink_info {.task_idx = 0, .parent_profile = _helper.operator_profile.get(), @@ -255,7 +252,7 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, SinkEosAndSpill) { Block block; // sink empty block - sink_local_state->_shared_state->is_spilled = false; + sink_local_state->_shared_state->_is_spilled = false; ASSERT_EQ(read_dependency->_ready.load(), false); st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "Sink failed: " << st.to_string(); @@ -266,7 +263,7 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, SinkEosAndSpill) { st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "Sink failed: " << st.to_string(); - sink_local_state->_shared_state->is_spilled = true; + sink_local_state->_shared_state->_is_spilled = true; ASSERT_EQ(read_dependency->_ready.load(), false); st = sink_operator->sink(_helper.runtime_state.get(), &block, false); ASSERT_TRUE(st.ok()) << "Sink failed: " << st.to_string(); @@ -289,11 +286,11 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, RevokeMemoryEmpty) { auto* sink_state = _helper.create_sink_local_state(_helper.runtime_state.get(), sink_operator.get(), shared_state); - shared_state->is_spilled = false; + shared_state->_is_spilled = false; // Expect revoke memory to trigger spilling - auto status = sink_state->revoke_memory(_helper.runtime_state.get(), nullptr); + auto status = sink_state->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(status.ok()) << "Revoke memory failed: " << status.to_string(); - ASSERT_TRUE(sink_state->_shared_state->is_spilled); + ASSERT_TRUE(sink_state->_shared_state->_is_spilled); } TEST_F(PartitionedHashJoinSinkOperatorTest, RevokeMemory) { @@ -318,24 +315,24 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, RevokeMemory) { status = partitioner->prepare(_helper.runtime_state.get(), sink_operator->_child->row_desc()); ASSERT_TRUE(status.ok()) << "Prepare partitioner failed: " << status.to_string(); sink_state->_partitioner = std::move(partitioner); - sink_state->_shared_state->is_spilled = false; + sink_state->_shared_state->_is_spilled = false; DCHECK_GE(sink_operator->_child->row_desc().get_column_id(1), 0); for (uint32_t i = 0; i != sink_operator->_partition_count; ++i) { - auto& spilling_stream = sink_state->_shared_state->spilled_streams[i]; - auto st = (ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - _helper.runtime_state.get(), spilling_stream, - print_id(_helper.runtime_state->query_id()), fmt::format("hash_build_sink_{}", i), - sink_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), sink_state->operator_profile())); + auto& spilling_file = sink_state->_shared_state->_spilled_build_groups[i]; + auto relative_path = fmt::format( + "{}/hash_build_sink_{}-{}-{}", print_id(_helper.runtime_state->query_id()), i, + sink_operator->node_id(), ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, + spilling_file); ASSERT_TRUE(st.ok()) << "Register spill stream failed: " << st.to_string(); } auto& inner_sink = sink_operator->_inner_sink_operator; auto inner_sink_local_state = std::make_unique( - inner_sink.get(), sink_state->_shared_state->inner_runtime_state.get()); + inner_sink.get(), sink_state->_shared_state->_inner_runtime_state.get()); inner_sink_local_state->_hash_table_memory_usage = sink_state->custom_profile()->add_counter("HashTableMemoryUsage", TUnit::BYTES); inner_sink_local_state->_build_arena_memory_usage = @@ -345,7 +342,7 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, RevokeMemory) { ASSERT_EQ(block.rows(), 3); inner_sink_local_state->_build_side_mutable_block = std::move(block); - sink_state->_shared_state->inner_runtime_state->emplace_sink_local_state( + sink_state->_shared_state->_inner_runtime_state->emplace_sink_local_state( 0, std::move(inner_sink_local_state)); sink_state->_finish_dependency = @@ -353,9 +350,9 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, RevokeMemory) { "HashJoinBuildFinishDependency", true); // Expect revoke memory to trigger spilling - status = sink_state->revoke_memory(_helper.runtime_state.get(), nullptr); + status = sink_state->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(status.ok()) << "Revoke memory failed: " << status.to_string(); - ASSERT_TRUE(sink_state->_shared_state->is_spilled); + ASSERT_TRUE(sink_state->_shared_state->_is_spilled); std::cout << "profile: " << sink_state->operator_profile()->pretty_print() << std::endl; @@ -367,12 +364,30 @@ TEST_F(PartitionedHashJoinSinkOperatorTest, RevokeMemory) { std::iota(large_data.begin(), large_data.end(), 0); Block large_block = ColumnHelper::create_block(large_data); - sink_state->_shared_state->partitioned_build_blocks[0] = + sink_state->_shared_state->_partitioned_build_blocks[0] = MutableBlock::create_unique(std::move(large_block)); - status = sink_state->revoke_memory(_helper.runtime_state.get(), nullptr); + status = sink_state->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(status.ok()) << "Revoke memory failed: " << status.to_string(); ASSERT_EQ(written_rows + 3 * 1024 * 1024, written_rows_counter->value()); } -} // namespace doris \ No newline at end of file +// Test that revocable_mem_size returns 0 immediately after revoke. +TEST_F(PartitionedHashJoinSinkOperatorTest, RevocableMemSizeAfterRevoke) { + auto [_, sink_operator] = _helper.create_operators(); + + std::shared_ptr shared_state; + auto* sink_state = _helper.create_sink_local_state(_helper.runtime_state.get(), + sink_operator.get(), shared_state); + + shared_state->_is_spilled = false; + + auto status = sink_state->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(status.ok()); + ASSERT_TRUE(sink_state->_shared_state->_is_spilled); + + // After revoke with no data, revocable_mem_size should be 0 + ASSERT_EQ(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); +} +} // namespace doris diff --git a/be/test/exec/operator/partitioned_hash_join_test_helper.cpp b/be/test/exec/operator/partitioned_hash_join_test_helper.cpp index 9891bc7635fab6..34ccedb946ea96 100644 --- a/be/test/exec/operator/partitioned_hash_join_test_helper.cpp +++ b/be/test/exec/operator/partitioned_hash_join_test_helper.cpp @@ -114,10 +114,12 @@ PartitionedHashJoinTestHelper::create_operators() { EXPECT_EQ(desc_tbl.get_tuple_descs().size(), 2); - auto probe_operator = std::make_shared( - obj_pool.get(), tnode, 0, desc_tbl, TEST_PARTITION_COUNT); - auto sink_operator = std::make_shared( - obj_pool.get(), 0, 0, tnode, desc_tbl, TEST_PARTITION_COUNT); + auto probe_operator = + std::make_shared(obj_pool.get(), tnode, 0, desc_tbl); + probe_operator->_partition_count = TEST_PARTITION_COUNT; + auto sink_operator = std::make_shared(obj_pool.get(), 0, 0, + tnode, desc_tbl); + sink_operator->_partition_count = TEST_PARTITION_COUNT; auto child_operator = std::make_shared(); auto probe_side_source_operator = std::make_shared(); @@ -171,24 +173,31 @@ PartitionedHashJoinProbeLocalState* PartitionedHashJoinTestHelper::create_probe_ shared_state = std::make_shared(); local_state->init_counters(); local_state->_shared_state = shared_state.get(); - shared_state->is_spilled = true; + shared_state->_is_spilled = true; ADD_TIMER(local_state->common_profile(), "ExecTime"); local_state->common_profile()->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 0); local_state->init_spill_read_counters(); local_state->init_spill_write_counters(); local_state->init_counters(); - local_state->_copy_shared_spill_profile = false; local_state->_internal_runtime_profile = std::make_unique("inner_test"); local_state->_partitioned_blocks.resize(probe_operator->_partition_count); - local_state->_probe_spilling_streams.resize(probe_operator->_partition_count); + local_state->_probe_spilling_groups.resize(probe_operator->_partition_count); + local_state->_probe_writers.resize(probe_operator->_partition_count); - shared_state->spilled_streams.resize(probe_operator->_partition_count); - shared_state->partitioned_build_blocks.resize(probe_operator->_partition_count); + shared_state->_spilled_build_groups.resize(probe_operator->_partition_count); + shared_state->_partitioned_build_blocks.resize(probe_operator->_partition_count); - shared_state->inner_runtime_state = std::make_unique(); - shared_state->inner_shared_state = std::make_shared(); + shared_state->_inner_runtime_state = std::make_unique(); + shared_state->_inner_shared_state = std::make_shared(); + + local_state->_dependency = shared_state->create_source_dependency( + probe_operator->operator_id(), probe_operator->node_id(), + "PartitionedHashJoinProbeTestDep"); + local_state->_wait_for_dependency_timer = + ADD_TIMER_WITH_LEVEL(local_state->common_profile(), + "WaitForDependency[PartitionedHashJoinProbeTestDep]Time", 1); state->emplace_local_state(probe_operator->operator_id(), std::move(local_state_uptr)); return local_state; @@ -203,7 +212,7 @@ PartitionedHashJoinSinkLocalState* PartitionedHashJoinTestHelper::create_sink_lo shared_state = std::make_shared(); local_state->init_spill_counters(); local_state->_shared_state = shared_state.get(); - shared_state->is_spilled = true; + shared_state->_is_spilled = true; ADD_TIMER(local_state->common_profile(), "ExecTime"); local_state->common_profile()->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 0); @@ -213,12 +222,15 @@ PartitionedHashJoinSinkLocalState* PartitionedHashJoinTestHelper::create_sink_lo sink_operator->dests_id().front(), sink_operator->operator_id(), "PartitionedHashJoinTestDep"); - shared_state->spilled_streams.resize(sink_operator->_partition_count); - shared_state->partitioned_build_blocks.resize(sink_operator->_partition_count); + shared_state->_spilled_build_groups.resize(sink_operator->_partition_count); + shared_state->_partitioned_build_blocks.resize(sink_operator->_partition_count); + + // Initialize vectors that are normally set up in init() + local_state->_build_writers.resize(sink_operator->_partition_count); + local_state->_rows_in_partitions.assign(sink_operator->_partition_count, 0); - shared_state->inner_runtime_state = std::make_unique(); - shared_state->inner_shared_state = std::make_shared(); - shared_state->setup_shared_profile(local_state->custom_profile()); + shared_state->_inner_runtime_state = std::make_unique(); + shared_state->_inner_shared_state = std::make_shared(); state->emplace_sink_local_state(sink_operator->operator_id(), std::move(local_state_uptr)); return local_state; diff --git a/be/test/exec/operator/partitioned_hash_join_test_helper.h b/be/test/exec/operator/partitioned_hash_join_test_helper.h index d954cf9bf94ffb..9dcbb7335f560c 100644 --- a/be/test/exec/operator/partitioned_hash_join_test_helper.h +++ b/be/test/exec/operator/partitioned_hash_join_test_helper.h @@ -33,7 +33,7 @@ #include "exec/operator/partitioned_hash_join_sink_operator.h" #include "exec/operator/spillable_operator_test_helper.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" @@ -43,15 +43,15 @@ namespace doris { class MockPartitionedHashJoinSharedState : public PartitionedHashJoinSharedState { public: MockPartitionedHashJoinSharedState() { - is_spilled = false; - inner_runtime_state = nullptr; - spilled_streams.clear(); - partitioned_build_blocks.clear(); + _is_spilled = false; + _inner_runtime_state = nullptr; + _spilled_build_groups.clear(); + _partitioned_build_blocks.clear(); } void init(size_t partition_count) { - spilled_streams.resize(partition_count); - partitioned_build_blocks.resize(partition_count); + _spilled_build_groups.resize(partition_count); + _partitioned_build_blocks.resize(partition_count); } }; diff --git a/be/test/exec/operator/spill_sort_sink_operator_test.cpp b/be/test/exec/operator/spill_sort_sink_operator_test.cpp index ef0558bbdded46..db47d9565a3fa9 100644 --- a/be/test/exec/operator/spill_sort_sink_operator_test.cpp +++ b/be/test/exec/operator/spill_sort_sink_operator_test.cpp @@ -230,7 +230,7 @@ TEST_F(SpillSortSinkOperatorTest, SinkWithSpill) { st = sink_operator->sink(_helper.runtime_state.get(), &input_block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); auto input_block2 = ColumnHelper::create_block({1, 2, 3, 4, 5, 5, 4, 3, 2, 1}); @@ -241,8 +241,6 @@ TEST_F(SpillSortSinkOperatorTest, SinkWithSpill) { st = sink_operator->sink(_helper.runtime_state.get(), &input_block2, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - ASSERT_GT(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); - // Because there are some rows in the sorter, // the sink operator will revoke memory when sinking eos with empty block. Block empty_block; @@ -297,7 +295,7 @@ TEST_F(SpillSortSinkOperatorTest, SinkWithSpill2) { st = sink_operator->sink(_helper.runtime_state.get(), &input_block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); ASSERT_EQ(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); @@ -353,11 +351,136 @@ TEST_F(SpillSortSinkOperatorTest, SinkWithSpillError) { st = sink_operator->sink(_helper.runtime_state.get(), &input_block, false); ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); - SpillableDebugPointHelper dp_helper("fault_inject::spill_stream::spill_block"); + SpillableDebugPointHelper dp_helper("fault_inject::spill_file::spill_block"); - st = sink_operator->revoke_memory(_helper.runtime_state.get(), nullptr); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); ASSERT_FALSE(st.ok()) << "spilll status should be failed"; } -} // namespace doris \ No newline at end of file +// Test multiple consecutive revoke_memory calls to verify repeated spilling works. +TEST_F(SpillSortSinkOperatorTest, SinkMultipleRevokes) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + auto tnode = _helper.create_test_plan_node(); + auto st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + auto shared_state = + std::dynamic_pointer_cast(sink_operator->create_shared_state()); + ASSERT_TRUE(shared_state != nullptr); + + shared_state->create_source_dependency(sink_operator->operator_id(), sink_operator->node_id(), + "SpillSortSinkOperatorTest"); + + LocalSinkStateInfo info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = {}}; + + st = sink_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + auto* sink_local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(sink_local_state != nullptr); + + st = sink_local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + // Perform 3 rounds of sink → revoke + for (int round = 0; round < 3; ++round) { + auto input_block = ColumnHelper::create_block( + {1 + round, 2 + round, 3 + round, 4 + round, 5 + round}); + input_block.insert(ColumnHelper::create_column_with_name({10, 9, 8, 7, 6})); + + st = sink_operator->sink(_helper.runtime_state.get(), &input_block, false); + ASSERT_TRUE(st.ok()) << "sink failed on round " << round << ": " << st.to_string(); + + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke_memory failed on round " << round << ": " << st.to_string(); + + ASSERT_EQ(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0) + << "revocable_mem_size should be 0 after revoke on round " << round; + } + + // After 3 rounds of spilling, should have 3 spill files + ASSERT_EQ(shared_state->sorted_spill_groups.size(), 3) + << "Should have 3 spill groups after 3 revokes"; + + ASSERT_TRUE(shared_state->is_spilled) << "is_spilled should be true after revoke"; +} + +// Test sinking large data (>1M rows), then verify spill counters. +TEST_F(SpillSortSinkOperatorTest, SinkLargeDataWithSpill) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + auto tnode = _helper.create_test_plan_node(); + auto st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + auto shared_state = + std::dynamic_pointer_cast(sink_operator->create_shared_state()); + ASSERT_TRUE(shared_state != nullptr); + + shared_state->create_source_dependency(sink_operator->operator_id(), sink_operator->node_id(), + "SpillSortSinkOperatorTest"); + + LocalSinkStateInfo info {.task_idx = 0, + .parent_profile = _helper.operator_profile.get(), + .sender_id = 0, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .tsink = {}}; + + st = sink_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + auto* sink_local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(sink_local_state != nullptr); + + st = sink_local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + // Create large data + const size_t count = 100000; + std::vector data(count); + std::iota(data.begin(), data.end(), 0); + std::vector data2(count); + std::iota(data2.begin(), data2.end(), 0); + + auto input_block = ColumnHelper::create_block(data); + input_block.insert(ColumnHelper::create_column_with_name(data2)); + + st = sink_operator->sink(_helper.runtime_state.get(), &input_block, false); + ASSERT_TRUE(st.ok()) << "sink failed: " << st.to_string(); + + ASSERT_GT(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); + + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); + + ASSERT_EQ(sink_operator->revocable_mem_size(_helper.runtime_state.get()), 0); + ASSERT_TRUE(shared_state->is_spilled); + + auto* spill_write_rows = sink_local_state->custom_profile()->get_counter("SpillWriteRows"); + ASSERT_TRUE(spill_write_rows != nullptr); + ASSERT_EQ(spill_write_rows->value(), count) + << "SpillWriteRows should match the number of rows sunk"; + + // Sink empty EOS after spill + Block empty_block; + st = sink_operator->sink(_helper.runtime_state.get(), &empty_block, true); + ASSERT_TRUE(st.ok()) << "sink eos failed: " << st.to_string(); +} + +} // namespace doris diff --git a/be/test/exec/operator/spill_sort_source_operator_test.cpp b/be/test/exec/operator/spill_sort_source_operator_test.cpp index 0bc1ae9bfb3722..5e59318dee4c1a 100644 --- a/be/test/exec/operator/spill_sort_source_operator_test.cpp +++ b/be/test/exec/operator/spill_sort_source_operator_test.cpp @@ -21,6 +21,7 @@ #include #include +#include #include "common/config.h" #include "core/block/block.h" @@ -40,6 +41,155 @@ class SpillSortSourceOperatorTest : public testing::Test { SpillSortTestHelper _helper; }; +namespace { + +struct SpillSortSourceTestContext { + std::shared_ptr shared_state; + SpillSortLocalState* local_state = nullptr; + SpillSortSinkLocalState* sink_local_state = nullptr; +}; + +void init_spill_sort_description(SpillSortSharedState* shared_state) { + auto* sorter = shared_state->in_mem_shared_state->sorter.get(); + auto& sort_desc = sorter->get_mutable_sort_description(); + sort_desc.resize(sorter->get_vsort_exec_exprs().ordering_expr_ctxs().size()); + for (int i = 0; i < static_cast(sort_desc.size()); ++i) { + sort_desc[i].column_number = i; + sort_desc[i].direction = 1; + sort_desc[i].nulls_direction = 1; + } +} + +Status prepare_spill_sort_source_context( + SpillSortTestHelper& helper, + const std::shared_ptr& source_operator, + const std::shared_ptr& sink_operator, + SpillSortSourceTestContext& context) { + auto tnode = helper.create_test_plan_node(); + RETURN_IF_ERROR(source_operator->init(tnode, helper.runtime_state.get())); + RETURN_IF_ERROR(source_operator->prepare(helper.runtime_state.get())); + + context.shared_state = + std::dynamic_pointer_cast(sink_operator->create_shared_state()); + DCHECK(context.shared_state != nullptr); + + RETURN_IF_ERROR(sink_operator->init(tnode, helper.runtime_state.get())); + RETURN_IF_ERROR(sink_operator->prepare(helper.runtime_state.get())); + + LocalSinkStateInfo sink_info { + 0, helper.operator_profile.get(), -1, context.shared_state.get(), {}, {}}; + RETURN_IF_ERROR(sink_operator->setup_local_state(helper.runtime_state.get(), sink_info)); + context.sink_local_state = reinterpret_cast( + helper.runtime_state->get_sink_local_state()); + DCHECK(context.sink_local_state != nullptr); + RETURN_IF_ERROR(context.sink_local_state->open(helper.runtime_state.get())); + + LocalStateInfo source_info {.parent_profile = helper.operator_profile.get(), + .scan_ranges = {}, + .shared_state = context.shared_state.get(), + .shared_state_map = {}, + .task_idx = 0}; + RETURN_IF_ERROR(source_operator->setup_local_state(helper.runtime_state.get(), source_info)); + context.local_state = reinterpret_cast( + helper.runtime_state->get_local_state(source_operator->operator_id())); + DCHECK(context.local_state != nullptr); + RETURN_IF_ERROR(context.local_state->open(helper.runtime_state.get())); + + context.shared_state->is_spilled = true; + init_spill_sort_description(context.shared_state.get()); + return Status::OK(); +} + +SpillFileSPtr create_sort_test_spill_file(RuntimeState* state, RuntimeProfile* profile, int node_id, + const std::string& prefix, + const std::vector& first_column, + const std::vector& second_column) { + EXPECT_EQ(first_column.size(), second_column.size()); + + SpillFileSPtr spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}", print_id(state->query_id()), prefix, node_id, + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + auto st = + ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, spill_file); + EXPECT_TRUE(st.ok()) << "create spill file failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(state, profile, writer); + EXPECT_TRUE(st.ok()) << "create writer failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + auto input_block = ColumnHelper::create_block(first_column); + input_block.insert(ColumnHelper::create_column_with_name(second_column)); + + st = writer->write_block(state, input_block); + EXPECT_TRUE(st.ok()) << "write block failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + st = writer->close(); + EXPECT_TRUE(st.ok()) << "close writer failed: " << st.to_string(); + if (!st.ok()) { + return nullptr; + } + + return spill_file; +} + +Status read_all_blocks_from_source(RuntimeState* state, SpillSortSourceOperatorX* source_operator, + std::unique_ptr& mutable_block) { + bool eos = false; + while (!eos) { + Block block; + RETURN_IF_ERROR(source_operator->get_block(state, &block, &eos)); + if (block.empty()) { + continue; + } + if (!mutable_block) { + mutable_block = MutableBlock::create_unique(std::move(block)); + } else { + RETURN_IF_ERROR(mutable_block->merge(std::move(block))); + } + } + return Status::OK(); +} + +std::vector collect_spill_files_for_cleanup( + const std::vector& original_files, SpillSortLocalState* local_state, + SpillSortSharedState* shared_state) { + std::vector spill_files; + std::unordered_set seen; + auto collect = [&](const SpillFileSPtr& spill_file) { + if (spill_file && seen.emplace(spill_file.get()).second) { + spill_files.emplace_back(spill_file); + } + }; + + for (const auto& spill_file : original_files) { + collect(spill_file); + } + for (const auto& spill_file : local_state->_current_merging_files) { + collect(spill_file); + } + for (const auto& spill_file : shared_state->sorted_spill_groups) { + collect(spill_file); + } + return spill_files; +} + +void delete_spill_files(const std::vector& spill_files) { + for (const auto& spill_file : spill_files) { + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(spill_file); + } +} + +} // namespace + TEST_F(SpillSortSourceOperatorTest, Basic) { auto [source_operator, sink_operator] = _helper.create_operators(); ASSERT_TRUE(source_operator != nullptr); @@ -118,8 +268,6 @@ TEST_F(SpillSortSourceOperatorTest, GetBlock) { auto* local_state = _helper.runtime_state->get_local_state(source_operator->operator_id()); ASSERT_TRUE(local_state != nullptr); - shared_state->setup_shared_profile(_helper.operator_profile.get()); - st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); @@ -196,8 +344,6 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill) { _helper.runtime_state->get_local_state(source_operator->operator_id())); ASSERT_TRUE(local_state != nullptr); - shared_state->setup_shared_profile(_helper.operator_profile.get()); - st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); @@ -205,22 +351,22 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill) { auto* sorter = shared_state->in_mem_shared_state->sorter.get(); - sorter->_sort_description.resize(sorter->_vsort_exec_exprs.ordering_expr_ctxs().size()); - for (int i = 0; i < sorter->_sort_description.size(); i++) { - sorter->_sort_description[i].column_number = i; - sorter->_sort_description[i].direction = 1; - sorter->_sort_description[i].nulls_direction = 1; + auto& sort_desc = sorter->get_mutable_sort_description(); + sort_desc.resize(sorter->get_vsort_exec_exprs().ordering_expr_ctxs().size()); + for (int i = 0; i < (int)sort_desc.size(); i++) { + sort_desc[i].column_number = i; + sort_desc[i].direction = 1; + sort_desc[i].nulls_direction = 1; } // Prepare stored streams for (size_t i = 0; i != 4; ++i) { - SpillStreamSPtr spill_stream; - st = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - _helper.runtime_state.get(), spill_stream, - print_id(_helper.runtime_state->query_id()), sink_operator->get_name(), - sink_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), _helper.operator_profile.get()); - ASSERT_TRUE(st.ok()) << "register_spill_stream failed: " << st.to_string(); + SpillFileSPtr spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}", print_id(_helper.runtime_state->query_id()), + sink_operator->get_name(), sink_operator->node_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, spill_file); + ASSERT_TRUE(st.ok()) << "create_spill_file failed: " << st.to_string(); std::vector data; std::vector data2; @@ -233,10 +379,16 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill) { input_block.insert(ColumnHelper::create_column_with_name(data2)); - st = spill_stream->spill_block(_helper.runtime_state.get(), input_block, true); - ASSERT_TRUE(st.ok()) << "spill_block failed: " << st.to_string(); + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_helper.runtime_state.get(), _helper.operator_profile.get(), + writer); + ASSERT_TRUE(st.ok()) << "create_writer failed: " << st.to_string(); + st = writer->write_block(_helper.runtime_state.get(), input_block); + ASSERT_TRUE(st.ok()) << "write_block failed: " << st.to_string(); + st = writer->close(); + ASSERT_TRUE(st.ok()) << "close writer failed: " << st.to_string(); - shared_state->sorted_streams.emplace_back(std::move(spill_stream)); + shared_state->sorted_spill_groups.emplace_back(std::move(spill_file)); } std::unique_ptr mutable_block; @@ -259,7 +411,7 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill) { } ASSERT_TRUE(eos); - ASSERT_TRUE(shared_state->sorted_streams.empty()) << "sorted_streams is not empty"; + ASSERT_TRUE(shared_state->sorted_spill_groups.empty()) << "sorted_spill_groups is not empty"; ASSERT_TRUE(mutable_block) << "mutable_block is null"; ASSERT_EQ(mutable_block->rows(), 40); auto output_block = mutable_block->to_block(); @@ -294,6 +446,49 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill) { std::cout << "************** HERE WE GO!!!!!! **************" << std::endl; } +// Verify that a normal revoke_memory invocation does not prematurely close the +// shared state. Closing is the responsibility of the sink/operator teardown +// path, not the spill logic itself. +TEST_F(SpillSortSourceOperatorTest, RevokeMemoryKeepsSharedStateOpen) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + // prepare sink operator and shared state as in other tests + auto tnode = _helper.create_test_plan_node(); + auto shared_state = + std::dynamic_pointer_cast(sink_operator->create_shared_state()); + ASSERT_TRUE(shared_state != nullptr); + + // initialize sink + auto st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + LocalSinkStateInfo sink_info {0, _helper.operator_profile.get(), -1, shared_state.get(), {}, + {}}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), sink_info); + ASSERT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + auto* sink_local_state = _helper.runtime_state->get_sink_local_state(); + DCHECK(sink_local_state != nullptr); + + // open the local state to initialize in-memory sorter etc. + st = sink_local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + // clear any closure flag before revoking memory + shared_state->is_closed = false; + + // call revoke_memory with no data; should succeed and leave shared_state open + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "revoke_memory failed: " << st.to_string(); + ASSERT_FALSE(shared_state->is_closed) << "shared state was closed by a successful revoke"; + + // cleanup + st = sink_local_state->close(_helper.runtime_state.get(), Status::OK()); + ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); +} + // Same as `GetBlockWithSpill`, but with a different `spill_sort_mem_limit` value. TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill2) { auto [source_operator, sink_operator] = _helper.create_operators(); @@ -340,8 +535,6 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill2) { _helper.runtime_state->get_local_state(source_operator->operator_id())); ASSERT_TRUE(local_state != nullptr); - shared_state->setup_shared_profile(_helper.operator_profile.get()); - st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); @@ -349,22 +542,22 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill2) { auto* sorter = shared_state->in_mem_shared_state->sorter.get(); - sorter->_sort_description.resize(sorter->_vsort_exec_exprs.ordering_expr_ctxs().size()); - for (int i = 0; i < sorter->_sort_description.size(); i++) { - sorter->_sort_description[i].column_number = i; - sorter->_sort_description[i].direction = 1; - sorter->_sort_description[i].nulls_direction = 1; + auto& sort_desc = sorter->get_mutable_sort_description(); + sort_desc.resize(sorter->get_vsort_exec_exprs().ordering_expr_ctxs().size()); + for (int i = 0; i < (int)sort_desc.size(); i++) { + sort_desc[i].column_number = i; + sort_desc[i].direction = 1; + sort_desc[i].nulls_direction = 1; } // Prepare stored streams for (size_t i = 0; i != 4; ++i) { - SpillStreamSPtr spill_stream; - st = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - _helper.runtime_state.get(), spill_stream, - print_id(_helper.runtime_state->query_id()), sink_operator->get_name(), - sink_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), _helper.operator_profile.get()); - ASSERT_TRUE(st.ok()) << "register_spill_stream failed: " << st.to_string(); + SpillFileSPtr spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}", print_id(_helper.runtime_state->query_id()), + sink_operator->get_name(), sink_operator->node_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, spill_file); + ASSERT_TRUE(st.ok()) << "create_spill_file failed: " << st.to_string(); std::vector data; std::vector data2; @@ -377,10 +570,16 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill2) { input_block.insert(ColumnHelper::create_column_with_name(data2)); - st = spill_stream->spill_block(_helper.runtime_state.get(), input_block, true); - ASSERT_TRUE(st.ok()) << "spill_block failed: " << st.to_string(); + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_helper.runtime_state.get(), _helper.operator_profile.get(), + writer); + ASSERT_TRUE(st.ok()) << "create_writer failed: " << st.to_string(); + st = writer->write_block(_helper.runtime_state.get(), input_block); + ASSERT_TRUE(st.ok()) << "write_block failed: " << st.to_string(); + st = writer->close(); + ASSERT_TRUE(st.ok()) << "close writer failed: " << st.to_string(); - shared_state->sorted_streams.emplace_back(std::move(spill_stream)); + shared_state->sorted_spill_groups.emplace_back(std::move(spill_file)); } auto query_options = _helper.runtime_state->query_options(); @@ -407,7 +606,7 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill2) { } } - ASSERT_TRUE(shared_state->sorted_streams.empty()) << "sorted_streams is not empty"; + ASSERT_TRUE(shared_state->sorted_spill_groups.empty()) << "sorted_spill_groups is not empty"; ASSERT_TRUE(mutable_block) << "mutable_block is null"; ASSERT_EQ(mutable_block->rows(), 40); auto output_block = mutable_block->to_block(); @@ -440,6 +639,202 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpill2) { ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); } +TEST_F(SpillSortSourceOperatorTest, ExecuteMergeSortSpillFilesFastPath) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + SpillSortSourceTestContext context; + auto st = prepare_spill_sort_source_context(_helper, source_operator, sink_operator, context); + ASSERT_TRUE(st.ok()) << "prepare spill sort source context failed: " << st.to_string(); + + std::vector original_files; + original_files.emplace_back(create_sort_test_spill_file( + _helper.runtime_state.get(), context.local_state->operator_profile(), + source_operator->node_id(), "spill_sort_merge_fast_0", {1, 5}, {10, 50})); + original_files.emplace_back(create_sort_test_spill_file( + _helper.runtime_state.get(), context.local_state->operator_profile(), + source_operator->node_id(), "spill_sort_merge_fast_1", {2, 6}, {20, 60})); + original_files.emplace_back(create_sort_test_spill_file( + _helper.runtime_state.get(), context.local_state->operator_profile(), + source_operator->node_id(), "spill_sort_merge_fast_2", {0, 4}, {0, 40})); + original_files.emplace_back(create_sort_test_spill_file( + _helper.runtime_state.get(), context.local_state->operator_profile(), + source_operator->node_id(), "spill_sort_merge_fast_3", {3, 7}, {30, 70})); + + for (const auto& spill_file : original_files) { + ASSERT_TRUE(spill_file != nullptr); + context.shared_state->sorted_spill_groups.emplace_back(spill_file); + } + + st = context.local_state->execute_merge_sort_spill_files(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "execute_merge_sort_spill_files failed: " << st.to_string(); + ASSERT_TRUE(context.shared_state->sorted_spill_groups.empty()); + ASSERT_TRUE(context.local_state->_merger != nullptr); + ASSERT_EQ(context.local_state->_current_merging_files.size(), 4); + ASSERT_EQ(context.local_state->_current_merging_readers.size(), 4); + + auto cleanup_files = collect_spill_files_for_cleanup(original_files, context.local_state, + context.shared_state.get()); + + std::unique_ptr mutable_block; + st = read_all_blocks_from_source(_helper.runtime_state.get(), source_operator.get(), + mutable_block); + ASSERT_TRUE(st.ok()) << "read merged blocks failed: " << st.to_string(); + ASSERT_TRUE(mutable_block != nullptr); + ASSERT_EQ(mutable_block->rows(), 8); + + auto output_block = mutable_block->to_block(); + const auto& col1 = output_block.get_by_position(0).column; + for (int i = 0; i < 8; ++i) { + ASSERT_EQ(col1->get_int(i), i); + } + + st = context.local_state->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); + ASSERT_TRUE(context.local_state->_current_merging_files.empty()); + ASSERT_TRUE(context.local_state->_current_merging_readers.empty()); + ASSERT_EQ(context.local_state->_merger, nullptr); + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "source close failed: " << st.to_string(); + st = context.sink_local_state->close(_helper.runtime_state.get(), Status::OK()); + ASSERT_TRUE(st.ok()) << "sink local close failed: " << st.to_string(); + + delete_spill_files(cleanup_files); +} + +TEST_F(SpillSortSourceOperatorTest, ExecuteMergeSortSpillFilesIntermediateRound) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + SpillSortSourceTestContext context; + auto st = prepare_spill_sort_source_context(_helper, source_operator, sink_operator, context); + ASSERT_TRUE(st.ok()) << "prepare spill sort source context failed: " << st.to_string(); + + std::vector original_files; + for (int i = 0; i < 9; ++i) { + auto spill_file = create_sort_test_spill_file( + _helper.runtime_state.get(), context.local_state->operator_profile(), + source_operator->node_id(), fmt::format("spill_sort_merge_intermediate_{}", i), {i}, + {100 + i}); + ASSERT_TRUE(spill_file != nullptr); + original_files.emplace_back(spill_file); + context.shared_state->sorted_spill_groups.emplace_back(spill_file); + } + + st = context.local_state->execute_merge_sort_spill_files(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "execute_merge_sort_spill_files failed: " << st.to_string(); + ASSERT_TRUE(context.shared_state->sorted_spill_groups.empty()); + ASSERT_TRUE(context.local_state->_merger != nullptr); + ASSERT_EQ(context.local_state->_current_merging_files.size(), 2); + ASSERT_EQ(context.local_state->_current_merging_readers.size(), 2); + + auto cleanup_files = collect_spill_files_for_cleanup(original_files, context.local_state, + context.shared_state.get()); + ASSERT_GT(cleanup_files.size(), original_files.size()); + + std::unique_ptr mutable_block; + st = read_all_blocks_from_source(_helper.runtime_state.get(), source_operator.get(), + mutable_block); + ASSERT_TRUE(st.ok()) << "read merged blocks failed: " << st.to_string(); + ASSERT_TRUE(mutable_block != nullptr); + ASSERT_EQ(mutable_block->rows(), 9); + + auto output_block = mutable_block->to_block(); + const auto& col1 = output_block.get_by_position(0).column; + for (int i = 0; i < 9; ++i) { + ASSERT_EQ(col1->get_int(i), i); + } + + st = context.local_state->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "source close failed: " << st.to_string(); + st = context.sink_local_state->close(_helper.runtime_state.get(), Status::OK()); + ASSERT_TRUE(st.ok()) << "sink local close failed: " << st.to_string(); + + delete_spill_files(cleanup_files); +} + +TEST_F(SpillSortSourceOperatorTest, ExecuteMergeSortSpillFilesRecoverSpillDataError) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + SpillSortSourceTestContext context; + auto st = prepare_spill_sort_source_context(_helper, source_operator, sink_operator, context); + ASSERT_TRUE(st.ok()) << "prepare spill sort source context failed: " << st.to_string(); + + std::vector original_files; + for (int i = 0; i < 9; ++i) { + auto spill_file = create_sort_test_spill_file( + _helper.runtime_state.get(), context.local_state->operator_profile(), + source_operator->node_id(), fmt::format("spill_sort_merge_recover_error_{}", i), + {i}, {i}); + ASSERT_TRUE(spill_file != nullptr); + original_files.emplace_back(spill_file); + context.shared_state->sorted_spill_groups.emplace_back(spill_file); + } + + { + SpillableDebugPointHelper dp_helper("fault_inject::spill_sort_source::recover_spill_data"); + st = context.local_state->execute_merge_sort_spill_files(_helper.runtime_state.get()); + } + ASSERT_FALSE(st.ok()); + ASSERT_TRUE(st.to_string().find("recover_spill_data failed") != std::string::npos) + << "unexpected error: " << st.to_string(); + ASSERT_TRUE(context.local_state->_merger != nullptr); + ASSERT_FALSE(context.local_state->_current_merging_readers.empty()); + + auto cleanup_files = collect_spill_files_for_cleanup(original_files, context.local_state, + context.shared_state.get()); + + st = context.local_state->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "source close failed: " << st.to_string(); + st = context.sink_local_state->close(_helper.runtime_state.get(), Status::OK()); + ASSERT_TRUE(st.ok()) << "sink local close failed: " << st.to_string(); + + delete_spill_files(cleanup_files); +} + +TEST_F(SpillSortSourceOperatorTest, ExecuteMergeSortSpillFilesSpillMergedDataError) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + SpillSortSourceTestContext context; + auto st = prepare_spill_sort_source_context(_helper, source_operator, sink_operator, context); + ASSERT_TRUE(st.ok()) << "prepare spill sort source context failed: " << st.to_string(); + + std::vector original_files; + for (int i = 0; i < 9; ++i) { + auto spill_file = create_sort_test_spill_file( + _helper.runtime_state.get(), context.local_state->operator_profile(), + source_operator->node_id(), fmt::format("spill_sort_merge_spill_error_{}", i), {i}, + {i}); + ASSERT_TRUE(spill_file != nullptr); + original_files.emplace_back(spill_file); + context.shared_state->sorted_spill_groups.emplace_back(spill_file); + } + + { + SpillableDebugPointHelper dp_helper("fault_inject::spill_sort_source::spill_merged_data"); + st = context.local_state->execute_merge_sort_spill_files(_helper.runtime_state.get()); + } + ASSERT_FALSE(st.ok()); + ASSERT_TRUE(st.to_string().find("spill_merged_data failed") != std::string::npos) + << "unexpected error: " << st.to_string(); + ASSERT_TRUE(context.local_state->_merger != nullptr); + ASSERT_FALSE(context.local_state->_current_merging_readers.empty()); + + auto cleanup_files = collect_spill_files_for_cleanup(original_files, context.local_state, + context.shared_state.get()); + + st = context.local_state->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "source close failed: " << st.to_string(); + st = context.sink_local_state->close(_helper.runtime_state.get(), Status::OK()); + ASSERT_TRUE(st.ok()) << "sink local close failed: " << st.to_string(); + + delete_spill_files(cleanup_files); +} + TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpillError) { auto [source_operator, sink_operator] = _helper.create_operators(); ASSERT_TRUE(source_operator != nullptr); @@ -485,8 +880,6 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpillError) { _helper.runtime_state->get_local_state(source_operator->operator_id())); ASSERT_TRUE(local_state != nullptr); - shared_state->setup_shared_profile(_helper.operator_profile.get()); - st = local_state->open(_helper.runtime_state.get()); ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); @@ -494,22 +887,22 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpillError) { auto* sorter = shared_state->in_mem_shared_state->sorter.get(); - sorter->_sort_description.resize(sorter->_vsort_exec_exprs.ordering_expr_ctxs().size()); - for (int i = 0; i < sorter->_sort_description.size(); i++) { - sorter->_sort_description[i].column_number = i; - sorter->_sort_description[i].direction = 1; - sorter->_sort_description[i].nulls_direction = 1; + auto& sort_desc = sorter->get_mutable_sort_description(); + sort_desc.resize(sorter->get_vsort_exec_exprs().ordering_expr_ctxs().size()); + for (int i = 0; i < (int)sort_desc.size(); i++) { + sort_desc[i].column_number = i; + sort_desc[i].direction = 1; + sort_desc[i].nulls_direction = 1; } // Prepare stored streams for (size_t i = 0; i != 4; ++i) { - SpillStreamSPtr spill_stream; - st = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( - _helper.runtime_state.get(), spill_stream, - print_id(_helper.runtime_state->query_id()), sink_operator->get_name(), - sink_operator->node_id(), std::numeric_limits::max(), - std::numeric_limits::max(), _helper.operator_profile.get()); - ASSERT_TRUE(st.ok()) << "register_spill_stream failed: " << st.to_string(); + SpillFileSPtr spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}", print_id(_helper.runtime_state->query_id()), + sink_operator->get_name(), sink_operator->node_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, spill_file); + ASSERT_TRUE(st.ok()) << "create_spill_file failed: " << st.to_string(); std::vector data; std::vector data2; @@ -522,13 +915,19 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpillError) { input_block.insert(ColumnHelper::create_column_with_name(data2)); - st = spill_stream->spill_block(_helper.runtime_state.get(), input_block, true); - ASSERT_TRUE(st.ok()) << "spill_block failed: " << st.to_string(); + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_helper.runtime_state.get(), _helper.operator_profile.get(), + writer); + ASSERT_TRUE(st.ok()) << "create_writer failed: " << st.to_string(); + st = writer->write_block(_helper.runtime_state.get(), input_block); + ASSERT_TRUE(st.ok()) << "write_block failed: " << st.to_string(); + st = writer->close(); + ASSERT_TRUE(st.ok()) << "close writer failed: " << st.to_string(); - shared_state->sorted_streams.emplace_back(std::move(spill_stream)); + shared_state->sorted_spill_groups.emplace_back(std::move(spill_file)); } - SpillableDebugPointHelper dp_helper("fault_inject::spill_stream::read_next_block"); + SpillableDebugPointHelper dp_helper("fault_inject::spill_file::read_next_block"); std::unique_ptr mutable_block; bool eos = false; @@ -561,4 +960,240 @@ TEST_F(SpillSortSourceOperatorTest, GetBlockWithSpillError) { ASSERT_TRUE(st.ok()) << "close failed: " << st.to_string(); } -} // namespace doris \ No newline at end of file +// Test reading from a single spill file to verify minimal sorted output. +TEST_F(SpillSortSourceOperatorTest, GetBlockWithSingleSpillFile) { + auto [source_operator, sink_operator] = _helper.create_operators(); + ASSERT_TRUE(source_operator != nullptr); + + auto tnode = _helper.create_test_plan_node(); + auto st = source_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + + st = source_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + auto shared_state = + std::dynamic_pointer_cast(sink_operator->create_shared_state()); + ASSERT_TRUE(shared_state != nullptr); + + st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "init failed: " << st.to_string(); + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "prepare failed: " << st.to_string(); + + LocalSinkStateInfo sink_info {0, _helper.operator_profile.get(), -1, shared_state.get(), {}, + {}}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), sink_info); + ASSERT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + auto* sink_local_state = _helper.runtime_state->get_sink_local_state(); + DCHECK(sink_local_state != nullptr); + st = sink_local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + LocalStateInfo info {.parent_profile = _helper.operator_profile.get(), + .scan_ranges = {}, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .task_idx = 0}; + st = source_operator->setup_local_state(_helper.runtime_state.get(), info); + ASSERT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + auto* local_state = reinterpret_cast( + _helper.runtime_state->get_local_state(source_operator->operator_id())); + ASSERT_TRUE(local_state != nullptr); + st = local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + shared_state->is_spilled = true; + + auto* sorter = shared_state->in_mem_shared_state->sorter.get(); + auto& sort_desc = sorter->get_mutable_sort_description(); + sort_desc.resize(sorter->get_vsort_exec_exprs().ordering_expr_ctxs().size()); + for (int i = 0; i < (int)sort_desc.size(); i++) { + sort_desc[i].column_number = i; + sort_desc[i].direction = 1; + sort_desc[i].nulls_direction = 1; + } + + // Create a single spill file with descending data + { + SpillFileSPtr spill_file; + auto relative_path = fmt::format("{}/{}-{}-{}", print_id(_helper.runtime_state->query_id()), + sink_operator->get_name(), sink_operator->node_id(), + ExecEnv::GetInstance()->spill_file_mgr()->next_id()); + st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(relative_path, spill_file); + ASSERT_TRUE(st.ok()) << "create_spill_file failed: " << st.to_string(); + + auto input_block = ColumnHelper::create_block({1, 2, 3, 4, 5}); + input_block.insert( + ColumnHelper::create_column_with_name({10, 20, 30, 40, 50})); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_helper.runtime_state.get(), _helper.operator_profile.get(), + writer); + ASSERT_TRUE(st.ok()); + st = writer->write_block(_helper.runtime_state.get(), input_block); + ASSERT_TRUE(st.ok()); + st = writer->close(); + ASSERT_TRUE(st.ok()); + + shared_state->sorted_spill_groups.emplace_back(std::move(spill_file)); + } + + // Read all blocks from source + std::unique_ptr mutable_block; + bool eos = false; + while (!eos) { + Block block; + shared_state->spill_block_batch_row_count = 100; + st = source_operator->get_block(_helper.runtime_state.get(), &block, &eos); + ASSERT_TRUE(st.ok()) << "get_block failed: " << st.to_string(); + if (block.empty()) { + continue; + } + if (!mutable_block) { + mutable_block = MutableBlock::create_unique(std::move(block)); + } else { + st = mutable_block->merge(std::move(block)); + ASSERT_TRUE(st.ok()); + } + } + + ASSERT_TRUE(eos); + ASSERT_TRUE(mutable_block) << "mutable_block is null"; + ASSERT_EQ(mutable_block->rows(), 5); + + auto output_block = mutable_block->to_block(); + const auto& col1 = output_block.get_by_position(0).column; + + // Verify sorted order (ascending) + for (size_t i = 1; i < col1->size(); ++i) { + ASSERT_GE(col1->get_int(i), col1->get_int(i - 1)); + } + + st = local_state->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); +} + +// Test full pipeline: sink data → revoke → read back sorted from source. +TEST_F(SpillSortSourceOperatorTest, EndToEndSinkAndSource) { + auto [source_operator, sink_operator] = _helper.create_operators(); + + auto tnode = _helper.create_test_plan_node(); + auto shared_state = + std::dynamic_pointer_cast(sink_operator->create_shared_state()); + ASSERT_TRUE(shared_state != nullptr); + + // Initialize and prepare both operators + auto st = sink_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = sink_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + st = source_operator->init(tnode, _helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = source_operator->prepare(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + shared_state->create_source_dependency(sink_operator->operator_id(), sink_operator->node_id(), + "SpillSortSinkOperatorTest"); + + // Setup sink local state + LocalSinkStateInfo sink_info {0, _helper.operator_profile.get(), -1, shared_state.get(), {}, + {}}; + st = sink_operator->setup_local_state(_helper.runtime_state.get(), sink_info); + ASSERT_TRUE(st.ok()); + + auto* sink_local_state = reinterpret_cast( + _helper.runtime_state->get_sink_local_state()); + ASSERT_TRUE(sink_local_state != nullptr); + st = sink_local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + // Setup source local state + LocalStateInfo source_info {.parent_profile = _helper.operator_profile.get(), + .scan_ranges = {}, + .shared_state = shared_state.get(), + .shared_state_map = {}, + .task_idx = 0}; + st = source_operator->setup_local_state(_helper.runtime_state.get(), source_info); + ASSERT_TRUE(st.ok()); + + auto* source_local_state = reinterpret_cast( + _helper.runtime_state->get_local_state(source_operator->operator_id())); + ASSERT_TRUE(source_local_state != nullptr); + st = source_local_state->open(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + // Sink batch 1: {5,3,1,4,2} → revoke + auto block1 = ColumnHelper::create_block({5, 3, 1, 4, 2}); + block1.insert(ColumnHelper::create_column_with_name({50, 30, 10, 40, 20})); + st = sink_operator->sink(_helper.runtime_state.get(), &block1, false); + ASSERT_TRUE(st.ok()); + st = sink_operator->revoke_memory(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + + // Sink batch 2: {10,8,6,9,7} → revoke + auto block2 = ColumnHelper::create_block({10, 8, 6, 9, 7}); + block2.insert(ColumnHelper::create_column_with_name({100, 80, 60, 90, 70})); + st = sink_operator->sink(_helper.runtime_state.get(), &block2, false); + ASSERT_TRUE(st.ok()); + + // Sink EOS (triggers final revoke since is_spilled) + Block empty_block; + st = sink_operator->sink(_helper.runtime_state.get(), &empty_block, true); + ASSERT_TRUE(st.ok()); + + ASSERT_TRUE(shared_state->is_spilled); + ASSERT_GE(shared_state->sorted_spill_groups.size(), 2u); + + // Read back from source + auto* sorter = shared_state->in_mem_shared_state->sorter.get(); + auto& sort_desc = sorter->get_mutable_sort_description(); + sort_desc.resize(sorter->get_vsort_exec_exprs().ordering_expr_ctxs().size()); + for (int i = 0; i < (int)sort_desc.size(); i++) { + sort_desc[i].column_number = i; + sort_desc[i].direction = 1; + sort_desc[i].nulls_direction = 1; + } + + std::unique_ptr mutable_block; + bool eos = false; + while (!eos) { + Block block; + shared_state->spill_block_batch_row_count = 100; + st = source_operator->get_block(_helper.runtime_state.get(), &block, &eos); + ASSERT_TRUE(st.ok()) << "get_block failed: " << st.to_string(); + if (block.empty()) continue; + if (!mutable_block) { + mutable_block = MutableBlock::create_unique(std::move(block)); + } else { + st = mutable_block->merge(std::move(block)); + ASSERT_TRUE(st.ok()); + } + } + + ASSERT_TRUE(eos); + ASSERT_TRUE(mutable_block); + ASSERT_EQ(mutable_block->rows(), 10); + + auto output_block = mutable_block->to_block(); + const auto& col1 = output_block.get_by_position(0).column; + + // Verify sorted order + for (size_t i = 1; i < col1->size(); ++i) { + ASSERT_GE(col1->get_int(i), col1->get_int(i - 1)) + << "Not sorted at index " << i << ": " << col1->get_int(i - 1) << " > " + << col1->get_int(i); + } + + st = source_local_state->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); + st = source_operator->close(_helper.runtime_state.get()); + ASSERT_TRUE(st.ok()); +} + +} // namespace doris diff --git a/be/test/exec/operator/spill_sort_test_helper.cpp b/be/test/exec/operator/spill_sort_test_helper.cpp index a11bab3b5c486a..80286f852f1fdd 100644 --- a/be/test/exec/operator/spill_sort_test_helper.cpp +++ b/be/test/exec/operator/spill_sort_test_helper.cpp @@ -128,13 +128,50 @@ TDescriptorTable SpillSortTestHelper::create_test_table_descriptor(bool nullable SpillSortLocalState* SpillSortTestHelper::create_source_local_state( RuntimeState* state, SpillSortSourceOperatorX* source_operator, std::shared_ptr& shared_state) { - return nullptr; + // Build a minimal local state manually. Many tests prefer to use the + // operators' own setup routines, but helper functions like this allow + // individual units to be exercised without the full pipeline task. + auto local_state_uptr = std::make_unique(state, source_operator); + auto* local_state = local_state_uptr.get(); + + shared_state = std::make_shared(); + local_state->_shared_state = shared_state.get(); + // default flags + shared_state->is_spilled = true; + + // lightweight profile counters so that any operations using them don't + // dereference null pointers + ADD_TIMER(local_state->common_profile(), "ExecTime"); + local_state->common_profile()->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 0); + local_state->init_spill_read_counters(); + local_state->init_spill_write_counters(); + local_state->_internal_runtime_profile = std::make_unique("inner_test"); + + state->emplace_local_state(source_operator->operator_id(), std::move(local_state_uptr)); + return local_state; } SpillSortSinkLocalState* SpillSortTestHelper::create_sink_local_state( RuntimeState* state, SpillSortSinkOperatorX* sink_operator, std::shared_ptr& shared_state) { - return nullptr; + auto local_state_uptr = SpillSortSinkLocalState::create_unique(sink_operator, state); + auto* local_state = local_state_uptr.get(); + + shared_state = std::make_shared(); + // make sure shared profile is ready when the local state writes to it + local_state->init_spill_counters(); + + ADD_TIMER(local_state->common_profile(), "ExecTime"); + local_state->common_profile()->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 0); + local_state->_internal_runtime_profile = std::make_unique("inner_test"); + + // create and attach dependency similar to what the operator would do + local_state->_dependency = shared_state->create_sink_dependency( + sink_operator->dests_id().front(), sink_operator->operator_id(), + "SpillSortSinkTestDep"); + + state->emplace_sink_local_state(sink_operator->operator_id(), std::move(local_state_uptr)); + return local_state; } std::tuple, std::shared_ptr> diff --git a/be/test/exec/operator/spill_sort_test_helper.h b/be/test/exec/operator/spill_sort_test_helper.h index 4a28ee06a718ba..c887212b2fd1f8 100644 --- a/be/test/exec/operator/spill_sort_test_helper.h +++ b/be/test/exec/operator/spill_sort_test_helper.h @@ -35,7 +35,7 @@ #include "exec/operator/spill_sort_source_operator.h" #include "exec/operator/spillable_operator_test_helper.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" #include "runtime/runtime_profile.h" diff --git a/be/test/exec/operator/spillable_operator_test_helper.cpp b/be/test/exec/operator/spillable_operator_test_helper.cpp index 8c57727ce74f7f..fb87b156545db0 100644 --- a/be/test/exec/operator/spillable_operator_test_helper.cpp +++ b/be/test/exec/operator/spillable_operator_test_helper.cpp @@ -73,6 +73,15 @@ void SpillableOperatorTestHelper::SetUp() { runtime_state->resize_op_id_to_local_state(-100); runtime_state->set_max_operator_id(-100); + // Configure spill partition counts to match TEST_PARTITION_COUNT + runtime_state->_query_options.__isset.spill_hash_join_partition_count = true; + runtime_state->_query_options.spill_hash_join_partition_count = TEST_PARTITION_COUNT; + runtime_state->_query_options.__isset.spill_aggregation_partition_count = true; + runtime_state->_query_options.spill_aggregation_partition_count = TEST_PARTITION_COUNT; + // Set a low min_revocable_mem so small test data can still trigger spill + runtime_state->_query_options.__isset.min_revocable_mem = true; + runtime_state->_query_options.min_revocable_mem = 1; + auto desc_table = create_test_table_descriptor(false); auto st = DescriptorTbl::create(obj_pool.get(), desc_table, &desc_tbl); DCHECK(!desc_table.slotDescriptors.empty()); @@ -85,16 +94,17 @@ void SpillableOperatorTestHelper::SetUp() { << " failed: " << st.to_string(); std::unordered_map> data_map; data_map.emplace("test", std::move(spill_data_dir)); - auto* spill_stream_manager = new SpillStreamManager(std::move(data_map)); - ExecEnv::GetInstance()->_spill_stream_mgr = spill_stream_manager; - st = spill_stream_manager->init(); - EXPECT_TRUE(st.ok()) << "init spill stream manager failed: " << st.to_string(); + + auto* spill_file_manager = new SpillFileManager(std::move(data_map)); + ExecEnv::GetInstance()->_spill_file_mgr = spill_file_manager; + st = spill_file_manager->init(); + EXPECT_TRUE(st.ok()) << "init spill file manager failed: " << st.to_string(); } void SpillableOperatorTestHelper::TearDown() { - doris::ExecEnv::GetInstance()->spill_stream_mgr()->stop(); - SAFE_DELETE(ExecEnv::GetInstance()->_spill_stream_mgr); runtime_state.reset(); + doris::ExecEnv::GetInstance()->spill_file_mgr()->stop(); + SAFE_DELETE(ExecEnv::GetInstance()->_spill_file_mgr); } } // namespace doris \ No newline at end of file diff --git a/be/test/exec/operator/spillable_operator_test_helper.h b/be/test/exec/operator/spillable_operator_test_helper.h index 9f87bfa98137ca..fa9bbc5d9dbdc1 100644 --- a/be/test/exec/operator/spillable_operator_test_helper.h +++ b/be/test/exec/operator/spillable_operator_test_helper.h @@ -24,12 +24,13 @@ #include #include +#include #include #include "common/object_pool.h" #include "core/block/block.h" #include "exec/pipeline/pipeline_task.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/runtime_profile.h" #include "testutil/mock/mock_runtime_state.h" @@ -79,15 +80,19 @@ class MockExpr : public VExpr { class SpillableDebugPointHelper { public: SpillableDebugPointHelper(const std::string name) - : _enable_debug_points(config::enable_debug_points) { + : _enable_debug_points(config::enable_debug_points), _debug_point_name(name) { config::enable_debug_points = true; DebugPoints::instance()->add(name); } - ~SpillableDebugPointHelper() { config::enable_debug_points = _enable_debug_points; } + ~SpillableDebugPointHelper() { + DebugPoints::instance()->remove(_debug_point_name); + config::enable_debug_points = _enable_debug_points; + } private: const bool _enable_debug_points; + const std::string _debug_point_name; }; class SpillableOperatorTestHelper { diff --git a/be/test/exec/pipeline/multi_cast_data_streamer_test.cpp b/be/test/exec/pipeline/multi_cast_data_streamer_test.cpp index e627fae8fb9acb..d50523605d0964 100644 --- a/be/test/exec/pipeline/multi_cast_data_streamer_test.cpp +++ b/be/test/exec/pipeline/multi_cast_data_streamer_test.cpp @@ -23,7 +23,7 @@ #include #include "exec/pipeline/dependency.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/runtime_profile.h" #include "storage/olap_define.h" #include "testutil/column_helper.h" @@ -76,7 +76,6 @@ class MultiCastDataStreamerTest : public testing::Test { shared_state = std::make_shared(&pool, cast_sender_count, 0); multi_cast_data_streamer = std::make_unique(&pool, cast_sender_count, 0); - shared_state->setup_shared_profile(profile.get()); multi_cast_data_streamer->set_sink_profile(profile.get()); source_profiles.resize(cast_sender_count); @@ -93,8 +92,6 @@ class MultiCastDataStreamerTest : public testing::Test { ADD_TIMER_WITH_LEVEL(source_common_profiles[i].get(), "ExecTime", 1); ADD_TIMER_WITH_LEVEL(source_custom_profiles[i].get(), "SpillTotalTime", 1); ADD_TIMER_WITH_LEVEL(source_custom_profiles[i].get(), "SpillRecoverTime", 1); - ADD_COUNTER_WITH_LEVEL(source_custom_profiles[i].get(), "SpillReadTaskWaitInQueueCount", - TUnit::UNIT, 1); ADD_COUNTER_WITH_LEVEL(source_custom_profiles[i].get(), "SpillReadTaskCount", TUnit::UNIT, 1); ADD_TIMER_WITH_LEVEL(source_custom_profiles[i].get(), "SpillReadTaskWaitInQueueTime", @@ -134,9 +131,9 @@ class MultiCastDataStreamerTest : public testing::Test { << " failed: " << st.to_string(); std::unordered_map> data_map; data_map.emplace("test", std::move(spill_data_dir)); - auto* spill_stream_manager = new SpillStreamManager(std::move(data_map)); - ExecEnv::GetInstance()->_spill_stream_mgr = spill_stream_manager; - st = spill_stream_manager->init(); + auto* spill_file_manager = new SpillFileManager(std::move(data_map)); + ExecEnv::GetInstance()->_spill_file_mgr = spill_file_manager; + st = spill_file_manager->init(); EXPECT_TRUE(st.ok()) << "init spill stream manager failed: " << st.to_string(); EXPECT_EQ(state.enable_spill(), false); @@ -146,8 +143,8 @@ class MultiCastDataStreamerTest : public testing::Test { ExecEnv::GetInstance()->_fragment_mgr->stop(); SAFE_DELETE(ExecEnv::GetInstance()->_fragment_mgr); ExecEnv::GetInstance()->_fragment_mgr = fragment_mgr; - doris::ExecEnv::GetInstance()->spill_stream_mgr()->stop(); - SAFE_DELETE(ExecEnv::GetInstance()->_spill_stream_mgr); + doris::ExecEnv::GetInstance()->spill_file_mgr()->stop(); + SAFE_DELETE(ExecEnv::GetInstance()->_spill_file_mgr); } ObjectPool pool; diff --git a/be/test/exec/pipeline/partitioned_agg_shared_state_test.cpp b/be/test/exec/pipeline/partitioned_agg_shared_state_test.cpp new file mode 100644 index 00000000000000..4118d923e57be6 --- /dev/null +++ b/be/test/exec/pipeline/partitioned_agg_shared_state_test.cpp @@ -0,0 +1,221 @@ +// 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 + +#include +#include + +#include "exec/common/agg_utils.h" +#include "exec/pipeline/dependency.h" +#include "exec/spill/spill_file_manager.h" +#include "io/fs/local_file_system.h" +#include "runtime/exec_env.h" + +namespace doris { + +class PartitionedAggSharedStateTest : public testing::Test { +protected: + void SetUp() override { + _exec_env = ExecEnv::GetInstance(); + ASSERT_TRUE(_exec_env != nullptr); + + // Initialize spill file manager if not already done + if (_exec_env->spill_file_mgr() == nullptr) { + _spill_dir = "./ut_dir/doris_ut_partitioned_agg_" + std::to_string(getpid()); + auto spill_data_dir = std::make_unique(_spill_dir, -1); + auto st = + io::global_local_filesystem()->create_directory(spill_data_dir->path(), false); + ASSERT_TRUE(st.ok()) << st.to_string(); + std::unordered_map> data_map; + data_map.emplace("test", std::move(spill_data_dir)); + _spill_file_mgr = std::make_unique(std::move(data_map)); + _exec_env->_spill_file_mgr = _spill_file_mgr.get(); + st = _spill_file_mgr->init(); + ASSERT_TRUE(st.ok()) << st.to_string(); + _owns_spill_mgr = true; + } + } + + void TearDown() override { + if (_owns_spill_mgr) { + _exec_env->_spill_file_mgr = nullptr; + _spill_file_mgr->stop(); + _spill_file_mgr.reset(); + if (!_spill_dir.empty()) { + std::filesystem::remove_all(_spill_dir); + } + } + } + + ExecEnv* _exec_env = nullptr; + std::unique_ptr _spill_file_mgr; + std::string _spill_dir; + bool _owns_spill_mgr = false; +}; + +TEST_F(PartitionedAggSharedStateTest, CloseWithEmptyPartitions) { + PartitionedAggSharedState state; + state.close(); + ASSERT_TRUE(state._spill_partitions.empty()); +} + +TEST_F(PartitionedAggSharedStateTest, CloseWithNullPartitions) { + PartitionedAggSharedState state; + state._spill_partitions.emplace_back(nullptr); + state._spill_partitions.emplace_back(nullptr); + state._spill_partitions.emplace_back(nullptr); + ASSERT_EQ(state._spill_partitions.size(), 3); + + state.close(); + ASSERT_TRUE(state._spill_partitions.empty()); +} + +TEST_F(PartitionedAggSharedStateTest, CloseCalledTwiceIsIdempotent) { + PartitionedAggSharedState state; + state._spill_partitions.emplace_back(nullptr); + ASSERT_EQ(state._spill_partitions.size(), 1); + + state.close(); + ASSERT_TRUE(state._spill_partitions.empty()); + + // Second close should be a safe no-op + state.close(); + ASSERT_TRUE(state._spill_partitions.empty()); +} + +TEST_F(PartitionedAggSharedStateTest, CloseWithMixedNullPartitions) { + PartitionedAggSharedState state; + state._spill_partitions.emplace_back(nullptr); + state._spill_partitions.emplace_back(nullptr); + state._spill_partitions.emplace_back(nullptr); + state._spill_partitions.emplace_back(nullptr); + state._spill_partitions.emplace_back(nullptr); + ASSERT_EQ(state._spill_partitions.size(), 5); + + state.close(); + ASSERT_TRUE(state._spill_partitions.empty()); +} + +TEST_F(PartitionedAggSharedStateTest, CloseClearsVector) { + PartitionedAggSharedState state; + // Add multiple null entries and verify size tracking + for (int i = 0; i < 10; ++i) { + state._spill_partitions.emplace_back(nullptr); + } + ASSERT_EQ(state._spill_partitions.size(), 10); + + state.close(); + ASSERT_EQ(state._spill_partitions.size(), 0); + ASSERT_TRUE(state._spill_partitions.empty()); +} + +TEST_F(PartitionedAggSharedStateTest, CloseCalledMultipleTimes) { + PartitionedAggSharedState state; + + for (int round = 0; round < 5; ++round) { + state._spill_partitions.emplace_back(nullptr); + state.close(); + ASSERT_TRUE(state._spill_partitions.empty()); + } +} + +// --- Tests covering PartitionedAggSourceOperatorX::revocable_mem_size logic --- +// revocable_mem_size checks: (1) _is_spilled, (2) spill_file != nullptr, +// then sums: block bytes + hash_table bytes + aggregate_data_container bytes. + +// Condition 1: _is_spilled defaults to false → early return 0. +TEST_F(PartitionedAggSharedStateTest, IsSpilledDefaultsFalse) { + PartitionedAggSharedState state; + EXPECT_FALSE(state._is_spilled.load()); +} + +TEST_F(PartitionedAggSharedStateTest, IsSpilledCanBeSet) { + PartitionedAggSharedState state; + state._is_spilled = true; + EXPECT_TRUE(state._is_spilled.load()); +} + +// Condition 2: _in_mem_shared_state defaults to null → hash table + container skipped. +TEST_F(PartitionedAggSharedStateTest, InMemSharedStateDefaultsNull) { + PartitionedAggSharedState state; + EXPECT_EQ(state._in_mem_shared_state, nullptr); +} + +// Hash table contribution: AggSharedState constructor always creates agg_data. +TEST_F(PartitionedAggSharedStateTest, AggSharedStateCreatesNonNullAggData) { + AggSharedState agg_state; + EXPECT_NE(agg_state.agg_data, nullptr); +} + +// Hash table contribution: default method_variant is monostate (index 0) → 0 bytes. +TEST_F(PartitionedAggSharedStateTest, AggSharedStateDefaultVariantIsMonostate) { + AggSharedState agg_state; + EXPECT_EQ(agg_state.agg_data->method_variant.index(), 0); +} + +// Container contribution: aggregate_data_container defaults to null → 0 bytes. +TEST_F(PartitionedAggSharedStateTest, AggSharedStateAggContainerDefaultsNull) { + AggSharedState agg_state; + EXPECT_EQ(agg_state.aggregate_data_container, nullptr); +} + +// Container contribution: freshly constructed container has 0 memory_usage. +TEST_F(PartitionedAggSharedStateTest, AggregateDataContainerInitialMemoryIsZero) { + AggregateDataContainer container(sizeof(uint32_t), 8); + EXPECT_EQ(container.memory_usage(), 0); +} + +// Container contribution: appending data allocates arena memory → memory_usage > 0. +TEST_F(PartitionedAggSharedStateTest, AggregateDataContainerMemoryGrowsAfterAppend) { + AggregateDataContainer container(sizeof(uint32_t), 8); + ASSERT_EQ(container.memory_usage(), 0); + uint32_t key = 42; + container.append_data(key); + EXPECT_GT(container.memory_usage(), 0); +} + +// Full state linkage: PartitionedAggSharedState holding an AggSharedState +// with monostate variant and null container → 0 bytes from both sources. +TEST_F(PartitionedAggSharedStateTest, PartitionedAggStateLinkedToAggStateWithDefaultData) { + AggSharedState agg_state; + PartitionedAggSharedState state; + state._in_mem_shared_state = &agg_state; + state._is_spilled = true; + + EXPECT_NE(state._in_mem_shared_state, nullptr); + EXPECT_NE(state._in_mem_shared_state->agg_data, nullptr); + // monostate → hash table contributes 0 bytes + EXPECT_EQ(state._in_mem_shared_state->agg_data->method_variant.index(), 0); + // null container → container contributes 0 bytes + EXPECT_EQ(state._in_mem_shared_state->aggregate_data_container, nullptr); +} + +// Container contribution through AggSharedState: memory_usage reflects arena allocation. +TEST_F(PartitionedAggSharedStateTest, AggSharedStateContainerMemoryUsage) { + AggSharedState agg_state; + agg_state.aggregate_data_container = + std::make_unique(sizeof(uint32_t), 8); + ASSERT_NE(agg_state.aggregate_data_container, nullptr); + EXPECT_EQ(agg_state.aggregate_data_container->memory_usage(), 0); + + uint32_t key = 99; + agg_state.aggregate_data_container->append_data(key); + EXPECT_GT(agg_state.aggregate_data_container->memory_usage(), 0); +} + +} // namespace doris \ No newline at end of file diff --git a/be/test/exec/pipeline/pipeline_task_test.cpp b/be/test/exec/pipeline/pipeline_task_test.cpp index 0d2b073bf948cd..7e6d95c2ba3077 100644 --- a/be/test/exec/pipeline/pipeline_task_test.cpp +++ b/be/test/exec/pipeline/pipeline_task_test.cpp @@ -20,11 +20,13 @@ #include "common/status.h" #include "exec/operator/operator.h" +#include "exec/operator/spill_utils.h" #include "exec/pipeline/dependency.h" #include "exec/pipeline/dummy_task_queue.h" #include "exec/pipeline/pipeline.h" #include "exec/pipeline/pipeline_fragment_context.h" #include "exec/pipeline/thrift_builder.h" +#include "exec/spill/spill_file.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" #include "testutil/mock/mock_runtime_state.h" @@ -884,9 +886,9 @@ TEST_F(PipelineTaskTest, TEST_RESERVE_MEMORY_FAIL) { } { task->_operators.front()->cast()._revocable_mem_size = - SpillStream::MIN_SPILL_WRITE_BATCH_MEM + 1; + SpillFile::MIN_SPILL_WRITE_BATCH_MEM + 1; task->_sink->cast()._revocable_mem_size = - SpillStream::MIN_SPILL_WRITE_BATCH_MEM + 1; + SpillFile::MIN_SPILL_WRITE_BATCH_MEM + 1; } { // Reserve failed and but not enable spill disk, so that the query will continue to run. @@ -1061,9 +1063,9 @@ TEST_F(PipelineTaskTest, TEST_RESERVE_MEMORY_FAIL_SPILLABLE) { } { task->_operators.front()->cast()._revocable_mem_size = - SpillStream::MIN_SPILL_WRITE_BATCH_MEM + 1; + SpillFile::MIN_SPILL_WRITE_BATCH_MEM + 1; task->_sink->cast()._revocable_mem_size = - SpillStream::MIN_SPILL_WRITE_BATCH_MEM + 1; + SpillFile::MIN_SPILL_WRITE_BATCH_MEM + 1; } { // Reserve failed and enable spill disk, so that the query be paused. @@ -1188,4 +1190,381 @@ TEST_F(PipelineTaskTest, TEST_INJECT_SHARED_STATE) { } } +TEST_F(PipelineTaskTest, TEST_SHOULD_TRIGGER_REVOKING) { + { + _query_options = TQueryOptionsBuilder() + .set_enable_local_exchange(true) + .set_enable_local_shuffle(true) + .set_runtime_filter_max_in_num(15) + .set_enable_reserve_memory(true) + .set_enable_spill(true) + .build(); + auto fe_address = TNetworkAddress(); + fe_address.hostname = LOCALHOST; + fe_address.port = DUMMY_PORT; + _query_ctx = + QueryContext::create(_query_id, ExecEnv::GetInstance(), _query_options, fe_address, + true, fe_address, QuerySource::INTERNAL_FRONTEND); + _task_scheduler = std::make_unique(); + _query_ctx->_task_scheduler = _task_scheduler.get(); + _build_fragment_context(); + } + TWorkloadGroupInfo twg_info; + twg_info.__set_id(0); + twg_info.__set_name("test_wg"); + twg_info.__set_version(0); + auto wg = std::make_shared(WorkloadGroupInfo::parse_topic_info(twg_info)); + const int64_t wg_mem_limit = 1000LL * 1024 * 1024; // 1 GB + wg->_memory_limit = wg_mem_limit; + wg->_memory_low_watermark = 50; // 50% + wg->_memory_high_watermark = 80; // 80% + wg->_total_mem_used = 0; + _query_ctx->_resource_ctx->set_workload_group(wg); + + auto query_mem_tracker = _query_ctx->query_mem_tracker(); + query_mem_tracker->set_limit(wg_mem_limit); + + auto num_instances = 1; + auto pip_id = 0; + auto task_id = 0; + auto pip = std::make_shared(pip_id, num_instances, num_instances); + { + OperatorPtr source_op; + source_op.reset(new DummyOperator()); + EXPECT_TRUE(pip->add_operator(source_op, num_instances).ok()); + + DataSinkOperatorPtr sink_op; + sink_op.reset(new DummySinkOperatorX(1, 2, 3)); + EXPECT_TRUE(pip->set_sink(sink_op).ok()); + } + auto profile = std::make_shared("Pipeline : " + std::to_string(pip_id)); + std::map, std::vector>>> + shared_state_map; + _runtime_state->resize_op_id_to_local_state(-1); + auto task = std::make_shared(pip, task_id, _runtime_state.get(), _context, + profile.get(), shared_state_map, task_id); + + // reserve_size that passes the (reserve * parallelism > query_limit / 5) gate + const size_t reserve_size = wg_mem_limit / 4; // 250MB > threshold of 200MB + + // Case 1: spill disabled -> false + { + ((MockRuntimeState*)_runtime_state.get())->set_enable_spill(false); + EXPECT_FALSE(task->_should_trigger_revoking(reserve_size)); + ((MockRuntimeState*)_runtime_state.get())->set_enable_spill(true); + } + // Case 2: no workload group -> false + { + _query_ctx->_resource_ctx->set_workload_group(nullptr); + EXPECT_FALSE(task->_should_trigger_revoking(reserve_size)); + _query_ctx->_resource_ctx->set_workload_group(wg); + } + // Case 3: effective query_limit = 0 (both tracker limit <= 0 and wg limit = 0) -> false + { + wg->_memory_limit = 0; + query_mem_tracker->set_limit(-1); + EXPECT_FALSE(task->_should_trigger_revoking(reserve_size)); + wg->_memory_limit = wg_mem_limit; + query_mem_tracker->set_limit(wg_mem_limit); + } + // Case 4: reserve_size too small (reserve * parallelism <= query_limit / 5) -> false + { EXPECT_FALSE(task->_should_trigger_revoking(wg_mem_limit / 5)); } + // Case 5: no memory pressure (neither query tracker nor wg watermark) -> false + { + // consumption + reserve = 100MB + 250MB = 350MB < 90% of 1GB (900MB); wg not at watermark + query_mem_tracker->consume(100LL * 1024 * 1024); + EXPECT_FALSE(task->_should_trigger_revoking(reserve_size)); + query_mem_tracker->release(100LL * 1024 * 1024); + } + // Case 6: high memory pressure via query tracker, no revocable memory -> false + { + // consumption + reserve >= 90% of query_limit + const int64_t consumption = int64_t(0.9 * wg_mem_limit) - int64_t(reserve_size) + 1; + query_mem_tracker->consume(consumption); + task->_operators.front()->cast()._revocable_mem_size = 0; + task->_sink->cast()._revocable_mem_size = 0; + EXPECT_FALSE(task->_should_trigger_revoking(reserve_size)); + query_mem_tracker->release(consumption); + } + // Case 7: high pressure via query tracker, sufficient revocable -> true + { + const int64_t consumption = int64_t(0.9 * wg_mem_limit) - int64_t(reserve_size) + 1; + query_mem_tracker->consume(consumption); + // total revocable >= 20% of query_limit = 200MB (100MB each from op and sink) + const size_t revocable = int64_t(0.2 * wg_mem_limit) / 2; + task->_operators.front()->cast()._revocable_mem_size = revocable; + task->_sink->cast()._revocable_mem_size = revocable; + EXPECT_TRUE(task->_should_trigger_revoking(reserve_size)); + query_mem_tracker->release(consumption); + task->_operators.front()->cast()._revocable_mem_size = 0; + task->_sink->cast()._revocable_mem_size = 0; + } + // Case 8: high pressure via wg low watermark, sufficient revocable -> true + { + // wg total_mem_used > 50% of wg_limit -> low watermark triggered + wg->_total_mem_used = int64_t(0.51 * wg_mem_limit); + const size_t revocable = int64_t(0.2 * wg_mem_limit) / 2; + task->_operators.front()->cast()._revocable_mem_size = revocable; + task->_sink->cast()._revocable_mem_size = revocable; + EXPECT_TRUE(task->_should_trigger_revoking(reserve_size)); + wg->_total_mem_used = 0; + task->_operators.front()->cast()._revocable_mem_size = 0; + task->_sink->cast()._revocable_mem_size = 0; + } + // Case 9: high pressure via wg high watermark, sufficient revocable -> true + { + // wg total_mem_used > 80% of wg_limit -> high watermark triggered + wg->_total_mem_used = int64_t(0.81 * wg_mem_limit); + const size_t revocable = int64_t(0.2 * wg_mem_limit) / 2; + task->_operators.front()->cast()._revocable_mem_size = revocable; + task->_sink->cast()._revocable_mem_size = revocable; + EXPECT_TRUE(task->_should_trigger_revoking(reserve_size)); + wg->_total_mem_used = 0; + task->_operators.front()->cast()._revocable_mem_size = 0; + task->_sink->cast()._revocable_mem_size = 0; + } + // Case 10: query_limit capped to wg limit when tracker limit > wg limit -> no extra pressure + { + // effective limit = wg_mem_limit; reserve = wg_mem_limit/4 > threshold (wg_mem_limit/5) + // but no consumption added, so no pressure -> false + query_mem_tracker->set_limit(wg_mem_limit * 2); + wg->_memory_limit = wg_mem_limit; + EXPECT_FALSE(task->_should_trigger_revoking(reserve_size)); + query_mem_tracker->set_limit(wg_mem_limit); + } +} + +TEST_F(PipelineTaskTest, TEST_DO_REVOKE_MEMORY) { + auto num_instances = 1; + auto pip_id = 0; + auto task_id = 0; + auto pip = std::make_shared(pip_id, num_instances, num_instances); + { + OperatorPtr source_op; + source_op.reset(new DummyOperator()); + EXPECT_TRUE(pip->add_operator(source_op, num_instances).ok()); + + DataSinkOperatorPtr sink_op; + sink_op.reset(new DummySinkOperatorX(1, 2, 3)); + EXPECT_TRUE(pip->set_sink(sink_op).ok()); + } + auto profile = std::make_shared("Pipeline : " + std::to_string(pip_id)); + std::map, std::vector>>> + shared_state_map; + _runtime_state->resize_op_id_to_local_state(-1); + auto task = std::make_shared(pip, task_id, _runtime_state.get(), _context, + profile.get(), shared_state_map, task_id); + { + std::vector scan_range; + int sender_id = 0; + TDataSink tsink; + EXPECT_TRUE(task->prepare(scan_range, sender_id, tsink).ok()); + _query_ctx->get_execution_dependency()->set_ready(); + } + // Case 1: fragment context expired -> InternalError + { + task->_fragment_context = std::weak_ptr(); + EXPECT_FALSE(task->do_revoke_memory(nullptr).ok()); + // Restore the fragment context + task->_fragment_context = _context; + } + // Case 2: operators below MIN threshold, null spill_context -> no revoke_memory called + { + task->_operators.front()->cast()._revocable_mem_size = 0; + task->_sink->cast()._revocable_mem_size = 0; + EXPECT_TRUE(task->do_revoke_memory(nullptr).ok()); + EXPECT_FALSE(task->_operators.front()->cast()._revoke_called); + EXPECT_FALSE(task->_sink->cast()._revoke_called); + } + // Case 3: operator has sufficient revocable memory -> operator revoke_memory called + { + task->_operators.front()->cast()._revocable_mem_size = + SpillFile::MIN_SPILL_WRITE_BATCH_MEM + 1; + task->_operators.front()->cast()._revoke_called = false; + task->_sink->cast()._revocable_mem_size = 0; + task->_sink->cast()._revoke_called = false; + EXPECT_TRUE(task->do_revoke_memory(nullptr).ok()); + EXPECT_TRUE(task->_operators.front()->cast()._revoke_called); + EXPECT_FALSE(task->_sink->cast()._revoke_called); + task->_operators.front()->cast()._revocable_mem_size = 0; + task->_operators.front()->cast()._revoke_called = false; + } + // Case 4: sink has sufficient revocable memory -> sink revoke_memory called + { + task->_operators.front()->cast()._revocable_mem_size = 0; + task->_operators.front()->cast()._revoke_called = false; + task->_sink->cast()._revocable_mem_size = + SpillFile::MIN_SPILL_WRITE_BATCH_MEM + 1; + task->_sink->cast()._revoke_called = false; + EXPECT_TRUE(task->do_revoke_memory(nullptr).ok()); + EXPECT_FALSE(task->_operators.front()->cast()._revoke_called); + EXPECT_TRUE(task->_sink->cast()._revoke_called); + task->_sink->cast()._revocable_mem_size = 0; + task->_sink->cast()._revoke_called = false; + } + // Case 5: non-null spill_context -> on_task_finished called, callback fires + { + bool callback_fired = false; + auto spill_ctx = std::make_shared( + 1, _query_id, [&callback_fired](SpillContext*) { callback_fired = true; }); + EXPECT_TRUE(task->do_revoke_memory(spill_ctx).ok()); + EXPECT_TRUE(callback_fired); + EXPECT_EQ(spill_ctx->running_tasks_count.load(), 0); + } + // Case 6: wake_up_early -> operators terminated, eos set, callback fires + { + task->_wake_up_early = true; + task->_eos = false; + task->_operators.front()->cast()._terminated = false; + task->_sink->cast()._terminated = false; + bool callback_fired = false; + auto spill_ctx = std::make_shared( + 1, _query_id, [&callback_fired](SpillContext*) { callback_fired = true; }); + EXPECT_TRUE(task->do_revoke_memory(spill_ctx).ok()); + EXPECT_TRUE(task->_eos); + EXPECT_TRUE(task->_operators.front()->cast()._terminated); + EXPECT_TRUE(task->_sink->cast()._terminated); + EXPECT_TRUE(callback_fired); + task->_wake_up_early = false; + } +} + +TEST_F(PipelineTaskTest, TEST_REVOKE_MEMORY) { + // Case 1: task is finalized -> on_task_finished called immediately + { + auto num_instances = 1; + auto pip_id = 0; + auto task_id = 0; + auto pip = std::make_shared(pip_id, num_instances, num_instances); + OperatorPtr source_op; + source_op.reset(new DummyOperator()); + EXPECT_TRUE(pip->add_operator(source_op, num_instances).ok()); + DataSinkOperatorPtr sink_op; + sink_op.reset(new DummySinkOperatorX(1, 2, 3)); + EXPECT_TRUE(pip->set_sink(sink_op).ok()); + + auto profile = std::make_shared("Pipeline : " + std::to_string(pip_id)); + std::map, + std::vector>>> + shared_state_map; + auto rs = std::make_unique(_query_id, 0, _query_options, + _query_ctx->query_globals, + ExecEnv::GetInstance(), _query_ctx.get()); + rs->set_task_execution_context(std::static_pointer_cast(_context)); + rs->resize_op_id_to_local_state(-1); + auto task = std::make_shared(pip, task_id, rs.get(), _context, profile.get(), + shared_state_map, task_id); + { + std::vector scan_range; + int sender_id = 0; + TDataSink tsink; + EXPECT_TRUE(task->prepare(scan_range, sender_id, tsink).ok()); + _query_ctx->get_execution_dependency()->set_ready(); + } + task->_exec_state = PipelineTask::State::FINALIZED; + EXPECT_TRUE(task->is_finalized()); + bool callback_fired = false; + auto spill_ctx = std::make_shared( + 1, _query_id, [&callback_fired](SpillContext*) { callback_fired = true; }); + EXPECT_TRUE(task->revoke_memory(spill_ctx).ok()); + EXPECT_TRUE(callback_fired); + EXPECT_EQ(spill_ctx->running_tasks_count.load(), 0); + } + // Case 2: _opened=true, revocable < MIN -> on_task_finished called immediately + { + auto num_instances = 1; + auto pip_id = 0; + auto task_id = 0; + auto pip = std::make_shared(pip_id, num_instances, num_instances); + OperatorPtr source_op; + source_op.reset(new DummyOperator()); + EXPECT_TRUE(pip->add_operator(source_op, num_instances).ok()); + DataSinkOperatorPtr sink_op; + sink_op.reset(new DummySinkOperatorX(1, 2, 3)); + EXPECT_TRUE(pip->set_sink(sink_op).ok()); + + auto profile = std::make_shared("Pipeline : " + std::to_string(pip_id)); + std::map, + std::vector>>> + shared_state_map; + auto rs = std::make_unique(_query_id, 0, _query_options, + _query_ctx->query_globals, + ExecEnv::GetInstance(), _query_ctx.get()); + rs->set_task_execution_context(std::static_pointer_cast(_context)); + rs->resize_op_id_to_local_state(-1); + auto task = std::make_shared(pip, task_id, rs.get(), _context, profile.get(), + shared_state_map, task_id); + { + std::vector scan_range; + int sender_id = 0; + TDataSink tsink; + EXPECT_TRUE(task->prepare(scan_range, sender_id, tsink).ok()); + _query_ctx->get_execution_dependency()->set_ready(); + } + task->_opened = true; + task->_operators.front()->cast()._revocable_mem_size = 0; + task->_sink->cast()._revocable_mem_size = 0; + bool callback_fired = false; + auto spill_ctx = std::make_shared( + 1, _query_id, [&callback_fired](SpillContext*) { callback_fired = true; }); + EXPECT_TRUE(task->revoke_memory(spill_ctx).ok()); + EXPECT_TRUE(callback_fired); + EXPECT_EQ(spill_ctx->running_tasks_count.load(), 0); + } + // Case 3: _opened=true, sufficient revocable -> RevokableTask submitted; run it to fire callback + { + auto num_instances = 1; + auto pip_id = 0; + auto task_id = 0; + auto pip = std::make_shared(pip_id, num_instances, num_instances); + OperatorPtr source_op; + source_op.reset(new DummyOperator()); + EXPECT_TRUE(pip->add_operator(source_op, num_instances).ok()); + DataSinkOperatorPtr sink_op; + sink_op.reset(new DummySinkOperatorX(1, 2, 3)); + EXPECT_TRUE(pip->set_sink(sink_op).ok()); + + auto profile = std::make_shared("Pipeline : " + std::to_string(pip_id)); + std::map, + std::vector>>> + shared_state_map; + auto rs = std::make_unique(_query_id, 0, _query_options, + _query_ctx->query_globals, + ExecEnv::GetInstance(), _query_ctx.get()); + rs->set_task_execution_context(std::static_pointer_cast(_context)); + rs->resize_op_id_to_local_state(-1); + auto task = std::make_shared(pip, task_id, rs.get(), _context, profile.get(), + shared_state_map, task_id); + { + std::vector scan_range; + int sender_id = 0; + TDataSink tsink; + EXPECT_TRUE(task->prepare(scan_range, sender_id, tsink).ok()); + _query_ctx->get_execution_dependency()->set_ready(); + } + task->_opened = true; + task->_operators.front()->cast()._revocable_mem_size = + SpillFile::MIN_SPILL_WRITE_BATCH_MEM + 1; + bool callback_fired = false; + auto spill_ctx = std::make_shared( + 1, _query_id, [&callback_fired](SpillContext*) { callback_fired = true; }); + EXPECT_TRUE(task->revoke_memory(spill_ctx).ok()); + // RevokableTask submitted but not yet executed, callback not fired + EXPECT_FALSE(callback_fired); + EXPECT_EQ(spill_ctx->running_tasks_count.load(), 1); + + // Take the submitted RevokableTask from the scheduler queue and run it + auto revokable_task = + ((MockTaskScheduler*)_query_ctx->_task_scheduler)->_task_queue->take(0); + EXPECT_NE(revokable_task, nullptr); + bool done = false; + EXPECT_TRUE(revokable_task->execute(&done).ok()); + // After execution, spill_context->on_task_finished() was called inside do_revoke_memory + EXPECT_TRUE(callback_fired); + EXPECT_EQ(spill_ctx->running_tasks_count.load(), 0); + } +} + } // namespace doris diff --git a/be/test/runtime/workload_group/workload_group_manager_test.cpp b/be/test/runtime/workload_group/workload_group_manager_test.cpp index 709b5f3940438c..d0fa10ae0b8a43 100644 --- a/be/test/runtime/workload_group/workload_group_manager_test.cpp +++ b/be/test/runtime/workload_group/workload_group_manager_test.cpp @@ -30,7 +30,7 @@ #include "common/config.h" #include "common/status.h" #include "exec/pipeline/pipeline_tracing.h" -#include "exec/spill/spill_stream_manager.h" +#include "exec/spill/spill_file_manager.h" #include "runtime/exec_env.h" #include "runtime/query_context.h" #include "runtime/runtime_query_statistics_mgr.h" @@ -63,9 +63,8 @@ class WorkloadGroupManagerTest : public testing::Test { } ExecEnv::GetInstance()->_runtime_query_statistics_mgr = new RuntimeQueryStatisticsMgr(); - ExecEnv::GetInstance()->_spill_stream_mgr = - new SpillStreamManager(std::move(spill_store_map)); - auto st = ExecEnv::GetInstance()->_spill_stream_mgr->init(); + ExecEnv::GetInstance()->_spill_file_mgr = new SpillFileManager(std::move(spill_store_map)); + auto st = ExecEnv::GetInstance()->_spill_file_mgr->init(); EXPECT_TRUE(st.ok()) << "init spill stream manager failed: " << st.to_string(); ExecEnv::GetInstance()->_pipeline_tracer_ctx = std::make_unique(); diff --git a/be/test/vec/spill/spill_file_test.cpp b/be/test/vec/spill/spill_file_test.cpp new file mode 100644 index 00000000000000..67740acdcd2e97 --- /dev/null +++ b/be/test/vec/spill/spill_file_test.cpp @@ -0,0 +1,1090 @@ +// 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 "exec/spill/spill_file.h" + +#include + +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "core/block/block.h" +#include "core/data_type/data_type_number.h" +#include "core/data_type/data_type_string.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" +#include "io/fs/local_file_system.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_profile.h" +#include "testutil/column_helper.h" +#include "testutil/mock/mock_runtime_state.h" + +namespace doris::vectorized { + +class SpillFileTest : public testing::Test { +protected: + void SetUp() override { + _runtime_state = std::make_unique(); + + _profile = std::make_unique("test"); + _custom_profile = std::make_unique("CustomCounters"); + _common_profile = std::make_unique("CommonCounters"); + + _common_profile->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 1); + ADD_TIMER_WITH_LEVEL(_common_profile.get(), "ExecTime", 1); + + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillTotalTime", 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillWriteTime", 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteTaskWaitInQueueCount", TUnit::UNIT, + 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteTaskCount", TUnit::UNIT, 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillWriteTaskWaitInQueueTime", 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileTime", 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillWriteSerializeBlockTime", 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteBlockCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteBlockBytes", TUnit::BYTES, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileBytes", TUnit::BYTES, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteRows", TUnit::UNIT, 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillReadFileTime", 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillReadDerializeBlockTime", 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadBlockCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadBlockBytes", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadFileBytes", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadRows", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadFileCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileTotalCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileCurrentCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileCurrentBytes", TUnit::UNIT, 1); + + _profile->add_child(_custom_profile.get(), true); + _profile->add_child(_common_profile.get(), true); + + _spill_dir = "./ut_dir/spill_file_test"; + auto spill_data_dir = std::make_unique(_spill_dir, 1024L * 1024 * 128); + auto st = io::global_local_filesystem()->create_directory(spill_data_dir->path(), false); + ASSERT_TRUE(st.ok()) << "create directory failed: " << st.to_string(); + + std::unordered_map> data_map; + _data_dir_ptr = spill_data_dir.get(); + data_map.emplace("test", std::move(spill_data_dir)); + auto* spill_file_manager = new SpillFileManager(std::move(data_map)); + ExecEnv::GetInstance()->_spill_file_mgr = spill_file_manager; + st = spill_file_manager->init(); + ASSERT_TRUE(st.ok()) << "init spill file manager failed: " << st.to_string(); + } + + void TearDown() override { + ExecEnv::GetInstance()->spill_file_mgr()->stop(); + SAFE_DELETE(ExecEnv::GetInstance()->_spill_file_mgr); + // Clean up test directory + auto st = io::global_local_filesystem()->delete_directory(_spill_dir); + (void)st; + _runtime_state.reset(); + } + + Block _create_int_block(const std::vector& data) { + return ColumnHelper::create_block(data); + } + + Block _create_two_column_block(const std::vector& col1, + const std::vector& col2) { + auto block = ColumnHelper::create_block(col1); + block.insert(ColumnHelper::create_column_with_name(col2)); + return block; + } + + std::unique_ptr _runtime_state; + std::unique_ptr _profile; + std::unique_ptr _custom_profile; + std::unique_ptr _common_profile; + std::string _spill_dir; + SpillDataDir* _data_dir_ptr = nullptr; +}; + +// ═══════════════════════════════════════════════════════════════════════ +// SpillFile basic tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, CreateSpillFile) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/test_file", + spill_file); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_TRUE(spill_file != nullptr); + ASSERT_FALSE(spill_file->ready_for_reading()); +} + +TEST_F(SpillFileTest, CreateWriterAndReader) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/create_wr", + spill_file); + ASSERT_TRUE(st.ok()) << st.to_string(); + + // Create writer + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_TRUE(writer != nullptr); + + // Close writer with no data written + st = writer->close(); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_TRUE(spill_file->ready_for_reading()); + + // Create reader on empty file (0 parts) + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + ASSERT_TRUE(reader != nullptr); + + st = reader->open(); + ASSERT_TRUE(st.ok()) << st.to_string(); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_TRUE(eos); + + st = reader->close(); + ASSERT_TRUE(st.ok()) << st.to_string(); +} + +// ═══════════════════════════════════════════════════════════════════════ +// SpillFileWriter tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, WriteSingleBlock) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/single_block", + spill_file); + ASSERT_TRUE(st.ok()); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({1, 2, 3, 4, 5}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()) << st.to_string(); + + st = writer->close(); + ASSERT_TRUE(st.ok()) << st.to_string(); + + ASSERT_TRUE(spill_file->ready_for_reading()); + + auto* write_rows_counter = _custom_profile->get_counter("SpillWriteRows"); + ASSERT_TRUE(write_rows_counter != nullptr); + ASSERT_EQ(write_rows_counter->value(), 5); +} + +TEST_F(SpillFileTest, WriteMultipleBlocks) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/multi_blocks", + spill_file); + ASSERT_TRUE(st.ok()); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + for (int i = 0; i < 5; ++i) { + auto block = _create_int_block({i * 10, i * 10 + 1, i * 10 + 2}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()) << "write block " << i << " failed: " << st.to_string(); + } + + st = writer->close(); + ASSERT_TRUE(st.ok()) << st.to_string(); + + auto* write_rows_counter = _custom_profile->get_counter("SpillWriteRows"); + ASSERT_EQ(write_rows_counter->value(), 15); + + auto* write_block_counter = _custom_profile->get_counter("SpillWriteBlockCount"); + ASSERT_EQ(write_block_counter->value(), 5); +} + +TEST_F(SpillFileTest, WriteTwoColumnBlock) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/two_col", + spill_file); + ASSERT_TRUE(st.ok()); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_two_column_block({1, 2, 3}, {100, 200, 300}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()) << st.to_string(); + + st = writer->close(); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_TRUE(spill_file->ready_for_reading()); +} + +TEST_F(SpillFileTest, WriteEmptyBlock) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/empty_block", + spill_file); + ASSERT_TRUE(st.ok()); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + Block empty_block; + st = writer->write_block(_runtime_state.get(), empty_block); + ASSERT_TRUE(st.ok()) << st.to_string(); + + st = writer->close(); + ASSERT_TRUE(st.ok()) << st.to_string(); +} + +TEST_F(SpillFileTest, DoubleCloseWriter) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/double_close", + spill_file); + ASSERT_TRUE(st.ok()); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({1, 2, 3}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + + // Double close should be a no-op + st = writer->close(); + ASSERT_TRUE(st.ok()); +} + +// ═══════════════════════════════════════════════════════════════════════ +// SpillFileReader tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, ReadSingleBlock) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/read_single", + spill_file); + ASSERT_TRUE(st.ok()); + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({10, 20, 30, 40, 50}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()) << st.to_string(); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_FALSE(eos); + ASSERT_EQ(block.rows(), 5); + + // Verify data + auto col = block.get_by_position(0).column; + ASSERT_EQ(col->get_int(0), 10); + ASSERT_EQ(col->get_int(1), 20); + ASSERT_EQ(col->get_int(2), 30); + ASSERT_EQ(col->get_int(3), 40); + ASSERT_EQ(col->get_int(4), 50); + + // Next read should be EOS + Block block2; + st = reader->read(&block2, &eos); + ASSERT_TRUE(st.ok()); + ASSERT_TRUE(eos); + + st = reader->close(); + ASSERT_TRUE(st.ok()); +} + +TEST_F(SpillFileTest, OpenCanRetryAfterFailure) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/open_retry", + spill_file); + ASSERT_TRUE(st.ok()); + + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({7, 8, 9}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + const auto part_path = + std::filesystem::path(_spill_dir) / "spill" / "test_query" / "open_retry" / "0"; + const auto backup_path = + std::filesystem::path(_spill_dir) / "spill" / "test_query" / "open_retry" / "0.bak"; + + std::filesystem::rename(part_path, backup_path); + + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_FALSE(st.ok()); + + std::filesystem::rename(backup_path, part_path); + + st = reader->open(); + ASSERT_TRUE(st.ok()) << st.to_string(); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_FALSE(eos); + ASSERT_EQ(block.rows(), 3); + + auto col = block.get_by_position(0).column; + ASSERT_EQ(col->get_int(0), 7); + ASSERT_EQ(col->get_int(1), 8); + ASSERT_EQ(col->get_int(2), 9); +} + +TEST_F(SpillFileTest, ReadMultipleBlocks) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/read_multi", + spill_file); + ASSERT_TRUE(st.ok()); + + const int num_blocks = 10; + const int rows_per_block = 100; + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + for (int b = 0; b < num_blocks; ++b) { + std::vector data(rows_per_block); + std::iota(data.begin(), data.end(), b * rows_per_block); + auto block = _create_int_block(data); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + } + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read all blocks + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + size_t total_rows = 0; + int block_count = 0; + bool eos = false; + while (!eos) { + Block block; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()) << st.to_string(); + if (!eos) { + total_rows += block.rows(); + ++block_count; + } + } + + ASSERT_EQ(total_rows, num_blocks * rows_per_block); + ASSERT_EQ(block_count, num_blocks); + + st = reader->close(); + ASSERT_TRUE(st.ok()); +} + +TEST_F(SpillFileTest, ReadTwoColumnBlock) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/read_two_col", + spill_file); + ASSERT_TRUE(st.ok()); + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_two_column_block({1, 2, 3, 4}, {100, 200, 300, 400}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + ASSERT_FALSE(eos); + ASSERT_EQ(block.rows(), 4); + ASSERT_EQ(block.columns(), 2); + + // Verify col1 + auto col1 = block.get_by_position(0).column; + ASSERT_EQ(col1->get_int(0), 1); + ASSERT_EQ(col1->get_int(3), 4); + + // Verify col2 + auto col2 = block.get_by_position(1).column; + ASSERT_EQ(col2->get_int(0), 100); + ASSERT_EQ(col2->get_int(3), 400); + + st = reader->close(); + ASSERT_TRUE(st.ok()); +} + +// ═══════════════════════════════════════════════════════════════════════ +// Roundtrip tests (write -> read -> verify) +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, RoundtripSingleBlock) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + "test_query/roundtrip_single", spill_file); + ASSERT_TRUE(st.ok()); + + std::vector original_data = {42, 7, 99, 1, 0, -5, 1000}; + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block(original_data); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read & verify + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(block.rows(), original_data.size()); + + auto col = block.get_by_position(0).column; + for (size_t i = 0; i < original_data.size(); ++i) { + ASSERT_EQ(col->get_int(i), original_data[i]) << "mismatch at index " << i; + } + + st = reader->close(); + ASSERT_TRUE(st.ok()); +} + +TEST_F(SpillFileTest, RoundtripMultipleBlocks) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + "test_query/roundtrip_multi", spill_file); + ASSERT_TRUE(st.ok()); + + std::vector> all_data = { + {1, 2, 3}, + {10, 20, 30, 40}, + {100, 200}, + {-1, -2, -3, -4, -5}, + }; + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + for (const auto& data : all_data) { + auto block = _create_int_block(data); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + } + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read & verify + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + size_t block_idx = 0; + bool eos = false; + while (!eos && block_idx < all_data.size()) { + Block block; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + if (eos) break; + + ASSERT_EQ(block.rows(), all_data[block_idx].size()) + << "block " << block_idx << " row count mismatch"; + + auto col = block.get_by_position(0).column; + for (size_t i = 0; i < all_data[block_idx].size(); ++i) { + ASSERT_EQ(col->get_int(i), all_data[block_idx][i]) + << "mismatch at block " << block_idx << " row " << i; + } + ++block_idx; + } + + ASSERT_EQ(block_idx, all_data.size()); + + st = reader->close(); + ASSERT_TRUE(st.ok()); +} + +TEST_F(SpillFileTest, RoundtripLargeData) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + "test_query/roundtrip_large", spill_file); + ASSERT_TRUE(st.ok()); + + const size_t row_count = 100000; + std::vector data(row_count); + std::iota(data.begin(), data.end(), 0); + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block(data); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read & verify + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(block.rows(), row_count); + + auto col = block.get_by_position(0).column; + for (size_t i = 0; i < row_count; i += 1000) { + ASSERT_EQ(col->get_int(i), (int32_t)i) << "mismatch at index " << i; + } + + st = reader->close(); + ASSERT_TRUE(st.ok()); +} + +// ═══════════════════════════════════════════════════════════════════════ +// Part rotation tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, PartRotation) { + // Set a very small part size to force rotation + auto saved_part_size = config::spill_file_part_size_bytes; + config::spill_file_part_size_bytes = 1024; // 1KB per part + + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/rotation", + spill_file); + ASSERT_TRUE(st.ok()); + + const int num_blocks = 20; + + // Write many blocks to trigger multiple part rotations + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + for (int i = 0; i < num_blocks; ++i) { + std::vector data(100); + std::iota(data.begin(), data.end(), i * 100); + auto block = _create_int_block(data); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + } + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read back and verify all data across multiple parts + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + size_t total_rows = 0; + int block_count = 0; + bool eos = false; + while (!eos) { + Block block; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + if (!eos) { + total_rows += block.rows(); + ++block_count; + } + } + + ASSERT_EQ(total_rows, num_blocks * 100); + ASSERT_EQ(block_count, num_blocks); + + st = reader->close(); + ASSERT_TRUE(st.ok()); + + config::spill_file_part_size_bytes = saved_part_size; +} + +TEST_F(SpillFileTest, PartRotationDataIntegrity) { + // Set a small part size to force rotation + auto saved_part_size = config::spill_file_part_size_bytes; + config::spill_file_part_size_bytes = 512; + + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + "test_query/rotation_integrity", spill_file); + ASSERT_TRUE(st.ok()); + + std::vector> all_data; + const int num_blocks = 30; + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + for (int i = 0; i < num_blocks; ++i) { + std::vector data(50); + std::iota(data.begin(), data.end(), i * 1000); + all_data.push_back(data); + auto block = _create_int_block(data); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + } + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read & verify data integrity across parts + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + size_t block_idx = 0; + bool eos = false; + while (!eos) { + Block block; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + if (eos) break; + + ASSERT_LT(block_idx, all_data.size()); + ASSERT_EQ(block.rows(), all_data[block_idx].size()); + + auto col = block.get_by_position(0).column; + for (size_t i = 0; i < all_data[block_idx].size(); ++i) { + ASSERT_EQ(col->get_int(i), all_data[block_idx][i]) + << "data mismatch at block " << block_idx << " row " << i; + } + ++block_idx; + } + + ASSERT_EQ(block_idx, all_data.size()); + + st = reader->close(); + ASSERT_TRUE(st.ok()); + + config::spill_file_part_size_bytes = saved_part_size; +} + +// ═══════════════════════════════════════════════════════════════════════ +// Seek tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, SeekToBlock) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/seek", + spill_file); + ASSERT_TRUE(st.ok()); + + const int num_blocks = 5; + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + for (int i = 0; i < num_blocks; ++i) { + auto block = _create_int_block({i * 10, i * 10 + 1, i * 10 + 2}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + } + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Seek to block 2 (0-based) and read + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + st = reader->seek(2); + ASSERT_TRUE(st.ok()) << st.to_string(); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + ASSERT_FALSE(eos); + ASSERT_EQ(block.rows(), 3); + + auto col = block.get_by_position(0).column; + ASSERT_EQ(col->get_int(0), 20); + ASSERT_EQ(col->get_int(1), 21); + ASSERT_EQ(col->get_int(2), 22); + + st = reader->close(); + ASSERT_TRUE(st.ok()); +} + +TEST_F(SpillFileTest, SeekBeyondEnd) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/seek_beyond", + spill_file); + ASSERT_TRUE(st.ok()); + + // Write 3 blocks + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + for (int i = 0; i < 3; ++i) { + auto block = _create_int_block({i}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + } + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + // Seek beyond the end + st = reader->seek(100); + ASSERT_TRUE(st.ok()) << st.to_string(); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + ASSERT_TRUE(eos); + + st = reader->close(); + ASSERT_TRUE(st.ok()); +} + +// ═══════════════════════════════════════════════════════════════════════ +// SpillFile GC/lifecycle tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, GCCleansUpFiles) { + std::string spill_file_dir; + + { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/gc_test", + spill_file); + ASSERT_TRUE(st.ok()); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({1, 2, 3}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + + // Remember the spill directory path + spill_file_dir = _data_dir_ptr->get_spill_data_path() + "/test_query/gc_test"; + + // Verify directory exists + bool exists = false; + st = io::global_local_filesystem()->exists(spill_file_dir, &exists); + ASSERT_TRUE(st.ok()); + ASSERT_TRUE(exists); + + // spill_file goes out of scope here, destructor calls gc() + } + + // After SpillFile is destroyed, the directory should be cleaned up + bool exists = false; + auto st = io::global_local_filesystem()->exists(spill_file_dir, &exists); + ASSERT_TRUE(st.ok()); + ASSERT_FALSE(exists); +} + +TEST_F(SpillFileTest, DeleteSpillFileThroughManager) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/mgr_delete", + spill_file); + ASSERT_TRUE(st.ok()); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({1, 2, 3}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + + // Delete through manager (async GC) + ExecEnv::GetInstance()->spill_file_mgr()->delete_spill_file(spill_file); + + // Run GC to process the deletion + ExecEnv::GetInstance()->spill_file_mgr()->gc(1000); +} + +// ═══════════════════════════════════════════════════════════════════════ +// SpillFileManager tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, ManagerNextId) { + auto id1 = ExecEnv::GetInstance()->spill_file_mgr()->next_id(); + auto id2 = ExecEnv::GetInstance()->spill_file_mgr()->next_id(); + auto id3 = ExecEnv::GetInstance()->spill_file_mgr()->next_id(); + + ASSERT_EQ(id2, id1 + 1); + ASSERT_EQ(id3, id2 + 1); +} + +TEST_F(SpillFileTest, ManagerCreateMultipleFiles) { + const int num_files = 5; + std::vector files; + + for (int i = 0; i < num_files; ++i) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + fmt::format("test_query/multi_{}", i), spill_file); + ASSERT_TRUE(st.ok()) << "create file " << i << " failed: " << st.to_string(); + files.push_back(spill_file); + } + + // Write and close each file + for (int i = 0; i < num_files; ++i) { + SpillFileWriterSPtr writer; + auto st = files[i]->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({i * 100}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read each file and verify + for (int i = 0; i < num_files; ++i) { + auto reader = files[i]->create_reader(_runtime_state.get(), _profile.get()); + auto st = reader->open(); + ASSERT_TRUE(st.ok()); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + ASSERT_EQ(block.rows(), 1); + + auto col = block.get_by_position(0).column; + ASSERT_EQ(col->get_int(0), i * 100); + + st = reader->close(); + ASSERT_TRUE(st.ok()); + } +} + +// ═══════════════════════════════════════════════════════════════════════ +// Profile counter tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, WriteCounters) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/counters", + spill_file); + ASSERT_TRUE(st.ok()); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({1, 2, 3, 4, 5}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + auto block2 = _create_int_block({10, 20, 30}); + st = writer->write_block(_runtime_state.get(), block2); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + + auto* write_rows = _custom_profile->get_counter("SpillWriteRows"); + ASSERT_TRUE(write_rows != nullptr); + ASSERT_EQ(write_rows->value(), 8); + + auto* write_blocks = _custom_profile->get_counter("SpillWriteBlockCount"); + ASSERT_TRUE(write_blocks != nullptr); + ASSERT_EQ(write_blocks->value(), 2); + + auto* write_bytes = _custom_profile->get_counter("SpillWriteFileBytes"); + ASSERT_TRUE(write_bytes != nullptr); + ASSERT_GT(write_bytes->value(), 0); +} + +TEST_F(SpillFileTest, ReadCounters) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file( + "test_query/read_counters", spill_file); + ASSERT_TRUE(st.ok()); + + // Write + { + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + auto block = _create_int_block({1, 2, 3, 4, 5}); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + } + + // Read + auto reader = spill_file->create_reader(_runtime_state.get(), _profile.get()); + st = reader->open(); + ASSERT_TRUE(st.ok()); + + Block block; + bool eos = false; + st = reader->read(&block, &eos); + ASSERT_TRUE(st.ok()); + + st = reader->close(); + ASSERT_TRUE(st.ok()); + + auto* read_blocks = _custom_profile->get_counter("SpillReadBlockCount"); + ASSERT_TRUE(read_blocks != nullptr); + ASSERT_EQ(read_blocks->value(), 1); + + auto* read_rows = _custom_profile->get_counter("SpillReadRows"); + ASSERT_TRUE(read_rows != nullptr); + ASSERT_EQ(read_rows->value(), 5); + + auto* read_file_size = _custom_profile->get_counter("SpillReadFileBytes"); + ASSERT_TRUE(read_file_size != nullptr); + ASSERT_GT(read_file_size->value(), 0); +} + +// ═══════════════════════════════════════════════════════════════════════ +// SpillDataDir tests +// ═══════════════════════════════════════════════════════════════════════ + +TEST_F(SpillFileTest, DataDirCapacityTracking) { + SpillFileSPtr spill_file; + auto st = ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file("test_query/capacity", + spill_file); + ASSERT_TRUE(st.ok()); + + auto initial_bytes = _data_dir_ptr->get_spill_data_bytes(); + + SpillFileWriterSPtr writer; + st = spill_file->create_writer(_runtime_state.get(), _profile.get(), writer); + ASSERT_TRUE(st.ok()); + + // Write a block to increase usage + std::vector data(1000); + std::iota(data.begin(), data.end(), 0); + auto block = _create_int_block(data); + st = writer->write_block(_runtime_state.get(), block); + ASSERT_TRUE(st.ok()); + + st = writer->close(); + ASSERT_TRUE(st.ok()); + + auto after_write_bytes = _data_dir_ptr->get_spill_data_bytes(); + ASSERT_GT(after_write_bytes, initial_bytes); +} + +} // namespace doris::vectorized diff --git a/be/test/vec/spill/spill_repartitioner_test.cpp b/be/test/vec/spill/spill_repartitioner_test.cpp new file mode 100644 index 00000000000000..53ffcf9f0ac00c --- /dev/null +++ b/be/test/vec/spill/spill_repartitioner_test.cpp @@ -0,0 +1,410 @@ +// 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 "exec/spill/spill_repartitioner.h" + +#include + +#include +#include +#include + +#include "core/block/block.h" +#include "core/data_type/data_type_number.h" +#include "exec/partitioner/partitioner.h" +#include "exec/spill/spill_file.h" +#include "exec/spill/spill_file_manager.h" +#include "exec/spill/spill_file_reader.h" +#include "exec/spill/spill_file_writer.h" +#include "io/fs/local_file_system.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_profile.h" +#include "testutil/column_helper.h" +#include "testutil/mock/mock_runtime_state.h" + +namespace doris { + +class SpillRepartitionerTest : public testing::Test { +protected: + void SetUp() override { + _runtime_state = std::make_unique(); + + // Profile hierarchy required by SpillFileWriter / SpillFileReader: + // _profile (operator) + // ├── CustomCounters (write/read timing and byte counters) + // └── CommonCounters (MemoryUsage high-water-mark) + _profile = std::make_unique("operator"); + _custom_profile = std::make_unique("CustomCounters"); + _common_profile = std::make_unique("CommonCounters"); + + _common_profile->AddHighWaterMarkCounter("MemoryUsage", TUnit::BYTES, "", 1); + ADD_TIMER_WITH_LEVEL(_common_profile.get(), "ExecTime", 1); + + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillTotalTime", 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillWriteTime", 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteTaskWaitInQueueCount", TUnit::UNIT, + 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteTaskCount", TUnit::UNIT, 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillWriteTaskWaitInQueueTime", 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileTime", 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillWriteSerializeBlockTime", 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteBlockCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteBlockBytes", TUnit::BYTES, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileBytes", TUnit::BYTES, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteRows", TUnit::UNIT, 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillReadFileTime", 1); + ADD_TIMER_WITH_LEVEL(_custom_profile.get(), "SpillReadDerializeBlockTime", 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadBlockCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadBlockBytes", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadFileBytes", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadRows", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillReadFileCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileTotalCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileCurrentCount", TUnit::UNIT, 1); + ADD_COUNTER_WITH_LEVEL(_custom_profile.get(), "SpillWriteFileCurrentBytes", TUnit::UNIT, 1); + + _profile->add_child(_custom_profile.get(), true); + _profile->add_child(_common_profile.get(), true); + + _spill_dir = "./ut_dir/spill_repartitioner_test"; + auto spill_data_dir = std::make_unique(_spill_dir, 256L * 1024 * 1024); + auto st = io::global_local_filesystem()->create_directory(spill_data_dir->path(), false); + ASSERT_TRUE(st.ok()) << st.to_string(); + + std::unordered_map> data_map; + data_map.emplace("test", std::move(spill_data_dir)); + auto* mgr = new SpillFileManager(std::move(data_map)); + ExecEnv::GetInstance()->_spill_file_mgr = mgr; + st = mgr->init(); + ASSERT_TRUE(st.ok()) << st.to_string(); + } + + void TearDown() override { + ExecEnv::GetInstance()->spill_file_mgr()->stop(); + SAFE_DELETE(ExecEnv::GetInstance()->_spill_file_mgr); + auto st = io::global_local_filesystem()->delete_directory(_spill_dir); + (void)st; + _runtime_state.reset(); + } + + // Write int32 values to a new SpillFile and return it (ready for reading). + SpillFileSPtr _make_input_file(const std::vector& data, const std::string& suffix) { + SpillFileSPtr f; + EXPECT_TRUE(ExecEnv::GetInstance()->spill_file_mgr()->create_spill_file(suffix, f).ok()); + SpillFileWriterSPtr w; + EXPECT_TRUE(f->create_writer(_runtime_state.get(), _profile.get(), w).ok()); + auto block = ColumnHelper::create_block(data); + EXPECT_TRUE(w->write_block(_runtime_state.get(), block).ok()); + EXPECT_TRUE(w->close().ok()); + return f; + } + + // Sum rows across all blocks in a spill file. + int64_t _count_rows(SpillFileSPtr& f) { + auto reader = f->create_reader(_runtime_state.get(), _profile.get()); + EXPECT_TRUE(reader->open().ok()); + int64_t total = 0; + bool eos = false; + while (!eos) { + Block b; + EXPECT_TRUE(reader->read(&b, &eos).ok()); + total += static_cast(b.rows()); + } + EXPECT_TRUE(reader->close().ok()); + return total; + } + + // Init repartitioner in column-index mode on a single int32 key at column 0. + void _init_col_mode(SpillRepartitioner& r, int fanout, int level = 0) { + r.init_with_key_columns({0}, {std::make_shared()}, _profile.get(), fanout, + level); + } + + std::unique_ptr _runtime_state; + std::unique_ptr _profile; + std::unique_ptr _custom_profile; + std::unique_ptr _common_profile; + std::string _spill_dir; +}; + +// ── create_output_spill_files ──────────────────────────────────────────────── + +TEST_F(SpillRepartitionerTest, CreateOutputSpillFilesProducesCorrectFanout) { + const int fanout = 6; + std::vector output_files; + auto st = SpillRepartitioner::create_output_spill_files(_runtime_state.get(), /*node_id=*/1, + "test/create", fanout, output_files); + ASSERT_TRUE(st.ok()) << st.to_string(); + ASSERT_EQ(static_cast(output_files.size()), fanout); + for (auto& f : output_files) { + ASSERT_TRUE(f != nullptr); + } +} + +// ── fanout() accessor ──────────────────────────────────────────────────────── + +TEST_F(SpillRepartitionerTest, FanoutAccessorReflectsInitValue) { + SpillRepartitioner r; + _init_col_mode(r, /*fanout=*/5); + ASSERT_EQ(r.fanout(), 5); +} + +// ── setup_output + finalize ────────────────────────────────────────────────── + +TEST_F(SpillRepartitionerTest, SetupOutputAndFinalizeSucceeds) { + const int fanout = 4; + SpillRepartitioner r; + _init_col_mode(r, fanout); + + std::vector output_files; + ASSERT_TRUE(SpillRepartitioner::create_output_spill_files(_runtime_state.get(), 0, "test/setup", + fanout, output_files) + .ok()); + ASSERT_TRUE(r.setup_output(_runtime_state.get(), output_files).ok()); + ASSERT_TRUE(r.finalize().ok()); + + // After finalize() all output writers are closed → files are ready to read. + for (auto& f : output_files) { + ASSERT_TRUE(f->ready_for_reading()); + } +} + +// ── route_block ────────────────────────────────────────────────────────────── + +TEST_F(SpillRepartitionerTest, RouteBlockConservesTotalRowCount) { + const int fanout = 4; + const int input_rows = 100; + + SpillRepartitioner r; + _init_col_mode(r, fanout); + + std::vector output_files; + ASSERT_TRUE(SpillRepartitioner::create_output_spill_files( + _runtime_state.get(), 0, "test/route_block", fanout, output_files) + .ok()); + ASSERT_TRUE(r.setup_output(_runtime_state.get(), output_files).ok()); + + // Build a block with values 0..input_rows-1 + std::vector vals(input_rows); + std::iota(vals.begin(), vals.end(), 0); + auto block = ColumnHelper::create_block(vals); + + ASSERT_TRUE(r.route_block(_runtime_state.get(), block).ok()); + ASSERT_TRUE(r.finalize().ok()); + + // All rows must appear in exactly one output partition. + int64_t total = 0; + for (auto& f : output_files) { + total += _count_rows(f); + } + ASSERT_EQ(total, input_rows); +} + +TEST_F(SpillRepartitionerTest, RouteEmptyBlockIsNoop) { + const int fanout = 4; + SpillRepartitioner r; + _init_col_mode(r, fanout); + + std::vector output_files; + ASSERT_TRUE(SpillRepartitioner::create_output_spill_files( + _runtime_state.get(), 0, "test/route_empty", fanout, output_files) + .ok()); + ASSERT_TRUE(r.setup_output(_runtime_state.get(), output_files).ok()); + + Block empty; + ASSERT_TRUE(r.route_block(_runtime_state.get(), empty).ok()); + ASSERT_TRUE(r.finalize().ok()); + + int64_t total = 0; + for (auto& f : output_files) { + total += _count_rows(f); + } + ASSERT_EQ(total, 0); +} + +// ── repartition(SpillFile) ─────────────────────────────────────────────────── + +TEST_F(SpillRepartitionerTest, RepartitionFromSpillFileConservesRows) { + const int fanout = 4; + const int input_rows = 80; + + std::vector vals(input_rows); + std::iota(vals.begin(), vals.end(), 0); + auto input_file = _make_input_file(vals, "test/repart_input"); + + SpillRepartitioner r; + _init_col_mode(r, fanout); + + std::vector output_files; + ASSERT_TRUE(SpillRepartitioner::create_output_spill_files( + _runtime_state.get(), 0, "test/repart_out", fanout, output_files) + .ok()); + ASSERT_TRUE(r.setup_output(_runtime_state.get(), output_files).ok()); + + bool done = false; + while (!done) { + ASSERT_TRUE(r.repartition(_runtime_state.get(), input_file, &done).ok()); + } + ASSERT_TRUE(r.finalize().ok()); + + int64_t total = 0; + for (auto& f : output_files) { + total += _count_rows(f); + } + ASSERT_EQ(total, input_rows); +} + +// ── repartition(SpillFileReader) ───────────────────────────────────────────── + +TEST_F(SpillRepartitionerTest, RepartitionFromReaderConservesRows) { + const int fanout = 4; + const int input_rows = 60; + + std::vector vals(input_rows); + std::iota(vals.begin(), vals.end(), 100); + auto input_file = _make_input_file(vals, "test/repart_reader_input"); + + SpillRepartitioner r; + _init_col_mode(r, fanout); + + std::vector output_files; + ASSERT_TRUE(SpillRepartitioner::create_output_spill_files( + _runtime_state.get(), 0, "test/repart_reader_out", fanout, output_files) + .ok()); + ASSERT_TRUE(r.setup_output(_runtime_state.get(), output_files).ok()); + + // Open a reader manually and pass it to the reader overload. + auto reader = input_file->create_reader(_runtime_state.get(), _profile.get()); + ASSERT_TRUE(reader->open().ok()); + + bool done = false; + while (!done) { + ASSERT_TRUE(r.repartition(_runtime_state.get(), reader, &done).ok()); + } + // repartition() resets the reader to nullptr on completion. + ASSERT_EQ(reader, nullptr); + ASSERT_TRUE(r.finalize().ok()); + + int64_t total = 0; + for (auto& f : output_files) { + total += _count_rows(f); + } + ASSERT_EQ(total, input_rows); +} + +// ── level-dependent routing ────────────────────────────────────────────────── + +// Route the same block at two different repartition levels and verify that the +// total row count is conserved at each level. Different levels use different +// hash salts, so the per-partition distributions will generally differ. +TEST_F(SpillRepartitionerTest, DifferentLevelsConserveRows) { + const int fanout = 4; + const int input_rows = 120; + + std::vector vals(input_rows); + std::iota(vals.begin(), vals.end(), 0); + + auto run_at_level = [&](int level, const std::string& label) -> std::vector { + SpillRepartitioner r; + _init_col_mode(r, fanout, level); + + std::vector output_files; + EXPECT_TRUE(SpillRepartitioner::create_output_spill_files(_runtime_state.get(), 0, label, + fanout, output_files) + .ok()); + EXPECT_TRUE(r.setup_output(_runtime_state.get(), output_files).ok()); + + auto block = ColumnHelper::create_block(vals); + EXPECT_TRUE(r.route_block(_runtime_state.get(), block).ok()); + EXPECT_TRUE(r.finalize().ok()); + + std::vector counts(fanout); + for (int i = 0; i < fanout; ++i) { + counts[i] = _count_rows(output_files[i]); + } + return counts; + }; + + auto counts0 = run_at_level(0, "test/level0"); + auto counts1 = run_at_level(1, "test/level1"); + + // Both levels must conserve all rows. + ASSERT_EQ(std::accumulate(counts0.begin(), counts0.end(), 0LL), input_rows); + ASSERT_EQ(std::accumulate(counts1.begin(), counts1.end(), 0LL), input_rows); + + // The per-partition distributions should differ between levels. + ASSERT_NE(counts0, counts1) << "Level 0 and level 1 produced identical distributions; " + "the level-dependent salt may not be working."; +} + +// ── multiple route_block calls before finalize ─────────────────────────────── + +TEST_F(SpillRepartitionerTest, MultipleRouteBlockCallsConserveTotalRows) { + const int fanout = 4; + SpillRepartitioner r; + _init_col_mode(r, fanout); + + std::vector output_files; + ASSERT_TRUE(SpillRepartitioner::create_output_spill_files( + _runtime_state.get(), 0, "test/multi_route", fanout, output_files) + .ok()); + ASSERT_TRUE(r.setup_output(_runtime_state.get(), output_files).ok()); + + int64_t expected_total = 0; + for (int i = 0; i < 5; ++i) { + std::vector vals(20); + std::iota(vals.begin(), vals.end(), i * 20); + auto block = ColumnHelper::create_block(vals); + ASSERT_TRUE(r.route_block(_runtime_state.get(), block).ok()); + expected_total += 20; + } + ASSERT_TRUE(r.finalize().ok()); + + int64_t total = 0; + for (auto& f : output_files) { + total += _count_rows(f); + } + ASSERT_EQ(total, expected_total); +} + +// ── repartition counter is updated ────────────────────────────────────────── + +TEST_F(SpillRepartitionerTest, RepartitionRowsCounterUpdated) { + const int fanout = 4; + const int input_rows = 50; + + SpillRepartitioner r; + _init_col_mode(r, fanout); + + std::vector output_files; + ASSERT_TRUE(SpillRepartitioner::create_output_spill_files(_runtime_state.get(), 0, + "test/counter", fanout, output_files) + .ok()); + ASSERT_TRUE(r.setup_output(_runtime_state.get(), output_files).ok()); + + std::vector vals(input_rows); + std::iota(vals.begin(), vals.end(), 0); + auto block = ColumnHelper::create_block(vals); + ASSERT_TRUE(r.route_block(_runtime_state.get(), block).ok()); + ASSERT_TRUE(r.finalize().ok()); + + auto* rows_counter = _profile->get_counter("SpillRepartitionRows"); + ASSERT_NE(rows_counter, nullptr); + ASSERT_EQ(rows_counter->value(), input_rows); +} + +} // 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 064f7f06ee6bf4..e29895c149c136 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 @@ -656,12 +656,19 @@ public class SessionVariable implements Serializable, Writable { public static final String ENABLE_FORCE_SPILL = "enable_force_spill"; public static final String ENABLE_RESERVE_MEMORY = "enable_reserve_memory"; public static final String SPILL_MIN_REVOCABLE_MEM = "spill_min_revocable_mem"; - public static final String SPILL_SORT_MEM_LIMIT = "spill_sort_mem_limit"; - // spill_sort_batch_bytes controls the memory size of a sindle block data of spill sort. - public static final String SPILL_SORT_BATCH_BYTES = "spill_sort_batch_bytes"; public static final String SPILL_AGGREGATION_PARTITION_COUNT = "spill_aggregation_partition_count"; public static final String SPILL_STREAMING_AGG_MEM_LIMIT = "spill_streaming_agg_mem_limit"; public static final String SPILL_HASH_JOIN_PARTITION_COUNT = "spill_hash_join_partition_count"; + public static final String SPILL_REPARTITION_MAX_DEPTH = "spill_repartition_max_depth"; + public static final String SPILL_BUFFER_SIZE_BYTES = "spill_buffer_size_bytes"; + public static final String SPILL_JOIN_BUILD_SINK_MEM_LIMIT_BYTES = + "spill_join_build_sink_mem_limit_bytes"; + public static final String SPILL_AGGREGATION_SINK_MEM_LIMIT_BYTES = + "spill_aggregation_sink_mem_limit_bytes"; + public static final String SPILL_SORT_SINK_MEM_LIMIT_BYTES = + "spill_sort_sink_mem_limit_bytes"; + public static final String SPILL_SORT_MERGE_MEM_LIMIT_BYTES = + "spill_sort_merge_mem_limit_bytes"; public static final String SPILL_REVOCABLE_MEMORY_HIGH_WATERMARK_PERCENT = "spill_revocable_memory_high_watermark_percent"; public static final String DATA_QUEUE_MAX_BLOCKS = "data_queue_max_blocks"; @@ -2769,7 +2776,7 @@ public boolean isEnableHboNonStrictMatchingMode() { @VariableMgr.VarAttr(name = MINIMUM_OPERATOR_MEMORY_REQUIRED_KB, needForward = true, description = {"一个算子运行需要的最小的内存大小", "The minimum memory required to be used by an operator, if not meet, the operator will not run"}) - public int minimumOperatorMemoryRequiredKB = 1000; + public int minimumOperatorMemoryRequiredKB = 32000; public static final String IGNORE_RUNTIME_FILTER_IDS = "ignore_runtime_filter_ids"; @@ -3060,20 +3067,18 @@ public void setDetailShapePlanNodes(String detailShapePlanNodes) { public boolean enableReserveMemory = true; @VariableMgr.VarAttr(name = SPILL_MIN_REVOCABLE_MEM, fuzzy = true) - public long spillMinRevocableMem = 32 * 1024 * 1024; + public long spillMinRevocableMem = 4 * 1024 * 1024; - // spill_sort_mem_limit controls the memory usage during merge sort phase of spill sort. - // During merge sort phase, mutiple sorted blocks will be read into memory and do merge sort, - // the count of blocks should be controlled or else will cause OOM, it's calculated as - // std::max(spill_sort_mem_limit / spill_sort_batch_bytes, 2) - @VariableMgr.VarAttr(name = SPILL_SORT_MEM_LIMIT) - public long spillSortMemLimit = 134217728; // 128M - - @VariableMgr.VarAttr(name = SPILL_SORT_BATCH_BYTES) - public long spillSortBatchBytes = 8388608; // 8M + @VariableMgr.VarAttr(name = SPILL_BUFFER_SIZE_BYTES, fuzzy = true, needForward = true, + description = {"落盘时写 block 的最大大小(字节)。如果一个 block 超过该阈值,会按此大小拆分后再写入磁盘。" + + "同时也控制 merge sort 阶段每个文件的读 buffer 大小。默认 8MB。", + "Maximum block size for spill writes (in bytes). Blocks larger than this threshold are " + + "split before writing to disk. Also controls per-file read buffer size during merge sort. " + + "Default is 8MB."}) + public long spillBufferSizeBytes = 8L * 1024L * 1024L; @VariableMgr.VarAttr(name = SPILL_AGGREGATION_PARTITION_COUNT, fuzzy = true) - public int spillAggregationPartitionCount = 32; + public int spillAggregationPartitionCount = 4; @VariableMgr.VarAttr(name = LOW_MEMORY_MODE_BUFFER_LIMIT, fuzzy = false) public long lowMemoryModeBufferLimit = 33554432; @@ -3084,7 +3089,38 @@ public void setDetailShapePlanNodes(String detailShapePlanNodes) { public long spillStreamingAggMemLimit = 268435456; //256MB @VariableMgr.VarAttr(name = SPILL_HASH_JOIN_PARTITION_COUNT, fuzzy = true) - public int spillHashJoinPartitionCount = 32; + public int spillHashJoinPartitionCount = 4; + + @VariableMgr.VarAttr(name = SPILL_REPARTITION_MAX_DEPTH, fuzzy = true, needForward = true, + description = {"重分区的最大递归深度,超过该深度不再继续重分区,\n默认值为 8", + "Maximum depth for repartition recursion. When exceeded, repartitioning will stop. Default is 8."}) + public int spillRepartitionMaxDepth = 8; + + @VariableMgr.VarAttr(name = SPILL_JOIN_BUILD_SINK_MEM_LIMIT_BYTES, fuzzy = true, needForward = true, + description = {"一旦触发 spill 后,join build sink 的 revocable memory 超过该阈值就主动落盘(字节)。默认 64MB。", + "After spill is triggered, join build sink will proactively spill when revocable memory " + + "exceeds this threshold (in bytes). Default is 64MB."}) + public long spillJoinBuildSinkMemLimitBytes = 64L * 1024L * 1024L; + + @VariableMgr.VarAttr(name = SPILL_AGGREGATION_SINK_MEM_LIMIT_BYTES, fuzzy = true, needForward = true, + description = {"一旦触发 spill 后,aggregation sink 的 revocable memory 超过该阈值就主动落盘(字节)。默认 64MB。", + "After spill is triggered, aggregation sink will proactively spill when revocable memory " + + "exceeds this threshold (in bytes). Default is 64GB."}) + public long spillAggregationSinkMemLimitBytes = 64L * 1024L * 1024L * 1024L; + + @VariableMgr.VarAttr(name = SPILL_SORT_SINK_MEM_LIMIT_BYTES, fuzzy = true, needForward = true, + description = {"一旦触发 spill 后,sort sink 的 revocable memory 超过该阈值就主动落盘(字节)。默认 64MB。", + "After spill is triggered, sort sink will proactively spill when revocable memory " + + "exceeds this threshold (in bytes). Default is 64MB."}) + public long spillSortSinkMemLimitBytes = 64L * 1024L * 1024L; + + @VariableMgr.VarAttr(name = SPILL_SORT_MERGE_MEM_LIMIT_BYTES, fuzzy = true, needForward = true, + description = {"一旦触发 spill 后,sort merge 阶段可用的总内存大小(字节)。" + + "该值除以 spill_buffer_size_bytes 即为可并行读取合并的文件数。默认 64MB。", + "After spill is triggered, total memory budget for the sort merge phase (in bytes). " + + "Divided by spill_buffer_size_bytes gives the number of files that can be merged " + + "in parallel. Default is 64MB."}) + public long spillSortMergeMemLimitBytes = 64L * 1024L * 1024L; @VariableMgr.VarAttr(name = SPILL_REVOCABLE_MEMORY_HIGH_WATERMARK_PERCENT, fuzzy = true) public int spillRevocableMemoryHighWatermarkPercent = -1; @@ -5198,13 +5234,18 @@ public TQueryOptions toThrift() { tResult.setEnableForceSpill(enableForceSpill); tResult.setEnableReserveMemory(enableReserveMemory); tResult.setMinRevocableMem(spillMinRevocableMem); - tResult.setSpillSortMemLimit(spillSortMemLimit); - tResult.setSpillSortBatchBytes(spillSortBatchBytes); tResult.setSpillAggregationPartitionCount(spillAggregationPartitionCount); tResult.setSpillStreamingAggMemLimit(spillStreamingAggMemLimit); tResult.setSpillHashJoinPartitionCount(spillHashJoinPartitionCount); tResult.setRevocableMemoryHighWatermarkPercent(spillRevocableMemoryHighWatermarkPercent); tResult.setDumpHeapProfileWhenMemLimitExceeded(dumpHeapProfileWhenMemLimitExceeded); + // Forward new spill-related tuning vars to BE + tResult.setSpillBufferSizeBytes(spillBufferSizeBytes); + tResult.setSpillRepartitionMaxDepth(spillRepartitionMaxDepth); + tResult.setSpillJoinBuildSinkMemLimitBytes(spillJoinBuildSinkMemLimitBytes); + tResult.setSpillAggregationSinkMemLimitBytes(spillAggregationSinkMemLimitBytes); + tResult.setSpillSortSinkMemLimitBytes(spillSortSinkMemLimitBytes); + tResult.setSpillSortMergeMemLimitBytes(spillSortMergeMemLimitBytes); tResult.setDataQueueMaxBlocks(dataQueueMaxBlocks); tResult.setLowMemoryModeBufferLimit(lowMemoryModeBufferLimit); diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 56e6724e900068..9679d549ad9308 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -436,6 +436,7 @@ struct TQueryOptions { 200: optional bool enable_adjust_conjunct_order_by_cost; // Use paimon-cpp to read Paimon splits on BE 201: optional bool enable_paimon_cpp_reader = false; + // Whether all fragments of this query are assigned to a single backend. // When true, the streaming aggregation operator can use more aggressive // hash table expansion thresholds since all data is local. @@ -445,6 +446,31 @@ struct TQueryOptions { 203: optional bool enable_inverted_index_wand_query = true; + // Per-read/per-write buffer size used during spill I/O, in bytes. Controls the + // I/O batch size for spill write and merge read. This value can be overridden + // per-query by setting the session variable `spill_buffer_size_bytes` in FE. + // Default is 8MB. + 204: optional i64 spill_buffer_size_bytes = 8388608 + + // Per-sink memory limit after spill is triggered. When a sink operator's revocable + // memory exceeds the corresponding threshold, it proactively spills to disk. + // Default is 64MB for all three. + 205: optional i64 spill_join_build_sink_mem_limit_bytes = 67108864 + 206: optional i64 spill_aggregation_sink_mem_limit_bytes = 67108864 + 207: optional i64 spill_sort_sink_mem_limit_bytes = 67108864 + + // Total memory budget for the sort merge phase after spill. Divided by + // spill_buffer_size_bytes gives the number of files merged in parallel. + // Default is 64MB. + 208: optional i64 spill_sort_merge_mem_limit_bytes = 67108864 + + // Maximum depth for repartitioning recursion. Controls how many recursive + // repartition rounds are allowed before giving up and treating a partition + // as terminal. This value can be overridden per-query by setting the + // session variable `spill_repartition_max_depth` in FE. Default is 8. + 209: optional i32 spill_repartition_max_depth = 8 + + // For cloud, to control if the content would be written into file cache // In write path, to control if the content would be written into file cache. // In read path, read from file cache or remote storage when execute query.