diff --git a/be/src/agent/workload_group_listener.cpp b/be/src/agent/workload_group_listener.cpp index 0cd5a3ee1ac748..741f064a8444de 100644 --- a/be/src/agent/workload_group_listener.cpp +++ b/be/src/agent/workload_group_listener.cpp @@ -17,6 +17,8 @@ #include "agent/workload_group_listener.h" +#include + #include "runtime/exec_env.h" #include "runtime/workload_group/workload_group.h" #include "runtime/workload_group/workload_group_manager.h" @@ -33,6 +35,8 @@ void WorkloadGroupListener::handle_topic_info(const std::vector& topi if (!topic_info.__isset.workload_group_info) { continue; } + LOG(INFO) << "Received publish workload group info request: " + << apache::thrift::ThriftDebugString(topic_info).c_str(); is_set_workload_group_info = true; // 1 parse topic info to group info diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 218dc2ae6b152e..1359ae51080ec5 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -16,6 +16,7 @@ // under the License. #include +#include #include #include @@ -118,7 +119,7 @@ DEFINE_String(mem_limit, "90%"); DEFINE_Double(soft_mem_limit_frac, "0.9"); // Cache capacity reduce mem limit as a fraction of soft mem limit. -DEFINE_mDouble(cache_capacity_reduce_mem_limit_frac, "0.6"); +DEFINE_mDouble(cache_capacity_reduce_mem_limit_frac, "0.7"); // Schema change memory limit as a fraction of soft memory limit. DEFINE_Double(schema_change_mem_limit_frac, "0.6"); @@ -1271,6 +1272,9 @@ DEFINE_Validator(spill_io_thread_pool_thread_num, [](const int config) -> bool { }); DEFINE_Int32(spill_io_thread_pool_queue_size, "102400"); +// paused query in queue timeout(ms) will be resumed or canceled +DEFINE_Int64(spill_in_paused_queue_timeout_ms, "60000"); + DEFINE_mBool(check_segment_when_build_rowset_meta, "false"); DEFINE_mInt32(max_s3_client_retry, "10"); diff --git a/be/src/common/config.h b/be/src/common/config.h index ddca52c607b495..60a718d1e70bb5 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1350,6 +1350,7 @@ DECLARE_mInt32(spill_gc_interval_ms); DECLARE_mInt32(spill_gc_work_time_ms); DECLARE_Int32(spill_io_thread_pool_thread_num); DECLARE_Int32(spill_io_thread_pool_queue_size); +DECLARE_Int64(spill_in_paused_queue_timeout_ms); DECLARE_mBool(check_segment_when_build_rowset_meta); diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index e930cf1d3132fb..2ba18320318177 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -320,15 +320,18 @@ void Daemon::memory_maintenance_thread() { doris::ExecEnv::GetInstance()->workload_group_mgr()->do_sweep(); doris::ExecEnv::GetInstance()->workload_group_mgr()->refresh_wg_weighted_memory_limit(); - // step 7. Analyze blocking queries. + // step 7: handle paused queries(caused by memory insufficient) + doris::ExecEnv::GetInstance()->workload_group_mgr()->handle_paused_queries(); + + // step 8. Analyze blocking queries. // TODO sort the operators that can spill, wake up the pipeline task spill // or continue execution according to certain rules or cancel query. - // step 8. Flush memtable + // step 9. Flush memtable doris::GlobalMemoryArbitrator::notify_memtable_memory_refresh(); // TODO notify flush memtable - // step 9. Reset Jemalloc dirty page decay. + // step 10. Reset Jemalloc dirty page decay. je_reset_dirty_decay(); } } @@ -542,7 +545,9 @@ void Daemon::cache_adjust_capacity_thread() { doris::GlobalMemoryArbitrator::cache_adjust_capacity_cv.wait_for( l, std::chrono::milliseconds(100)); } - double adjust_weighted = GlobalMemoryArbitrator::last_cache_capacity_adjust_weighted; + double adjust_weighted = std::min( + GlobalMemoryArbitrator::last_cache_capacity_adjust_weighted, + GlobalMemoryArbitrator::last_wg_trigger_cache_capacity_adjust_weighted); if (_stop_background_threads_latch.count() == 0) { break; } @@ -562,6 +567,7 @@ void Daemon::cache_adjust_capacity_thread() { LOG(INFO) << fmt::format( "[MemoryGC] refresh cache capacity end, free memory {}, details: {}", PrettyPrinter::print(freed_mem, TUnit::BYTES), ss.str()); + GlobalMemoryArbitrator::last_affected_cache_capacity_adjust_weighted = adjust_weighted; } while (true); } diff --git a/be/src/common/status.h b/be/src/common/status.h index 0252ec8564feeb..841d509069932b 100644 --- a/be/src/common/status.h +++ b/be/src/common/status.h @@ -131,6 +131,9 @@ namespace ErrorCode { E(BAD_CAST, -254, true); \ E(ARITHMETIC_OVERFLOW_ERRROR, -255, false); \ E(PERMISSION_DENIED, -256, false); \ + E(QUERY_MEMORY_EXCEEDED, -257, false); \ + E(WORKLOAD_GROUP_MEMORY_EXCEEDED, -258, false); \ + E(PROCESS_MEMORY_EXCEEDED, -259, false); \ E(CE_CMD_PARAMS_ERROR, -300, true); \ E(CE_BUFFER_TOO_SMALL, -301, true); \ E(CE_CMD_NOT_VALID, -302, true); \ @@ -381,6 +384,11 @@ class [[nodiscard]] Status { _code = rhs._code; if (rhs._err_msg) { _err_msg = std::make_unique(*rhs._err_msg); + } else { + // If rhs error msg is empty, then should also clear current error msg + // For example, if rhs is OK and current status is error, then copy to current + // status, should clear current error message. + _err_msg.reset(); } return *this; } @@ -390,6 +398,8 @@ class [[nodiscard]] Status { _code = rhs._code; if (rhs._err_msg) { _err_msg = std::move(rhs._err_msg); + } else { + _err_msg.reset(); } return *this; } diff --git a/be/src/exec/schema_scanner/schema_backend_active_tasks.cpp b/be/src/exec/schema_scanner/schema_backend_active_tasks.cpp index eb7b373c7dc7f6..eafee0ebe73925 100644 --- a/be/src/exec/schema_scanner/schema_backend_active_tasks.cpp +++ b/be/src/exec/schema_scanner/schema_backend_active_tasks.cpp @@ -31,6 +31,7 @@ std::vector SchemaBackendActiveTasksScanner::_s_tbls_ // name, type, size {"BE_ID", TYPE_BIGINT, sizeof(int64_t), false}, {"FE_HOST", TYPE_VARCHAR, sizeof(StringRef), false}, + {"WORKLOAD_GROUP_ID", TYPE_BIGINT, sizeof(int64_t), false}, {"QUERY_ID", TYPE_VARCHAR, sizeof(StringRef), false}, {"TASK_TIME_MS", TYPE_BIGINT, sizeof(int64_t), false}, {"TASK_CPU_TIME_MS", TYPE_BIGINT, sizeof(int64_t), false}, @@ -41,6 +42,8 @@ std::vector SchemaBackendActiveTasksScanner::_s_tbls_ {"SHUFFLE_SEND_BYTES", TYPE_BIGINT, sizeof(int64_t), false}, {"SHUFFLE_SEND_ROWS", TYPE_BIGINT, sizeof(int64_t), false}, {"QUERY_TYPE", TYPE_VARCHAR, sizeof(StringRef), false}, + {"SPILL_WRITE_BYTES_TO_LOCAL_STORAGE", TYPE_BIGINT, sizeof(int64_t), false}, + {"SPILL_READ_BYTES_FROM_LOCAL_STORAGE", TYPE_BIGINT, sizeof(int64_t), false}, }; SchemaBackendActiveTasksScanner::SchemaBackendActiveTasksScanner() diff --git a/be/src/exec/schema_scanner/schema_workload_group_resource_usage_scanner.cpp b/be/src/exec/schema_scanner/schema_workload_group_resource_usage_scanner.cpp index 805bf12cc38ae6..4d524872c3ba5f 100644 --- a/be/src/exec/schema_scanner/schema_workload_group_resource_usage_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_workload_group_resource_usage_scanner.cpp @@ -38,6 +38,7 @@ std::vector SchemaBackendWorkloadGroupResourceUsage:: {"CPU_USAGE_PERCENT", TYPE_DOUBLE, sizeof(double), false}, {"LOCAL_SCAN_BYTES_PER_SECOND", TYPE_BIGINT, sizeof(int64_t), false}, {"REMOTE_SCAN_BYTES_PER_SECOND", TYPE_BIGINT, sizeof(int64_t), false}, + {"WRITE_BUFFER_USAGE_BYTES", TYPE_BIGINT, sizeof(int64_t), false}, }; SchemaBackendWorkloadGroupResourceUsage::SchemaBackendWorkloadGroupResourceUsage() diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp index f8cc79b205535f..62d9ecf44a93ff 100644 --- a/be/src/olap/memtable.cpp +++ b/be/src/olap/memtable.cpp @@ -143,7 +143,8 @@ void MemTable::_init_agg_functions(const vectorized::Block* block) { } MemTable::~MemTable() { - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_query_thread_context.query_mem_tracker); + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER( + _query_thread_context.query_mem_tracker->write_tracker()); if (_is_flush_success) { // If the memtable is flush success, then its memtracker's consumption should be 0 if (_mem_tracker->consumption() != 0 && config::crash_in_memory_tracker_inaccurate) { @@ -182,6 +183,8 @@ int RowInBlockComparator::operator()(const RowInBlock* left, const RowInBlock* r Status MemTable::insert(const vectorized::Block* input_block, const std::vector& row_idxs) { + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER( + _query_thread_context.query_mem_tracker->write_tracker()); SCOPED_CONSUME_MEM_TRACKER(_mem_tracker); if (_is_first_insertion) { @@ -579,6 +582,8 @@ void MemTable::_aggregate() { } void MemTable::shrink_memtable_by_agg() { + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER( + _query_thread_context.query_mem_tracker->write_tracker()); SCOPED_CONSUME_MEM_TRACKER(_mem_tracker); if (_keys_type == KeysType::DUP_KEYS) { return; @@ -608,6 +613,20 @@ bool MemTable::need_agg() const { return false; } +size_t MemTable::get_flush_reserve_memory_size() const { + size_t reserve_size = 0; + if (_keys_type == KeysType::DUP_KEYS) { + if (_tablet_schema->num_key_columns() == 0) { + // no need to reserve + } else { + reserve_size = _input_mutable_block.allocated_bytes(); + } + } else { + reserve_size = _input_mutable_block.allocated_bytes(); + } + return reserve_size; +} + Status MemTable::_to_block(std::unique_ptr* res) { size_t same_keys_num = _sort(); if (_keys_type == KeysType::DUP_KEYS || same_keys_num == 0) { diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h index 77ff2e886bff36..09591df2745780 100644 --- a/be/src/olap/memtable.h +++ b/be/src/olap/memtable.h @@ -181,6 +181,7 @@ class MemTable { int64_t tablet_id() const { return _tablet_id; } size_t memory_usage() const { return _mem_tracker->consumption(); } + size_t get_flush_reserve_memory_size() const; // insert tuple from (row_pos) to (row_pos+num_rows) Status insert(const vectorized::Block* block, const std::vector& row_idxs); diff --git a/be/src/olap/memtable_flush_executor.cpp b/be/src/olap/memtable_flush_executor.cpp index 0181cc1d64d91a..4107113e11fd3d 100644 --- a/be/src/olap/memtable_flush_executor.cpp +++ b/be/src/olap/memtable_flush_executor.cpp @@ -28,6 +28,7 @@ #include "common/signal_handler.h" #include "olap/memtable.h" #include "olap/rowset/rowset_writer.h" +#include "olap/storage_engine.h" #include "util/debug_points.h" #include "util/doris_metrics.h" #include "util/metrics.h" @@ -137,6 +138,37 @@ Status FlushToken::wait() { return Status::OK(); } +Status FlushToken::_try_reserve_memory(QueryThreadContext query_thread_context, int64_t size) { + auto* thread_context = doris::thread_context(); + auto* memtable_flush_executor = + ExecEnv::GetInstance()->storage_engine().memtable_flush_executor(); + Status st; + do { + // only try to reserve process memory + st = thread_context->try_reserve_process_memory(size); + if (st.ok()) { + memtable_flush_executor->inc_flushing_task(); + break; + } + if (_is_shutdown() || query_thread_context.get_memory_tracker()->is_query_cancelled()) { + st = Status::Cancelled("flush memtable already cancelled"); + break; + } + // Make sure at least one memtable is flushing even reserve memory failed. + if (memtable_flush_executor->check_and_inc_has_any_flushing_task()) { + // If there are already any flushing task, Wait for some time and retry. + LOG_EVERY_T(INFO, 60) << fmt::format( + "Failed to reserve memory {} for flush memtable, retry after 100ms", + PrettyPrinter::print_bytes(size)); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } else { + st = Status::OK(); + break; + } + } while (true); + return st; +} + Status FlushToken::_do_flush_memtable(MemTable* memtable, int32_t segment_id, int64_t* flush_size) { VLOG_CRITICAL << "begin to flush memtable for tablet: " << memtable->tablet_id() << ", memsize: " << memtable->memory_usage() @@ -147,8 +179,19 @@ Status FlushToken::_do_flush_memtable(MemTable* memtable, int32_t segment_id, in SCOPED_ATTACH_TASK(memtable->query_thread_context()); signal::set_signal_task_id(_rowset_writer->load_id()); signal::tablet_id = memtable->tablet_id(); + + DEFER_RELEASE_RESERVED(); + + auto reserve_size = memtable->get_flush_reserve_memory_size(); + RETURN_IF_ERROR(_try_reserve_memory(memtable->query_thread_context(), reserve_size)); { + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER( + memtable->query_thread_context().query_mem_tracker->write_tracker()); SCOPED_CONSUME_MEM_TRACKER(memtable->mem_tracker()); + + Defer defer {[&]() { + ExecEnv::GetInstance()->storage_engine().memtable_flush_executor()->dec_flushing_task(); + }}; std::unique_ptr block; RETURN_IF_ERROR(memtable->to_block(&block)); RETURN_IF_ERROR(_rowset_writer->flush_memtable(block.get(), segment_id, flush_size)); diff --git a/be/src/olap/memtable_flush_executor.h b/be/src/olap/memtable_flush_executor.h index 753f1106646407..e3bb41ed45c069 100644 --- a/be/src/olap/memtable_flush_executor.h +++ b/be/src/olap/memtable_flush_executor.h @@ -94,6 +94,8 @@ class FlushToken : public std::enable_shared_from_this { Status _do_flush_memtable(MemTable* memtable, int32_t segment_id, int64_t* flush_size); + Status _try_reserve_memory(QueryThreadContext query_thread_context, int64_t size); + // Records the current flush status of the tablet. // Note: Once its value is set to Failed, it cannot return to SUCCESS. std::shared_mutex _flush_status_lock; @@ -139,9 +141,26 @@ class MemTableFlushExecutor { std::shared_ptr rowset_writer, bool is_high_priority, std::shared_ptr wg_sptr); + // return true if it already has any flushing task + bool check_and_inc_has_any_flushing_task() { + // need to use CAS instead of only `if (0 == _flushing_task_count)` statement, + // to avoid concurrent entries both pass the if statement + int expected_count = 0; + if (!_flushing_task_count.compare_exchange_strong(expected_count, 1)) { + return true; + } + DCHECK(expected_count == 0 && _flushing_task_count == 1); + return false; + } + + void inc_flushing_task() { _flushing_task_count++; } + + void dec_flushing_task() { _flushing_task_count--; } + private: std::unique_ptr _flush_pool; std::unique_ptr _high_prio_flush_pool; + std::atomic _flushing_task_count = 0; }; } // namespace doris diff --git a/be/src/olap/memtable_memory_limiter.cpp b/be/src/olap/memtable_memory_limiter.cpp index 043ce9967fbe5a..22b842ec672705 100644 --- a/be/src/olap/memtable_memory_limiter.cpp +++ b/be/src/olap/memtable_memory_limiter.cpp @@ -22,6 +22,7 @@ #include "common/config.h" #include "olap/memtable.h" #include "olap/memtable_writer.h" +#include "runtime/workload_group/workload_group_manager.h" #include "util/doris_metrics.h" #include "util/mem_info.h" #include "util/metrics.h" @@ -110,7 +111,23 @@ int64_t MemTableMemoryLimiter::_need_flush() { return need_flush - _queue_mem_usage; } -void MemTableMemoryLimiter::handle_memtable_flush() { +void MemTableMemoryLimiter::handle_workload_group_memtable_flush(WorkloadGroupPtr wg) { + // It means some query is pending on here to flush memtable and to continue running. + // So that should wait here. + // Wait at most 1s, because this code is not aware cancel flag. If the load task is cancelled + // Should releae memory quickly. + using namespace std::chrono_literals; + int32_t sleep_times = 10; + while (wg != nullptr && wg->enable_write_buffer_limit() && wg->exceed_write_buffer_limit() && + sleep_times > 0) { + std::this_thread::sleep_for(100ms); + --sleep_times; + } + // Check process memory again. + handle_memtable_flush(wg); +} + +void MemTableMemoryLimiter::handle_memtable_flush(WorkloadGroupPtr wg) { // Check the soft limit. DCHECK(_load_soft_mem_limit > 0); if (!_soft_limit_reached() || _load_usage_low()) { @@ -133,30 +150,69 @@ void MemTableMemoryLimiter::handle_memtable_flush() { if (need_flush > 0) { auto limit = _hard_limit_reached() ? Limit::HARD : Limit::SOFT; LOG(INFO) << "reached memtable memory " << (limit == Limit::HARD ? "hard" : "soft") - << ", " << GlobalMemoryArbitrator::process_memory_used_details_str() + << ", " << GlobalMemoryArbitrator::process_memory_used_details_str() << ", " + << GlobalMemoryArbitrator::sys_mem_available_details_str() << ", load mem: " << PrettyPrinter::print_bytes(_mem_tracker->consumption()) << ", memtable writers num: " << _writers.size() << ", active: " << PrettyPrinter::print_bytes(_active_mem_usage) << ", queue: " << PrettyPrinter::print_bytes(_queue_mem_usage) - << ", flush: " << PrettyPrinter::print_bytes(_flush_mem_usage); - _flush_active_memtables(need_flush); + << ", flush: " << PrettyPrinter::print_bytes(_flush_mem_usage) + << ", wg: " << (wg ? wg->debug_string() : "null\n") + << doris::ProcessProfile::instance() + ->memory_profile() + ->process_memory_detail_str(); + _flush_active_memtables(0, need_flush); } } while (_hard_limit_reached() && !_load_usage_low()); g_memtable_memory_limit_waiting_threads << -1; timer.stop(); int64_t time_ms = timer.elapsed_time() / 1000 / 1000; g_memtable_memory_limit_latency_ms << time_ms; - LOG(INFO) << "waited " << time_ms << " ms for memtable memory limit"; + LOG(INFO) << "waited " << time_ms << " ms for memtable memory limit" + << ", " << GlobalMemoryArbitrator::process_memory_used_details_str() << ", " + << GlobalMemoryArbitrator::sys_mem_available_details_str() + << ", load mem: " << PrettyPrinter::print_bytes(_mem_tracker->consumption()) + << ", memtable writers num: " << _writers.size() + << ", active: " << PrettyPrinter::print_bytes(_active_mem_usage) + << ", queue: " << PrettyPrinter::print_bytes(_queue_mem_usage) + << ", flush: " << PrettyPrinter::print_bytes(_flush_mem_usage) + << ", wg: " << (wg ? wg->debug_string() : "null.\n") + << doris::ProcessProfile::instance()->memory_profile()->process_memory_detail_str(); +} + +int64_t MemTableMemoryLimiter::flush_workload_group_memtables(uint64_t wg_id, int64_t need_flush) { + std::unique_lock l(_lock); + return _flush_active_memtables(wg_id, need_flush); } -void MemTableMemoryLimiter::_flush_active_memtables(int64_t need_flush) { +void MemTableMemoryLimiter::get_workload_group_memtable_usage(uint64_t wg_id, int64_t* active_bytes, + int64_t* queue_bytes, + int64_t* flush_bytes) { + std::unique_lock l(_lock); + *active_bytes = 0; + *queue_bytes = 0; + *flush_bytes = 0; + for (auto it = _writers.begin(); it != _writers.end(); ++it) { + if (auto writer = it->lock()) { + // If wg id is specified, but wg id not match, then not need flush + if (writer->workload_group_id() != wg_id) { + continue; + } + *active_bytes += writer->active_memtable_mem_consumption(); + *queue_bytes += writer->mem_consumption(MemType::WRITE_FINISHED); + *flush_bytes += writer->mem_consumption(MemType::FLUSH); + } + } +} + +int64_t MemTableMemoryLimiter::_flush_active_memtables(uint64_t wg_id, int64_t need_flush) { if (need_flush <= 0) { - return; + return 0; } _refresh_mem_tracker(); if (_active_writers.size() == 0) { - return; + return 0; } using WriterMem = std::pair, int64_t>; @@ -181,6 +237,10 @@ void MemTableMemoryLimiter::_flush_active_memtables(int64_t need_flush) { if (w == nullptr) { continue; } + // If wg id is specified, but wg id not match, then not need flush + if (wg_id != 0 && w->workload_group_id() != wg_id) { + continue; + } int64_t mem = w->active_memtable_mem_consumption(); if (mem < sort_mem * 0.9) { // if the memtable writer just got flushed, don't flush it again @@ -200,6 +260,7 @@ void MemTableMemoryLimiter::_flush_active_memtables(int64_t need_flush) { } LOG(INFO) << "flushed " << num_flushed << " out of " << _active_writers.size() << " active writers, flushed size: " << PrettyPrinter::print_bytes(mem_flushed); + return mem_flushed; } void MemTableMemoryLimiter::refresh_mem_tracker() { @@ -223,11 +284,13 @@ void MemTableMemoryLimiter::refresh_mem_tracker() { _last_limit = limit; _log_timer.reset(); LOG(INFO) << ss.str() << ", " << GlobalMemoryArbitrator::process_memory_used_details_str() + << ", " << GlobalMemoryArbitrator::sys_mem_available_details_str() << ", load mem: " << PrettyPrinter::print_bytes(_mem_tracker->consumption()) << ", memtable writers num: " << _writers.size() << ", active: " << PrettyPrinter::print_bytes(_active_mem_usage) << ", queue: " << PrettyPrinter::print_bytes(_queue_mem_usage) - << ", flush: " << PrettyPrinter::print_bytes(_flush_mem_usage); + << ", flush: " << PrettyPrinter::print_bytes(_flush_mem_usage) << "\n" + << doris::ProcessProfile::instance()->memory_profile()->process_memory_detail_str(); } void MemTableMemoryLimiter::_refresh_mem_tracker() { @@ -243,8 +306,12 @@ void MemTableMemoryLimiter::_refresh_mem_tracker() { if (active_usage > 0) { _active_writers.push_back(writer); } - _flush_mem_usage += writer->mem_consumption(MemType::FLUSH); - _queue_mem_usage += writer->mem_consumption(MemType::WRITE_FINISHED); + + auto flush_usage = writer->mem_consumption(MemType::FLUSH); + _flush_mem_usage += flush_usage; + + auto write_usage = writer->mem_consumption(MemType::WRITE_FINISHED); + _queue_mem_usage += write_usage; ++it; } else { *it = std::move(_writers.back()); diff --git a/be/src/olap/memtable_memory_limiter.h b/be/src/olap/memtable_memory_limiter.h index 1e32cb165e4721..155a1dd424b05b 100644 --- a/be/src/olap/memtable_memory_limiter.h +++ b/be/src/olap/memtable_memory_limiter.h @@ -21,6 +21,7 @@ #include "common/status.h" #include "runtime/memory/mem_tracker.h" +#include "runtime/workload_group/workload_group.h" #include "util/countdown_latch.h" #include "util/stopwatch.hpp" @@ -37,9 +38,17 @@ class MemTableMemoryLimiter { Status init(int64_t process_mem_limit); + void handle_workload_group_memtable_flush(WorkloadGroupPtr wg); // check if the total mem consumption exceeds limit. // If yes, it will flush memtable to try to reduce memory consumption. - void handle_memtable_flush(); + // Every write operation will call this API to check if need flush memtable OR hang + // when memory is not available. + void handle_memtable_flush(WorkloadGroupPtr wg); + + int64_t flush_workload_group_memtables(uint64_t wg_id, int64_t need_flush_bytes); + + void get_workload_group_memtable_usage(uint64_t wg_id, int64_t* active_bytes, + int64_t* queue_bytes, int64_t* flush_bytes); void register_writer(std::weak_ptr writer); @@ -57,7 +66,7 @@ class MemTableMemoryLimiter { bool _hard_limit_reached(); bool _load_usage_low(); int64_t _need_flush(); - void _flush_active_memtables(int64_t need_flush); + int64_t _flush_active_memtables(uint64_t wg_id, int64_t need_flush); void _refresh_mem_tracker(); std::mutex _lock; diff --git a/be/src/olap/memtable_writer.h b/be/src/olap/memtable_writer.h index 713400793a1754..924c373531022d 100644 --- a/be/src/olap/memtable_writer.h +++ b/be/src/olap/memtable_writer.h @@ -106,6 +106,14 @@ class MemTableWriter { uint64_t flush_running_count() const; + uint64_t workload_group_id() const { + auto wg = _query_thread_context.wg_wptr.lock(); + if (wg != nullptr) { + return wg->id(); + } + return 0; + } + private: // push a full memtable to flush executor Status _flush_memtable_async(); diff --git a/be/src/pipeline/common/agg_utils.h b/be/src/pipeline/common/agg_utils.h index 135bc67712345f..146649f96b186d 100644 --- a/be/src/pipeline/common/agg_utils.h +++ b/be/src/pipeline/common/agg_utils.h @@ -250,6 +250,28 @@ struct AggregateDataContainer { ConstIterator cend() const { return end(); } Iterator end() { return {this, _total_count}; } + [[nodiscard]] uint32_t total_count() const { return _total_count; } + + size_t estimate_memory(size_t rows) const { + bool need_to_expand = false; + if (_total_count == 0) { + need_to_expand = true; + } else if ((_index_in_sub_container + rows) > SUB_CONTAINER_CAPACITY) { + need_to_expand = true; + rows -= (SUB_CONTAINER_CAPACITY - _index_in_sub_container); + } + + if (!need_to_expand) { + return 0; + } + + size_t count = (rows + SUB_CONTAINER_CAPACITY - 1) / SUB_CONTAINER_CAPACITY; + size_t size = _size_of_key * SUB_CONTAINER_CAPACITY; + size += _size_of_aggregate_states * SUB_CONTAINER_CAPACITY; + size *= count; + return size; + } + void init_once() { if (_inited) { return; diff --git a/be/src/pipeline/dependency.cpp b/be/src/pipeline/dependency.cpp index dcf5c7a0a81d7c..ae60740ae98a60 100644 --- a/be/src/pipeline/dependency.cpp +++ b/be/src/pipeline/dependency.cpp @@ -307,9 +307,8 @@ Status AggSharedState::reset_hash_table() { agg_data->method_variant); } -void PartitionedAggSharedState::init_spill_params(size_t spill_partition_count_bits) { - partition_count_bits = spill_partition_count_bits; - partition_count = (1 << spill_partition_count_bits); +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) { @@ -317,6 +316,19 @@ void PartitionedAggSharedState::init_spill_params(size_t spill_partition_count_b } } +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, vectorized::SpillStreamSPtr& spill_stream) { @@ -355,6 +367,14 @@ void PartitionedAggSharedState::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); + } + } +} + void SpillSortSharedState::close() { // need to use CAS instead of only `if (!is_closed)` statement, // to avoid concurrent entry of close() both pass the if statement @@ -370,9 +390,11 @@ void SpillSortSharedState::close() { } MultiCastSharedState::MultiCastSharedState(const RowDescriptor& row_desc, ObjectPool* pool, - int cast_sender_count) + int cast_sender_count, int node_id) : multi_cast_data_streamer(std::make_unique( - row_desc, pool, cast_sender_count, true)) {} + row_desc, this, pool, cast_sender_count, node_id, true)) {} + +void MultiCastSharedState::update_spill_stream_profiles(RuntimeProfile* source_profile) {} int AggSharedState::get_slot_column_id(const vectorized::AggFnEvaluator* evaluator) { auto ctxs = evaluator->input_exprs_ctxs(); diff --git a/be/src/pipeline/dependency.h b/be/src/pipeline/dependency.h index fd179cdfd0a797..9ce85bcb7b5520 100644 --- a/be/src/pipeline/dependency.h +++ b/be/src/pipeline/dependency.h @@ -27,6 +27,7 @@ #include #include +#include "common/config.h" #include "common/logging.h" #include "gutil/integral_types.h" #include "pipeline/common/agg_utils.h" @@ -423,32 +424,50 @@ struct AggSharedState : public BasicSharedState { Status _destroy_agg_status(vectorized::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 init_spill_params(size_t spill_partition_count_bits); + 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; - size_t partition_count_bits; size_t partition_count; size_t max_partition_index; - Status sink_status; 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 >> (32 - partition_count_bits)) & max_partition_index; - } + size_t get_partition_index(size_t hash_value) const { return hash_value % partition_count; } }; struct AggSpillPartition { @@ -494,30 +513,31 @@ struct SortSharedState : public BasicSharedState { }; struct SpillSortSharedState : public BasicSharedState, + public BasicSpillSharedState, public std::enable_shared_from_this { ENABLE_FACTORY_CREATOR(SpillSortSharedState) SpillSortSharedState() = default; ~SpillSortSharedState() override = default; - // This number specifies the maximum size of sub blocks - static constexpr size_t SORT_BLOCK_SPILL_BATCH_BYTES = 8 * 1024 * 1024; - void update_spill_block_batch_row_count(const vectorized::Block* block) { + void update_spill_block_batch_row_count(RuntimeState* state, const vectorized::Block* block) { auto rows = block->rows(); if (rows > 0 && 0 == avg_row_bytes) { avg_row_bytes = std::max((std::size_t)1, block->bytes() / rows); spill_block_batch_row_count = - (SORT_BLOCK_SPILL_BATCH_BYTES + avg_row_bytes - 1) / avg_row_bytes; + (state->spill_sort_batch_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; bool enable_spill = false; bool is_spilled = false; std::atomic_bool is_closed = false; - Status sink_status; std::shared_ptr in_mem_shared_state_sptr; std::deque sorted_streams; @@ -543,10 +563,14 @@ struct CacheSharedState : public BasicSharedState { class MultiCastDataStreamer; -struct MultiCastSharedState : public BasicSharedState { -public: - MultiCastSharedState(const RowDescriptor& row_desc, ObjectPool* pool, int cast_sender_count); +struct MultiCastSharedState : public BasicSharedState, + public BasicSpillSharedState, + public std::enable_shared_from_this { + MultiCastSharedState(const RowDescriptor& row_desc, 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; }; struct BlockRowPos { @@ -616,9 +640,18 @@ 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; @@ -746,6 +779,7 @@ struct LocalExchangeSharedState : public BasicSharedState { std::unique_ptr exchanger {}; std::vector mem_counters; std::atomic mem_usage = 0; + size_t _buffer_mem_limit = config::local_exchange_buffer_mem_limit; // We need to make sure to add mem_usage first and then enqueue, otherwise sub mem_usage may cause negative mem_usage during concurrent dequeue. std::mutex le_lock; virtual void create_dependencies(int local_exchange_id) { @@ -791,7 +825,7 @@ struct LocalExchangeSharedState : public BasicSharedState { } virtual void add_total_mem_usage(size_t delta, int channel_id) { - if (mem_usage.fetch_add(delta) + delta > config::local_exchange_buffer_mem_limit) { + if (mem_usage.fetch_add(delta) + delta > _buffer_mem_limit) { sink_deps.front()->block(); } } @@ -800,10 +834,15 @@ struct LocalExchangeSharedState : public BasicSharedState { auto prev_usage = mem_usage.fetch_sub(delta); DCHECK_GE(prev_usage - delta, 0) << "prev_usage: " << prev_usage << " delta: " << delta << " channel_id: " << channel_id; - if (prev_usage - delta <= config::local_exchange_buffer_mem_limit) { + if (prev_usage - delta <= _buffer_mem_limit) { sink_deps.front()->set_ready(); } } + + virtual void set_low_memory_mode(RuntimeState* state) { + _buffer_mem_limit = std::min(config::local_exchange_buffer_mem_limit, + state->low_memory_mode_buffer_limit()); + } }; struct LocalMergeExchangeSharedState : public LocalExchangeSharedState { @@ -849,6 +888,14 @@ struct LocalMergeExchangeSharedState : public LocalExchangeSharedState { source_deps[channel_id]->set_ready(); } + void set_low_memory_mode(RuntimeState* state) override { + _buffer_mem_limit = std::min(config::local_exchange_buffer_mem_limit, + state->low_memory_mode_buffer_limit()); + DCHECK(!_queues_mem_usage.empty()); + _each_queue_limit = + std::max(64 * 1024, _buffer_mem_limit / _queues_mem_usage.size()); + } + Dependency* get_sink_dep_by_channel_id(int channel_id) override { return sink_deps[channel_id].get(); } @@ -859,7 +906,7 @@ struct LocalMergeExchangeSharedState : public LocalExchangeSharedState { private: std::vector _queues_mem_usage; - const int64_t _each_queue_limit; + int64_t _each_queue_limit; }; #include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index 44e58535b75b71..f3afd4f41a00d3 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -24,6 +24,8 @@ #include "common/status.h" #include "pipeline/exec/operator.h" #include "runtime/primitive_type.h" +#include "runtime/thread_context.h" +#include "util/runtime_profile.h" #include "vec/common/hash_table/hash.h" #include "vec/exprs/vectorized_agg_fn.h" @@ -72,6 +74,9 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); + _memory_usage_container = ADD_COUNTER(profile(), "MemoryUsageContainer", TUnit::BYTES); + _memory_usage_arena = ADD_COUNTER(profile(), "MemoryUsageArena", TUnit::BYTES); + return Status::OK(); } @@ -173,6 +178,8 @@ Status AggSinkLocalState::_create_agg_status(vectorized::AggregateDataPtr data) Status AggSinkLocalState::_execute_without_key(vectorized::Block* block) { DCHECK(_agg_data->without_key != nullptr); SCOPED_TIMER(_build_timer); + _memory_usage_last_executing = 0; + SCOPED_PEAK_MEM(&_memory_usage_last_executing); for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { RETURN_IF_ERROR(Base::_shared_state->aggregate_evaluators[i]->execute_single_add( block, @@ -184,6 +191,8 @@ Status AggSinkLocalState::_execute_without_key(vectorized::Block* block) { } Status AggSinkLocalState::_merge_with_serialized_key(vectorized::Block* block) { + _memory_usage_last_executing = 0; + SCOPED_PEAK_MEM(&_memory_usage_last_executing); if (_shared_state->reach_limit) { return _merge_with_serialized_key_helper(block); } else { @@ -224,16 +233,20 @@ void AggSinkLocalState::_update_memusage_with_serialized_key() { }, [&](auto& agg_method) -> void { auto& data = *agg_method.hash_table; - int64_t arena_memory_usage = - _agg_arena_pool->size() + + int64_t memory_usage_arena = _agg_arena_pool->size(); + int64_t memory_usage_container = _shared_state->aggregate_data_container->memory_usage(); int64_t hash_table_memory_usage = data.get_buffer_size_in_bytes(); - COUNTER_SET(_memory_used_counter, - arena_memory_usage + hash_table_memory_usage); - - COUNTER_SET(_serialize_key_arena_memory_usage, arena_memory_usage); + COUNTER_SET(_memory_usage_arena, memory_usage_arena); + COUNTER_SET(_memory_usage_container, memory_usage_container); COUNTER_SET(_hash_table_memory_usage, hash_table_memory_usage); + COUNTER_SET(_serialize_key_arena_memory_usage, + memory_usage_arena + memory_usage_container); + + COUNTER_SET(_memory_used_counter, memory_usage_arena + + memory_usage_container + + hash_table_memory_usage); }}, _agg_data->method_variant); } @@ -383,6 +396,9 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b Status AggSinkLocalState::_merge_without_key(vectorized::Block* block) { SCOPED_TIMER(_merge_timer); DCHECK(_agg_data->without_key != nullptr); + + _memory_usage_last_executing = 0; + SCOPED_PEAK_MEM(&_memory_usage_last_executing); for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { if (Base::_shared_state->aggregate_evaluators[i]->is_merge()) { int col_id = AggSharedState::get_slot_column_id( @@ -418,6 +434,8 @@ void AggSinkLocalState::_update_memusage_without_key() { } Status AggSinkLocalState::_execute_with_serialized_key(vectorized::Block* block) { + _memory_usage_last_executing = 0; + SCOPED_PEAK_MEM(&_memory_usage_last_executing); if (_shared_state->reach_limit) { return _execute_with_serialized_key_helper(block); } else { @@ -695,6 +713,22 @@ Status AggSinkLocalState::_init_hash_method(const vectorized::VExprContextSPtrs& return Status::OK(); } +size_t AggSinkLocalState::get_reserve_mem_size(RuntimeState* state, bool eos) const { + size_t size_to_reserve = std::visit( + [&](auto&& arg) -> size_t { + using HashTableCtxType = std::decay_t; + if constexpr (std::is_same_v) { + return 0; + } else { + return arg.hash_table->estimate_memory(state->batch_size()); + } + }, + _agg_data->method_variant); + + size_to_reserve += _memory_usage_last_executing; + return size_to_reserve; +} + AggSinkOperatorX::AggSinkOperatorX(ObjectPool* pool, int operator_id, const TPlanNode& tnode, const DescriptorTbl& descs, bool require_bucket_distribution) : DataSinkOperatorX(operator_id, tnode.node_id), @@ -852,9 +886,15 @@ Status AggSinkOperatorX::reset_hash_table(RuntimeState* state) { auto& ss = *local_state.Base::_shared_state; RETURN_IF_ERROR(ss.reset_hash_table()); local_state._agg_arena_pool = ss.agg_arena_pool.get(); + local_state._serialize_key_arena_memory_usage->set((int64_t)0); return Status::OK(); } +size_t AggSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + return local_state.get_reserve_mem_size(state, eos); +} + Status AggSinkLocalState::close(RuntimeState* state, Status exec_status) { SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_close_timer); diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h index 21ee640613789e..86d20f9bf80a7a 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.h +++ b/be/src/pipeline/exec/aggregation_sink_operator.h @@ -21,6 +21,7 @@ #include "pipeline/exec/operator.h" #include "runtime/exec_env.h" +#include "util/runtime_profile.h" namespace doris::pipeline { #include "common/compile_check_begin.h" @@ -96,6 +97,8 @@ class AggSinkLocalState : public PipelineXSinkLocalState { Status _create_agg_status(vectorized::AggregateDataPtr data); size_t _memory_usage() const; + size_t get_reserve_mem_size(RuntimeState* state, bool eos) const; + RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; RuntimeProfile::Counter* _hash_table_limit_compute_timer = nullptr; @@ -107,6 +110,8 @@ class AggSinkLocalState : public PipelineXSinkLocalState { RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; RuntimeProfile::Counter* _hash_table_size_counter = nullptr; RuntimeProfile::Counter* _serialize_key_arena_memory_usage = nullptr; + RuntimeProfile::Counter* _memory_usage_container = nullptr; + RuntimeProfile::Counter* _memory_usage_arena = nullptr; bool _should_limit_output = false; @@ -120,6 +125,8 @@ class AggSinkLocalState : public PipelineXSinkLocalState { std::unique_ptr _agg_profile_arena; std::unique_ptr _executor = nullptr; + + int64_t _memory_usage_last_executing = 0; }; class AggSinkOperatorX final : public DataSinkOperatorX { @@ -158,6 +165,8 @@ class AggSinkOperatorX final : public DataSinkOperatorX { Status reset_hash_table(RuntimeState* state); + size_t get_reserve_mem_size(RuntimeState* state, bool eos) override; + using DataSinkOperatorX::node_id; using DataSinkOperatorX::operator_id; using DataSinkOperatorX::get_local_state; diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index 9feb3493068f97..2ffa444435d0ea 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -22,17 +22,15 @@ #include "common/exception.h" #include "pipeline/exec/operator.h" +#include "runtime/thread_context.h" +#include "util/runtime_profile.h" #include "vec/exprs/vectorized_agg_fn.h" +#include "vec/exprs/vexpr_fwd.h" namespace doris::pipeline { #include "common/compile_check_begin.h" -AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) - : Base(state, parent), - _get_results_timer(nullptr), - _hash_table_iterate_timer(nullptr), - _insert_keys_to_column_timer(nullptr), - _insert_values_to_column_timer(nullptr) {} +AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent) {} Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); @@ -47,28 +45,40 @@ Status AggLocalState::init(RuntimeState* state, LocalStateInfo& info) { _deserialize_data_timer = ADD_TIMER(Base::profile(), "DeserializeAndMergeTime"); _hash_table_compute_timer = ADD_TIMER(Base::profile(), "HashTableComputeTime"); _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); - _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); + _hash_table_input_counter = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "HashTableInputCount", TUnit::UNIT, 1); + _hash_table_memory_usage = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "MemoryUsageHashTable", TUnit::BYTES, 1); + _hash_table_size_counter = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "HashTableSize", TUnit::UNIT, 1); + + _memory_usage_container = ADD_COUNTER(profile(), "MemoryUsageContainer", TUnit::BYTES); + _memory_usage_arena = ADD_COUNTER(profile(), "MemoryUsageArena", TUnit::BYTES); auto& p = _parent->template cast(); if (p._without_key) { if (p._needs_finalize) { - _executor.get_result = std::bind(&AggLocalState::_get_without_key_result, this, - std::placeholders::_1, std::placeholders::_2, - std::placeholders::_3); + _executor.get_result = [this](RuntimeState* state, vectorized::Block* block, + bool* eos) { + return _get_without_key_result(state, block, eos); + }; } else { - _executor.get_result = std::bind(&AggLocalState::_get_results_without_key, this, - std::placeholders::_1, std::placeholders::_2, - std::placeholders::_3); + _executor.get_result = [this](RuntimeState* state, vectorized::Block* block, + bool* eos) { + return _get_results_without_key(state, block, eos); + }; } } else { if (p._needs_finalize) { - _executor.get_result = std::bind( - &AggLocalState::_get_with_serialized_key_result, this, std::placeholders::_1, - std::placeholders::_2, std::placeholders::_3); + _executor.get_result = [this](RuntimeState* state, vectorized::Block* block, + bool* eos) { + return _get_with_serialized_key_result(state, block, eos); + }; } else { - _executor.get_result = std::bind( - &AggLocalState::_get_results_with_serialized_key, this, std::placeholders::_1, - std::placeholders::_2, std::placeholders::_3); + _executor.get_result = [this](RuntimeState* state, vectorized::Block* block, + bool* eos) { + return _get_results_with_serialized_key(state, block, eos); + }; } } @@ -436,11 +446,11 @@ AggSourceOperatorX::AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, Status AggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); RETURN_IF_ERROR(local_state._executor.get_result(state, block, eos)); local_state.make_nullable_output_key(block); // dispose the having clause, should not be execute in prestreaming agg - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, block, block->columns())); local_state.do_agg_limit(block, eos); return Status::OK(); } @@ -474,6 +484,7 @@ void AggLocalState::make_nullable_output_key(vectorized::Block* block) { template Status AggLocalState::merge_with_serialized_key_helper(vectorized::Block* block) { SCOPED_TIMER(_merge_timer); + SCOPED_PEAK_MEM(&_estimate_memory_usage); size_t key_size = Base::_shared_state->probe_expr_ctxs.size(); vectorized::ColumnRawPtrs key_columns(key_size); @@ -561,52 +572,79 @@ template Status AggSourceOperatorX::merge_with_serialized_key_helper( template Status AggSourceOperatorX::merge_with_serialized_key_helper( RuntimeState* state, vectorized::Block* block); +size_t AggSourceOperatorX::get_estimated_memory_size_for_merging(RuntimeState* state, + size_t rows) const { + auto& local_state = get_local_state(state); + size_t size = std::visit( + vectorized::Overload { + [&](std::monostate& arg) -> size_t { + throw doris::Exception(ErrorCode::INTERNAL_ERROR, "uninited hash table"); + return 0; + }, + [&](auto& agg_method) { return agg_method.hash_table->estimate_memory(rows); }}, + local_state._shared_state->agg_data->method_variant); + size += local_state._shared_state->aggregate_data_container->estimate_memory(rows); + return size; +} + void AggLocalState::_emplace_into_hash_table(vectorized::AggregateDataPtr* places, vectorized::ColumnRawPtrs& key_columns, size_t num_rows) { - std::visit(vectorized::Overload { - [&](std::monostate& arg) -> void { - throw doris::Exception(ErrorCode::INTERNAL_ERROR, "uninited hash table"); - }, - [&](auto& agg_method) -> void { - SCOPED_TIMER(_hash_table_compute_timer); - using HashMethodType = std::decay_t; - using AggState = typename HashMethodType::State; - AggState state(key_columns); - agg_method.init_serialized_keys(key_columns, num_rows); - - auto creator = [this](const auto& ctor, auto& key, auto& origin) { - HashMethodType::try_presis_key_and_origin( - key, origin, *_shared_state->agg_arena_pool); - auto mapped = - Base::_shared_state->aggregate_data_container->append_data( - origin); - auto st = _create_agg_status(mapped); - if (!st) { - throw Exception(st.code(), st.to_string()); - } - ctor(key, mapped); - }; - - auto creator_for_null_key = [&](auto& mapped) { - mapped = _shared_state->agg_arena_pool->aligned_alloc( - _shared_state->total_size_of_aggregate_states, - _shared_state->align_aggregate_states); - auto st = _create_agg_status(mapped); - if (!st) { - throw Exception(st.code(), st.to_string()); - } - }; - - SCOPED_TIMER(_hash_table_emplace_timer); - for (size_t i = 0; i < num_rows; ++i) { - places[i] = *agg_method.lazy_emplace(state, i, creator, - creator_for_null_key); - } - - COUNTER_UPDATE(_hash_table_input_counter, num_rows); - }}, - _shared_state->agg_data->method_variant); + std::visit( + vectorized::Overload { + [&](std::monostate& arg) -> void { + throw doris::Exception(ErrorCode::INTERNAL_ERROR, "uninited hash table"); + }, + [&](auto& agg_method) -> void { + SCOPED_TIMER(_hash_table_compute_timer); + using HashMethodType = std::decay_t; + using AggState = typename HashMethodType::State; + AggState state(key_columns); + agg_method.init_serialized_keys(key_columns, num_rows); + + auto creator = [this](const auto& ctor, auto& key, auto& origin) { + HashMethodType::try_presis_key_and_origin( + key, origin, *_shared_state->agg_arena_pool); + auto mapped = + Base::_shared_state->aggregate_data_container->append_data( + origin); + auto st = _create_agg_status(mapped); + if (!st) { + throw Exception(st.code(), st.to_string()); + } + ctor(key, mapped); + }; + + auto creator_for_null_key = [&](auto& mapped) { + mapped = _shared_state->agg_arena_pool->aligned_alloc( + _shared_state->total_size_of_aggregate_states, + _shared_state->align_aggregate_states); + auto st = _create_agg_status(mapped); + if (!st) { + throw Exception(st.code(), st.to_string()); + } + }; + + SCOPED_TIMER(_hash_table_emplace_timer); + for (size_t i = 0; i < num_rows; ++i) { + places[i] = *agg_method.lazy_emplace(state, i, creator, + creator_for_null_key); + } + + COUNTER_UPDATE(_hash_table_input_counter, num_rows); + COUNTER_SET(_hash_table_memory_usage, + static_cast( + agg_method.hash_table->get_buffer_size_in_bytes())); + COUNTER_SET(_hash_table_size_counter, + static_cast(agg_method.hash_table->size())); + COUNTER_SET( + _memory_usage_container, + static_cast( + _shared_state->aggregate_data_container->memory_usage())); + COUNTER_SET(_memory_usage_arena, + static_cast(_shared_state->agg_arena_pool->size())); + }}, + _shared_state->agg_data->method_variant); } void AggLocalState::_find_in_hash_table(vectorized::AggregateDataPtr* places, diff --git a/be/src/pipeline/exec/aggregation_source_operator.h b/be/src/pipeline/exec/aggregation_source_operator.h index 6de2bf93dbc758..79b9c26f224de4 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.h +++ b/be/src/pipeline/exec/aggregation_source_operator.h @@ -79,8 +79,12 @@ class AggLocalState final : public PipelineXLocalState { RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; RuntimeProfile::Counter* _hash_table_input_counter = nullptr; + RuntimeProfile::Counter* _hash_table_size_counter = nullptr; + RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; RuntimeProfile::Counter* _merge_timer = nullptr; RuntimeProfile::Counter* _deserialize_data_timer = nullptr; + RuntimeProfile::Counter* _memory_usage_container = nullptr; + RuntimeProfile::Counter* _memory_usage_arena = nullptr; using vectorized_get_result = std::function; @@ -97,7 +101,7 @@ class AggSourceOperatorX : public OperatorX { using Base = OperatorX; AggSourceOperatorX(ObjectPool* pool, const TPlanNode& tnode, int operator_id, const DescriptorTbl& descs); - ~AggSourceOperatorX() = default; + ~AggSourceOperatorX() override = default; Status get_block(RuntimeState* state, vectorized::Block* block, bool* eos) override; @@ -106,6 +110,8 @@ class AggSourceOperatorX : public OperatorX { template Status merge_with_serialized_key_helper(RuntimeState* state, vectorized::Block* block); + size_t get_estimated_memory_size_for_merging(RuntimeState* state, size_t rows) const; + private: friend class AggLocalState; diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index 7cc25eef9446d6..549b080e9104a2 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -21,6 +21,7 @@ #include #include "pipeline/exec/operator.h" +#include "runtime/runtime_state.h" #include "vec/exprs/vectorized_agg_fn.h" namespace doris::pipeline { @@ -266,6 +267,10 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)input_block->rows()); + + local_state._reserve_mem_size = 0; + SCOPED_PEAK_MEM(&local_state._reserve_mem_size); + local_state._shared_state->input_eos = eos; if (local_state._shared_state->input_eos && input_block->rows() == 0) { local_state._dependency->set_ready_to_read(); @@ -335,6 +340,11 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block return Status::OK(); } +size_t AnalyticSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + return local_state._reserve_mem_size; +} + Status AnalyticSinkOperatorX::_insert_range_column(vectorized::Block* block, const vectorized::VExprContextSPtr& expr, vectorized::IColumn* dst_column, size_t length) { diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index 0ff7c4e4e047bd..67ef7df783d120 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -65,6 +65,8 @@ class AnalyticSinkLocalState : public PipelineXSinkLocalState _agg_expr_ctxs; vectorized::VExprContextSPtrs _partition_by_eq_expr_ctxs; vectorized::VExprContextSPtrs _order_by_eq_expr_ctxs; + + int64_t _reserve_mem_size = 0; }; class AnalyticSinkOperatorX final : public DataSinkOperatorX { @@ -93,6 +95,8 @@ class AnalyticSinkOperatorX final : public DataSinkOperatorXinput_eos && (local_state._output_block_index == local_state._shared_state->input_blocks.size() || local_state._shared_state->input_total_rows == 0)) { @@ -551,8 +552,7 @@ Status AnalyticSourceOperatorX::get_block(RuntimeState* state, vectorized::Block } } RETURN_IF_ERROR(local_state.output_current_block(block)); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, block, block->columns())); local_state.reached_limit(block, eos); return Status::OK(); } diff --git a/be/src/pipeline/exec/assert_num_rows_operator.cpp b/be/src/pipeline/exec/assert_num_rows_operator.cpp index 345e42b7d96837..1a72e56a8a499f 100644 --- a/be/src/pipeline/exec/assert_num_rows_operator.cpp +++ b/be/src/pipeline/exec/assert_num_rows_operator.cpp @@ -43,6 +43,7 @@ Status AssertNumRowsOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc bool* eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); + SCOPED_PEAK_MEM(&local_state.estimate_memory_usage()); local_state.add_num_rows_returned(block->rows()); int64_t num_rows_returned = local_state.num_rows_returned(); bool assert_res = false; @@ -110,13 +111,17 @@ Status AssertNumRowsOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc return it->second; } }; - LOG(INFO) << "Expected " << to_string_lambda(_assertion) << " " << _desired_num_rows - << " to be returned by expression " << _subquery_string; - return Status::Cancelled("Expected {} {} to be returned by expression {}", - to_string_lambda(_assertion), _desired_num_rows, _subquery_string); + LOG(WARNING) << "Expected " << to_string_lambda(_assertion) << " " << _desired_num_rows + << " to be returned by expression " << _subquery_string + << ", actual returned: " << num_rows_returned << ", node id: " << _node_id + << ", child id: " << _child->node_id(); + return Status::Cancelled( + "AssertOperator(node id: {}) Expected {} {} to be returned by expression {}(actual " + "rows: {}) ", + _node_id, to_string_lambda(_assertion), _desired_num_rows, num_rows_returned, + _subquery_string); } - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, block, block->columns())); return Status::OK(); } diff --git a/be/src/pipeline/exec/cache_source_operator.cpp b/be/src/pipeline/exec/cache_source_operator.cpp index cace8465fc2d46..b515aeb495751e 100644 --- a/be/src/pipeline/exec/cache_source_operator.cpp +++ b/be/src/pipeline/exec/cache_source_operator.cpp @@ -111,7 +111,7 @@ std::string CacheSourceLocalState::debug_string(int indentation_level) const { if (_shared_state) { fmt::format_to(debug_string_buffer, ", data_queue: (is_all_finish = {}, has_data = {})", _shared_state->data_queue.is_all_finish(), - _shared_state->data_queue.remaining_has_data()); + _shared_state->data_queue.has_more_data()); } return fmt::to_string(debug_string_buffer); } diff --git a/be/src/pipeline/exec/data_queue.cpp b/be/src/pipeline/exec/data_queue.cpp index 436a98e6b0369e..85354ece76af85 100644 --- a/be/src/pipeline/exec/data_queue.cpp +++ b/be/src/pipeline/exec/data_queue.cpp @@ -72,17 +72,6 @@ void DataQueue::push_free_block(std::unique_ptr block, int ch _free_blocks[child_idx].emplace_back(std::move(block)); } -//use sink to check can_write -bool DataQueue::has_enough_space_to_push() { - DCHECK(_cur_bytes_in_queue.size() == 1); - return _cur_bytes_in_queue[0].load() < MAX_BYTE_OF_QUEUE / 2; -} - -//use source to check can_read -bool DataQueue::has_data_or_finished(int child_idx) { - return remaining_has_data() || _is_finished[child_idx]; -} - //check which queue have data, and save the idx in _flag_queue_idx, //so next loop, will check the record idx + 1 first //maybe it's useful with many queue, others maybe always 0 diff --git a/be/src/pipeline/exec/data_queue.h b/be/src/pipeline/exec/data_queue.h index d97f58c0debdb6..aabe0ba8797726 100644 --- a/be/src/pipeline/exec/data_queue.h +++ b/be/src/pipeline/exec/data_queue.h @@ -53,10 +53,11 @@ class DataQueue { bool is_finish(int child_idx = 0); bool is_all_finish(); - bool has_enough_space_to_push(); - bool has_data_or_finished(int child_idx = 0); + // This function is not thread safe, should be called in Operator::get_block() bool remaining_has_data(); + bool has_more_data() const { return _cur_blocks_total_nums.load() > 0; } + int64_t max_bytes_in_queue() const { return _max_bytes_in_queue; } int64_t max_size_of_queue() const { return _max_size_of_queue; } @@ -73,6 +74,8 @@ class DataQueue { void set_max_blocks_in_sub_queue(int64_t max_blocks) { _max_blocks_in_sub_queue = max_blocks; } + void set_low_memory_mode() { _max_blocks_in_sub_queue = 1; } + private: std::vector> _queue_blocks_lock; std::vector>> _queue_blocks; @@ -101,7 +104,7 @@ class DataQueue { //this only use to record the queue[0] for profile int64_t _max_bytes_in_queue = 0; int64_t _max_size_of_queue = 0; - static constexpr int64_t MAX_BYTE_OF_QUEUE = 1024l * 1024 * 1024 / 10; + static constexpr int64_t MAX_BYTE_OF_QUEUE = 1024L * 1024 * 1024 / 10; // data queue is multi sink one source std::shared_ptr _source_dependency = nullptr; diff --git a/be/src/pipeline/exec/datagen_operator.cpp b/be/src/pipeline/exec/datagen_operator.cpp index d400953799e5bb..d82f074118b422 100644 --- a/be/src/pipeline/exec/datagen_operator.cpp +++ b/be/src/pipeline/exec/datagen_operator.cpp @@ -70,6 +70,7 @@ Status DataGenSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* RETURN_IF_CANCELLED(state); auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); + SCOPED_PEAK_MEM(&local_state.estimate_memory_usage()); { SCOPED_TIMER(local_state._table_function_execution_timer); RETURN_IF_ERROR(local_state._table_func->get_next(state, block, eos)); diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp index bb282fd118e5c0..f58f679e3e670d 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp @@ -195,6 +195,10 @@ Status DistinctStreamingAggLocalState::_distinct_pre_agg_with_serialized_key( size_t rows = in_block->rows(); _distinct_row.clear(); _distinct_row.reserve(rows); + if (_parent->cast()._is_streaming_preagg && + state()->get_query_ctx()->low_memory_mode()) { + _stop_emplace_flag = true; + } if (!_stop_emplace_flag) { _emplace_into_hash_table_to_distinct(_distinct_row, key_columns, rows); @@ -449,8 +453,7 @@ Status DistinctStreamingAggOperatorX::pull(RuntimeState* state, vectorized::Bloc local_state._make_nullable_output_key(block); if (!_is_streaming_preagg) { // dispose the having clause, should not be execute in prestreaming agg - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, block, block->columns())); } local_state.add_num_rows_returned(block->rows()); // If the limit is not reached, it is important to ensure that _aggregated_block is empty diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 800ef6150738d6..3fb460c9cc7408 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -68,8 +68,8 @@ void BroadcastPBlockHolderMemLimiter::acquire(BroadcastPBlockHolder& holder) { auto size = holder._pblock->column_values().size(); _total_queue_buffer_size += size; _total_queue_blocks_count++; - if (_total_queue_buffer_size >= config::exchg_node_buffer_size_bytes || - _total_queue_blocks_count >= config::num_broadcast_buffer) { + if (_total_queue_buffer_size >= _total_queue_buffer_size_limit || + _total_queue_blocks_count >= _total_queue_blocks_count_limit) { _broadcast_dependency->block(); } } @@ -89,13 +89,14 @@ void BroadcastPBlockHolderMemLimiter::release(const BroadcastPBlockHolder& holde namespace pipeline { ExchangeSinkBuffer::ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, - RuntimeState* state, + PlanNodeId node_id, RuntimeState* state, const std::vector& sender_ins_ids) : HasTaskExecutionCtx(state), _queue_capacity(0), _is_failed(false), _query_id(std::move(query_id)), _dest_node_id(dest_node_id), + _node_id(node_id), _state(state), _context(state->get_query_ctx()), _exchange_sink_num(sender_ins_ids.size()) { @@ -434,6 +435,8 @@ void ExchangeSinkBuffer::_ended(InstanceLoId id) { void ExchangeSinkBuffer::_failed(InstanceLoId id, const std::string& err) { _is_failed = true; + LOG(INFO) << "send rpc failed, instance id: " << id << ", _dest_node_id: " << _dest_node_id + << ", node id: " << _node_id << ", err: " << err; _context->cancel(Status::Cancelled(err)); } diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index 458c7c3f66e3ee..899a2991110353 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -85,14 +85,23 @@ class BroadcastPBlockHolderMemLimiter public: BroadcastPBlockHolderMemLimiter() = delete; - BroadcastPBlockHolderMemLimiter(std::shared_ptr& broadcast_dependency) { + BroadcastPBlockHolderMemLimiter(std::shared_ptr& broadcast_dependency) + : _total_queue_buffer_size_limit(config::exchg_node_buffer_size_bytes), + _total_queue_blocks_count_limit(config::num_broadcast_buffer) { _broadcast_dependency = broadcast_dependency; } + void set_low_memory_mode() { + _total_queue_buffer_size_limit = 1024 * 1024; + _total_queue_blocks_count_limit = 8; + } + void acquire(BroadcastPBlockHolder& holder); void release(const BroadcastPBlockHolder& holder); private: + std::atomic_int64_t _total_queue_buffer_size_limit {0}; + std::atomic_int64_t _total_queue_blocks_count_limit {0}; std::atomic_int64_t _total_queue_buffer_size {0}; std::atomic_int64_t _total_queue_blocks_count {0}; std::shared_ptr _broadcast_dependency; @@ -215,13 +224,13 @@ void transmit_blockv2(PBackendService_Stub& stub, #endif class ExchangeSinkBuffer : public HasTaskExecutionCtx { public: - ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, RuntimeState* state, - const std::vector& sender_ins_ids); - + ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, PlanNodeId node_id, + RuntimeState* state, const std::vector& sender_ins_ids); #ifdef BE_TEST ExchangeSinkBuffer(RuntimeState* state, int64_t sinknum) : HasTaskExecutionCtx(state), _exchange_sink_num(sinknum) {}; #endif + ~ExchangeSinkBuffer() override = default; void construct_request(TUniqueId); @@ -239,6 +248,8 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx { _queue_deps[sender_ins_id] = queue_dependency; _parents[sender_ins_id] = local_state; } + + void set_low_memory_mode() { _queue_capacity = 8; } #ifdef BE_TEST public: #else @@ -251,7 +262,7 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx { // store data in non-broadcast shuffle phmap::flat_hash_map>> _instance_to_package_queue; - size_t _queue_capacity; + std::atomic _queue_capacity; // store data in broadcast shuffle phmap::flat_hash_map>> @@ -284,6 +295,8 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx { std::atomic _is_failed; PUniqueId _query_id; PlanNodeId _dest_node_id; + + PlanNodeId _node_id; std::atomic _rpc_count = 0; RuntimeState* _state = nullptr; QueryContext* _context = nullptr; diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index cc789f6e25b20b..864f50c8ae0b13 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -179,7 +179,7 @@ void ExchangeSinkLocalState::on_channel_finished(InstanceLoId channel_id) { std::lock_guard lock(_finished_channels_mutex); if (_finished_channels.contains(channel_id)) { - LOG(WARNING) << "query: " << print_id(_state->query_id()) + LOG(WARNING) << "Query: " << print_id(_state->query_id()) << ", on_channel_finished on already finished channel: " << channel_id; return; } else { @@ -355,6 +355,11 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block return Status::EndOfFile("all data stream channels EOF"); } + // When `local_state.only_local_exchange` the `sink_buffer` is nullptr. + if (state->get_query_ctx()->low_memory_mode() && local_state._sink_buffer != nullptr) { + local_state._sink_buffer->set_low_memory_mode(); + } + if (_part_type == TPartitionType::UNPARTITIONED || local_state.channels.size() == 1) { // 1. serialize depends on it is not local exchange // 2. send block @@ -393,6 +398,9 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block block_holder->reset_block(); } + if (state->get_query_ctx()->low_memory_mode()) { + local_state._broadcast_pb_mem_limiter->set_low_memory_mode(); + } local_state._broadcast_pb_mem_limiter->acquire(*block_holder); size_t idx = 0; @@ -558,8 +566,8 @@ std::shared_ptr ExchangeSinkOperatorX::_create_buffer( PUniqueId id; id.set_hi(_state->query_id().hi); id.set_lo(_state->query_id().lo); - auto sink_buffer = - std::make_unique(id, _dest_node_id, state(), sender_ins_ids); + auto sink_buffer = std::make_unique(id, _dest_node_id, _node_id, state(), + sender_ins_ids); for (const auto& _dest : _dests) { sink_buffer->construct_request(_dest.fragment_instance_id); } diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp index dbde9abd05dc34..6a1157f6da6237 100644 --- a/be/src/pipeline/exec/exchange_source_operator.cpp +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -17,6 +17,8 @@ #include "exchange_source_operator.h" +#include + #include #include @@ -72,7 +74,7 @@ Status ExchangeLocalState::init(RuntimeState* state, LocalStateInfo& info) { _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, timer_name, 1); for (size_t i = 0; i < queues.size(); i++) { deps[i] = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), - "SHUFFLE_DATA_DEPENDENCY"); + fmt::format("SHUFFLE_DATA_DEPENDENCY_{}", i)); queues[i]->set_dependency(deps[i]); metrics[i] = _runtime_profile->add_nonzero_counter(fmt::format("WaitForData{}", i), TUnit ::TIME_NS, timer_name, 1); @@ -165,6 +167,7 @@ Status ExchangeSourceOperatorX::get_block(RuntimeState* state, vectorized::Block RETURN_IF_ERROR(doris::vectorized::VExprContext::filter_block(local_state.conjuncts(), block, block->columns())); } + // In vsortrunmerger, it will set eos=true, and block not empty // so that eos==true, could not make sure that block not have valid data if (!*eos || block->rows() > 0) { diff --git a/be/src/pipeline/exec/file_scan_operator.cpp b/be/src/pipeline/exec/file_scan_operator.cpp index 7afbb29134c079..1571b585545879 100644 --- a/be/src/pipeline/exec/file_scan_operator.cpp +++ b/be/src/pipeline/exec/file_scan_operator.cpp @@ -60,9 +60,20 @@ std::string FileScanLocalState::name_suffix() const { void FileScanLocalState::set_scan_ranges(RuntimeState* state, const std::vector& scan_ranges) { + auto wg_ptr = state->get_query_ctx()->workload_group(); _max_scanners = config::doris_scanner_thread_pool_thread_num / state->query_parallel_instance_num(); - _max_scanners = std::max(std::max(_max_scanners, state->parallel_scan_max_scanners_count()), 1); + if (wg_ptr && !state->get_query_ctx()->enable_mem_overcommit()) { + const auto total_slots = wg_ptr->total_query_slot_count(); + const auto query_slots = state->get_query_ctx()->get_slot_count(); + _max_scanners = _max_scanners * query_slots / total_slots; + } + + const auto parallel_scan_max_scanners_count = state->parallel_scan_max_scanners_count(); + if (parallel_scan_max_scanners_count > 0) { + _max_scanners = + std::max(std::min(_max_scanners, state->parallel_scan_max_scanners_count()), 1); + } // For select * from table limit 10; should just use one thread. if (should_run_serial()) { _max_scanners = 1; @@ -82,7 +93,7 @@ void FileScanLocalState::set_scan_ranges(RuntimeState* state, std::make_shared(scan_ranges, _max_scanners); } _max_scanners = std::min(_max_scanners, _split_source->num_scan_ranges()); - if (scan_ranges.size() > 0 && + if (!scan_ranges.empty() && scan_ranges[0].scan_range.ext_scan_range.file_scan_range.__isset.params) { // for compatibility. // in new implement, the tuple id is set in prepare phase diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 016ea494062303..da28d3331ef346 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -20,10 +20,15 @@ #include #include +#include "common/cast_set.h" +#include "common/exception.h" #include "exprs/bloom_filter_func.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/operator.h" #include "pipeline/pipeline_task.h" +#include "util/pretty_printer.h" +#include "vec/columns/column_nullable.h" +#include "vec/core/block.h" #include "vec/data_types/data_type_nullable.h" #include "vec/utils/template_helpers.hpp" @@ -118,6 +123,93 @@ Status HashJoinBuildSinkLocalState::open(RuntimeState* state) { return Status::OK(); } +size_t HashJoinBuildSinkLocalState::get_reserve_mem_size(RuntimeState* state, bool eos) { + if (!_should_build_hash_table) { + return 0; + } + + if (_shared_state->build_block) { + return 0; + } + + size_t size_to_reserve = 0; + + const size_t build_block_rows = _build_side_mutable_block.rows(); + if (build_block_rows != 0) { + const auto bytes = _build_side_mutable_block.bytes(); + const auto allocated_bytes = _build_side_mutable_block.allocated_bytes(); + const auto bytes_per_row = bytes / build_block_rows; + const auto estimated_size_of_next_block = bytes_per_row * state->batch_size(); + // If the new size is greater than 85% of allocalted bytes, it maybe need to realloc. + if (((estimated_size_of_next_block + bytes) * 100 / allocated_bytes) >= 85) { + size_to_reserve += static_cast(static_cast(allocated_bytes) * 1.15); + } + } + + if (eos) { + const size_t rows = build_block_rows + state->batch_size(); + const auto bucket_size = JoinHashTable::calc_bucket_size(rows); + + size_to_reserve += bucket_size * sizeof(uint32_t); // JoinHashTable::first + size_to_reserve += rows * sizeof(uint32_t); // JoinHashTable::next + + auto& p = _parent->cast(); + if (p._join_op == TJoinOp::FULL_OUTER_JOIN || p._join_op == TJoinOp::RIGHT_OUTER_JOIN || + p._join_op == TJoinOp::RIGHT_ANTI_JOIN || p._join_op == TJoinOp::RIGHT_SEMI_JOIN) { + size_to_reserve += rows * sizeof(uint8_t); // JoinHashTable::visited + } + size_to_reserve += _evaluate_mem_usage; + + vectorized::ColumnRawPtrs raw_ptrs(_build_expr_ctxs.size()); + + if (build_block_rows > 0) { + auto block = _build_side_mutable_block.to_block(); + std::vector converted_columns; + Defer defer([&]() { + for (auto i : converted_columns) { + auto& data = block.get_by_position(i); + data.column = vectorized::remove_nullable(data.column); + data.type = vectorized::remove_nullable(data.type); + } + _build_side_mutable_block = vectorized::MutableBlock(std::move(block)); + }); + vectorized::ColumnUInt8::MutablePtr null_map_val; + if (p._join_op == TJoinOp::LEFT_OUTER_JOIN || p._join_op == TJoinOp::FULL_OUTER_JOIN) { + converted_columns = _convert_block_to_null(block); + // first row is mocked + for (int i = 0; i < block.columns(); i++) { + auto [column, is_const] = unpack_if_const(block.safe_get_by_position(i).column); + assert_cast(column->assume_mutable().get()) + ->get_null_map_column() + .get_data() + .data()[0] = 1; + } + } + + null_map_val = vectorized::ColumnUInt8::create(); + null_map_val->get_data().assign(build_block_rows, (uint8_t)0); + + // Get the key column that needs to be built + Status st = _extract_join_column(block, null_map_val, raw_ptrs, _build_col_ids); + if (!st.ok()) { + throw Exception(st); + } + + std::visit(vectorized::Overload {[&](std::monostate& arg) { + LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); + }, + [&](auto&& hash_map_context) { + size_to_reserve += hash_map_context.estimated_size( + raw_ptrs, block.rows(), true, true, + bucket_size); + }}, + _shared_state->hash_table_variants->method_variant); + } + } + return size_to_reserve; +} + Status HashJoinBuildSinkLocalState::close(RuntimeState* state, Status exec_status) { if (_closed) { return Status::OK(); @@ -247,6 +339,7 @@ Status HashJoinBuildSinkLocalState::_do_evaluate(vectorized::Block& block, vectorized::VExprContextSPtrs& exprs, RuntimeProfile::Counter& expr_call_timer, std::vector& res_col_ids) { + auto origin_size = block.allocated_bytes(); for (size_t i = 0; i < exprs.size(); ++i) { int result_col_id = -1; // execute build column @@ -260,6 +353,8 @@ Status HashJoinBuildSinkLocalState::_do_evaluate(vectorized::Block& block, block.get_by_position(result_col_id).column->convert_to_full_column_if_const(); res_col_ids[i] = result_col_id; } + + _evaluate_mem_usage = block.allocated_bytes() - origin_size; return Status::OK(); } @@ -310,6 +405,11 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, if (UNLIKELY(rows == 0)) { return Status::OK(); } + + LOG(INFO) << "build block rows: " << block.rows() << ", columns count: " << block.columns() + << ", bytes/allocated_bytes: " << PrettyPrinter::print_bytes(block.bytes()) << "/" + << PrettyPrinter::print_bytes(block.allocated_bytes()); + block.replace_if_overflow(); vectorized::ColumnRawPtrs raw_ptrs(_build_expr_ctxs.size()); @@ -334,9 +434,9 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, } // Get the key column that needs to be built - Status st = _extract_join_column(block, null_map_val, raw_ptrs, _build_col_ids); + RETURN_IF_ERROR(_extract_join_column(block, null_map_val, raw_ptrs, _build_col_ids)); - st = std::visit( + Status st = std::visit( vectorized::Overload { [&](std::monostate& arg, auto join_op, auto short_circuit_for_null_in_build_side, @@ -363,7 +463,6 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, _shared_state->hash_table_variants->method_variant, _shared_state->join_op_variants, vectorized::make_bool_variant(p._short_circuit_for_null_in_build_side), vectorized::make_bool_variant((p._have_other_join_conjunct))); - return st; } @@ -619,4 +718,22 @@ Status HashJoinBuildSinkOperatorX::sink(RuntimeState* state, vectorized::Block* return Status::OK(); } +size_t HashJoinBuildSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + return local_state.get_reserve_mem_size(state, eos); +} + +size_t HashJoinBuildSinkOperatorX::get_memory_usage(RuntimeState* state) const { + auto& local_state = get_local_state(state); + return local_state._memory_used_counter->value(); +} + +std::string HashJoinBuildSinkOperatorX::get_memory_usage_debug_str(RuntimeState* state) const { + auto& local_state = get_local_state(state); + return fmt::format("build block: {}, hash table: {}, build key arena: {}", + PrettyPrinter::print_bytes(local_state._build_blocks_memory_usage->value()), + PrettyPrinter::print_bytes(local_state._hash_table_memory_usage->value()), + PrettyPrinter::print_bytes(local_state._build_arena_memory_usage->value())); +} + } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index 91465380d70348..7c6f4100fd24cf 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -56,6 +56,8 @@ class HashJoinBuildSinkLocalState final Status disable_runtime_filters(RuntimeState* state); + [[nodiscard]] size_t get_reserve_mem_size(RuntimeState* state, bool eos); + protected: Status _hash_table_init(RuntimeState* state); void _set_build_side_has_external_nullmap(vectorized::Block& block, @@ -79,6 +81,7 @@ class HashJoinBuildSinkLocalState final bool _should_build_hash_table = true; bool _runtime_filters_disabled = false; + size_t _evaluate_mem_usage = 0; size_t _build_side_rows = 0; @@ -122,6 +125,12 @@ class HashJoinBuildSinkOperatorX final Status sink(RuntimeState* state, vectorized::Block* in_block, bool eos) override; + size_t get_reserve_mem_size(RuntimeState* state, bool eos) override; + + [[nodiscard]] size_t get_memory_usage(RuntimeState* state) const; + + std::string get_memory_usage_debug_str(RuntimeState* state) const; + bool should_dry_run(RuntimeState* state) override { return _is_broadcast_join && !state->get_sink_local_state() ->cast() diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index 1f30a6183a22b9..0dd85c00f5847c 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -338,6 +338,7 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc return st; } + local_state._estimate_memory_usage += temp_block.allocated_bytes(); RETURN_IF_ERROR( local_state.filter_data_and_build_output(state, output_block, eos, &temp_block)); // Here make _join_block release the columns' ptr @@ -424,8 +425,7 @@ Status HashJoinProbeLocalState::filter_data_and_build_output(RuntimeState* state } { SCOPED_TIMER(_join_filter_timer); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, temp_block, - temp_block->columns())); + RETURN_IF_ERROR(filter_block(_conjuncts, temp_block, temp_block->columns())); } RETURN_IF_ERROR(_build_output_block(temp_block, output_block, false)); @@ -466,8 +466,12 @@ Status HashJoinProbeOperatorX::push(RuntimeState* state, vectorized::Block* inpu auto& local_state = get_local_state(state); local_state.prepare_for_next(); local_state._probe_eos = eos; - if (input_block->rows() > 0) { - COUNTER_UPDATE(local_state._probe_rows_counter, input_block->rows()); + + const auto rows = input_block->rows(); + size_t origin_size = input_block->allocated_bytes(); + + if (rows > 0) { + COUNTER_UPDATE(local_state._probe_rows_counter, rows); std::vector res_col_ids(local_state._probe_expr_ctxs.size()); RETURN_IF_ERROR(_do_evaluate(*input_block, local_state._probe_expr_ctxs, *local_state._probe_expr_call_timer, res_col_ids)); @@ -478,6 +482,8 @@ Status HashJoinProbeOperatorX::push(RuntimeState* state, vectorized::Block* inpu RETURN_IF_ERROR(local_state._extract_join_column(*input_block, res_col_ids)); + local_state._estimate_memory_usage += (input_block->allocated_bytes() - origin_size); + if (&local_state._probe_block != input_block) { input_block->swap(local_state._probe_block); COUNTER_SET(local_state._memory_used_counter, diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index 55a8835f55b9cf..e3aab7b6bd4f24 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -115,6 +115,8 @@ class HashJoinProbeLocalState final std::unique_ptr _process_hashtable_ctx_variants = std::make_unique(); + ssize_t _estimated_mem_in_push = -1; + RuntimeProfile::Counter* _probe_expr_call_timer = nullptr; RuntimeProfile::Counter* _probe_side_output_timer = nullptr; RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage = nullptr; diff --git a/be/src/pipeline/exec/join/process_hash_table_probe.h b/be/src/pipeline/exec/join/process_hash_table_probe.h index 052ed5875ae296..7305da5f3fe98d 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe.h @@ -72,7 +72,7 @@ struct ProcessHashTableProbe { // each matching join column need to be processed by other join conjunct. so the struct of mutable block // and output block may be different // The output result is determined by the other join conjunct result and same_to_prev struct - Status do_other_join_conjuncts(vectorized::Block* output_block, std::vector& visited, + Status do_other_join_conjuncts(vectorized::Block* output_block, DorisVector& visited, bool has_null_in_build_side); template diff --git a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h index 8c5b051b684bc1..17e0f7330261fa 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h @@ -479,7 +479,7 @@ Status ProcessHashTableProbe::do_mark_join_conjuncts(vectorized::Blo template Status ProcessHashTableProbe::do_other_join_conjuncts(vectorized::Block* output_block, - std::vector& visited, + DorisVector& visited, bool has_null_in_build_side) { // dispose the other join conjunct exec auto row_count = output_block->rows(); diff --git a/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp b/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp index eb72e9601e1acf..03e63c507bdf29 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.cpp @@ -23,9 +23,9 @@ namespace doris::pipeline { #include "common/compile_check_begin.h" std::string MultiCastDataStreamSinkLocalState::name_suffix() { - auto& sinks = static_cast(_parent)->sink_node().sinks; + const auto& sinks = static_cast(_parent)->sink_node().sinks; std::string id_name = " (dst id : "; - for (auto& sink : sinks) { + for (const auto& sink : sinks) { id_name += std::to_string(sink.dest_node_id) + ","; } id_name += ")"; @@ -34,19 +34,39 @@ std::string MultiCastDataStreamSinkLocalState::name_suffix() { std::shared_ptr MultiCastDataStreamSinkOperatorX::create_shared_state() const { std::shared_ptr ss = - std::make_shared(_row_desc, _pool, _cast_sender_count); + std::make_shared(_row_desc, _pool, _cast_sender_count, _node_id); ss->id = operator_id(); - for (auto& dest : dests_id()) { + for (const auto& dest : dests_id()) { ss->related_op_ids.insert(dest); } return ss; } +std::vector MultiCastDataStreamSinkLocalState::dependencies() const { + auto dependencies = Base::dependencies(); + dependencies.emplace_back(_shared_state->multi_cast_data_streamer->get_spill_dependency()); + return dependencies; +} + +Status MultiCastDataStreamSinkLocalState::open(RuntimeState* state) { + RETURN_IF_ERROR(Base::open(state)); + _shared_state->multi_cast_data_streamer->set_sink_profile(profile()); + _shared_state->setup_shared_profile(profile()); + _shared_state->multi_cast_data_streamer->set_write_dependency(_dependency); + return Status::OK(); +} + +std::string MultiCastDataStreamSinkLocalState::debug_string(int indentation_level) const { + fmt::memory_buffer debug_string_buffer; + fmt::format_to(debug_string_buffer, "{}, {}", Base::debug_string(indentation_level), + _shared_state->multi_cast_data_streamer->debug_string()); + return fmt::to_string(debug_string_buffer); +} + Status MultiCastDataStreamSinkOperatorX::sink(RuntimeState* state, vectorized::Block* in_block, bool eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); - COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); if (in_block->rows() > 0 || eos) { COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); auto st = local_state._shared_state->multi_cast_data_streamer->push(state, in_block, eos); diff --git a/be/src/pipeline/exec/multi_cast_data_stream_sink.h b/be/src/pipeline/exec/multi_cast_data_stream_sink.h index 9d69b3fb5bdc9e..d36088ce0491f2 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_sink.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_sink.h @@ -17,23 +17,33 @@ #pragma once +#include + +#include "common/status.h" #include "operator.h" +#include "pipeline/exec/data_queue.h" namespace doris::pipeline { #include "common/compile_check_begin.h" class MultiCastDataStreamSinkOperatorX; class MultiCastDataStreamSinkLocalState final - : public PipelineXSinkLocalState { + : public PipelineXSpillSinkLocalState { ENABLE_FACTORY_CREATOR(MultiCastDataStreamSinkLocalState); MultiCastDataStreamSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : Base(parent, state) {} friend class MultiCastDataStreamSinkOperatorX; friend class DataSinkOperatorX; - using Base = PipelineXSinkLocalState; + using Base = PipelineXSpillSinkLocalState; using Parent = MultiCastDataStreamSinkOperatorX; std::string name_suffix() override; + Status open(RuntimeState* state) override; + + std::vector dependencies() const override; + + std::string debug_string(int indentation_level) const override; + private: std::shared_ptr _multi_cast_data_streamer; }; diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp index e45e59d17e27b3..61adfed7573bb5 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp @@ -38,6 +38,8 @@ Status MultiCastDataStreamSourceLocalState::init(RuntimeState* state, LocalState SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); auto& p = _parent->cast(); + _shared_state->multi_cast_data_streamer->set_source_profile(p._consumer_id, + _runtime_profile.get()); _shared_state->multi_cast_data_streamer->set_dep_by_sender_idx(p._consumer_id, _dependency); _wait_for_rf_timer = ADD_TIMER(_runtime_profile, "WaitForRuntimeFilter"); _filter_timer = ADD_TIMER(_runtime_profile, "FilterTime"); @@ -50,6 +52,14 @@ Status MultiCastDataStreamSourceLocalState::init(RuntimeState* state, LocalState return Status::OK(); } +std::vector MultiCastDataStreamSourceLocalState::dependencies() const { + auto dependencies = Base::dependencies(); + auto& p = _parent->cast(); + dependencies.emplace_back( + _shared_state->multi_cast_data_streamer->get_spill_read_dependency(p._consumer_id)); + return dependencies; +} + Status MultiCastDataStreamSourceLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); @@ -92,9 +102,9 @@ Status MultiCastDataStreamerSourceOperatorX::get_block(RuntimeState* state, { SCOPED_TIMER(local_state._get_data_timer); RETURN_IF_ERROR(local_state._shared_state->multi_cast_data_streamer->pull( - _consumer_id, output_block, eos)); + state, _consumer_id, output_block, eos)); } - if (!local_state._conjuncts.empty()) { + if (!local_state._conjuncts.empty() && !output_block->empty()) { SCOPED_TIMER(local_state._filter_timer); RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, output_block->columns())); diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.h b/be/src/pipeline/exec/multi_cast_data_stream_source.h index 57410bf8d9568a..c1af8c5b21cd33 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -37,11 +37,12 @@ namespace pipeline { class MultiCastDataStreamer; class MultiCastDataStreamerSourceOperatorX; -class MultiCastDataStreamSourceLocalState final : public PipelineXLocalState, - public RuntimeFilterConsumer { +class MultiCastDataStreamSourceLocalState final + : public PipelineXSpillLocalState, + public RuntimeFilterConsumer { public: ENABLE_FACTORY_CREATOR(MultiCastDataStreamSourceLocalState); - using Base = PipelineXLocalState; + using Base = PipelineXSpillLocalState; using Parent = MultiCastDataStreamerSourceOperatorX; MultiCastDataStreamSourceLocalState(RuntimeState* state, OperatorXBase* parent); Status init(RuntimeState* state, LocalStateInfo& info) override; @@ -62,6 +63,8 @@ class MultiCastDataStreamSourceLocalState final : public PipelineXLocalState dependencies() const override; + private: friend class MultiCastDataStreamerSourceOperatorX; vectorized::VExprContextSPtrs _output_expr_contexts; diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.cpp b/be/src/pipeline/exec/multi_cast_data_streamer.cpp index 25c939695f90ef..733092bf458254 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.cpp +++ b/be/src/pipeline/exec/multi_cast_data_streamer.cpp @@ -17,9 +17,26 @@ #include "multi_cast_data_streamer.h" +#include +#include + +#include +#include +#include + +#include "common/config.h" +#include "common/exception.h" +#include "common/logging.h" +#include "common/status.h" #include "pipeline/dependency.h" #include "pipeline/exec/multi_cast_data_stream_source.h" +#include "pipeline/exec/spill_utils.h" +#include "runtime/exec_env.h" #include "runtime/runtime_state.h" +#include "util/pretty_printer.h" +#include "util/uid_util.h" +#include "vec/core/block.h" +#include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { #include "common/compile_check_begin.h" @@ -29,44 +46,242 @@ MultiCastBlock::MultiCastBlock(vectorized::Block* block, int un_finish_copy, siz block->clear(); } -Status MultiCastDataStreamer::pull(int sender_idx, doris::vectorized::Block* block, bool* eos) { - int* un_finish_copy = nullptr; +Status MultiCastDataStreamer::pull(RuntimeState* state, int sender_idx, vectorized::Block* block, + bool* eos) { + MultiCastBlock* multi_cast_block = nullptr; { std::lock_guard l(_mutex); + for (auto it = _spill_readers[sender_idx].begin(); + it != _spill_readers[sender_idx].end();) { + if ((*it)->all_data_read) { + it = _spill_readers[sender_idx].erase(it); + } else { + it++; + } + } + + if (!_cached_blocks[sender_idx].empty()) { + *block = std::move(_cached_blocks[sender_idx].front()); + _cached_blocks[sender_idx].erase(_cached_blocks[sender_idx].begin()); + + *eos = _cached_blocks[sender_idx].empty() && _spill_readers[sender_idx].empty() && _eos; + return Status::OK(); + } + + if (!_spill_readers[sender_idx].empty()) { + auto reader_item = _spill_readers[sender_idx].front(); + if (!reader_item->stream->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); + reader_item->block_offset = 0; + } + + auto spill_func = [this, reader_item, sender_idx]() { + vectorized::Block block; + bool spill_eos = false; + size_t read_size = 0; + while (!spill_eos) { + RETURN_IF_ERROR(reader_item->reader->read(&block, &spill_eos)); + if (!block.empty()) { + std::lock_guard l(_mutex); + read_size += block.allocated_bytes(); + _cached_blocks[sender_idx].emplace_back(std::move(block)); + if (_cached_blocks[sender_idx].size() >= 32 || + read_size > 2 * 1024 * 1024) { + break; + } + } + } + + if (spill_eos || !_cached_blocks[sender_idx].empty()) { + reader_item->all_data_read = spill_eos; + _set_ready_for_read(sender_idx); + } + return Status::OK(); + }; + + auto catch_exception_func = [spill_func = std::move(spill_func)]() { + RETURN_IF_CATCH_EXCEPTION(return spill_func();); + }; + + _spill_read_dependencies[sender_idx]->block(); + auto spill_runnable = std::make_shared( + state, _spill_read_dependencies[sender_idx], _source_profiles[sender_idx], + _shared_state->shared_from_this(), catch_exception_func); + auto* thread_pool = + ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(); + RETURN_IF_ERROR(thread_pool->submit(std::move(spill_runnable))); + return Status::OK(); + } + auto& pos_to_pull = _sender_pos_to_read[sender_idx]; const auto end = _multi_cast_blocks.end(); - DCHECK(pos_to_pull != end); + if (pos_to_pull == end) { + _block_reading(sender_idx); + VLOG_DEBUG << "Query: " << print_id(state->query_id()) + << ", pos_to_pull end: " << (void*)(_write_dependency); + *eos = _eos; + return Status::OK(); + } + DCHECK_GT(pos_to_pull->_un_finish_copy, 0); + DCHECK_LE(pos_to_pull->_un_finish_copy, _cast_sender_count); *block = *pos_to_pull->_block; - _cumulative_mem_size -= pos_to_pull->_mem_size; - - un_finish_copy = &pos_to_pull->_un_finish_copy; + multi_cast_block = &(*pos_to_pull); + _copying_count.fetch_add(1); pos_to_pull++; if (pos_to_pull == end) { _block_reading(sender_idx); + *eos = _eos; } - - *eos = _eos and pos_to_pull == end; } - _copy_block(block, *un_finish_copy); - - return Status::OK(); + return _copy_block(state, sender_idx, block, *multi_cast_block); } -void MultiCastDataStreamer::_copy_block(vectorized::Block* block, int& un_finish_copy) { +Status MultiCastDataStreamer::_copy_block(RuntimeState* state, int32_t sender_idx, + vectorized::Block* block, + MultiCastBlock& multi_cast_block) { const auto rows = block->rows(); for (int i = 0; i < block->columns(); ++i) { block->get_by_position(i).column = block->get_by_position(i).column->clone_resized(rows); } + std::unique_lock l(_mutex); - un_finish_copy--; - if (un_finish_copy == 0) { + multi_cast_block._un_finish_copy--; + auto copying_count = _copying_count.fetch_sub(1) - 1; + if (multi_cast_block._un_finish_copy == 0) { + DCHECK_EQ(_multi_cast_blocks.front()._un_finish_copy, 0); + DCHECK_EQ(&(_multi_cast_blocks.front()), &multi_cast_block); _multi_cast_blocks.pop_front(); + _write_dependency->set_ready(); + } else if (copying_count == 0) { + bool spilled = false; + RETURN_IF_ERROR(_trigger_spill_if_need(state, &spilled)); } + + return Status::OK(); +} + +Status MultiCastDataStreamer::_trigger_spill_if_need(RuntimeState* state, bool* triggered) { + if (!state->enable_spill()) { + *triggered = false; + return Status::OK(); + } + + vectorized::SpillStreamSPtr spill_stream; + *triggered = false; + if (_cumulative_mem_size.load() >= config::exchg_node_buffer_size_bytes && + _multi_cast_blocks.size() >= 4) { + _write_dependency->block(); + + if (_copying_count.load() != 0) { + return Status::OK(); + } + + bool has_reached_end = false; + std::vector distances(_cast_sender_count); + size_t total_count = _multi_cast_blocks.size(); + for (int i = 0; i < _sender_pos_to_read.size(); ++i) { + distances[i] = std::distance(_multi_cast_blocks.begin(), _sender_pos_to_read[i]); + if (_sender_pos_to_read[i] == _multi_cast_blocks.end()) { + has_reached_end = true; + CHECK_EQ(distances[i], total_count); + } + + if (!_spill_readers[i].empty()) { + CHECK_EQ(distances[i], 0); + } + } + + 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_profile)); + 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_profiles[i]); + auto reader_item = std::make_shared( + std::move(reader), spill_stream, distances[i], false); + _spill_readers[i].emplace_back(std::move(reader_item)); + } + + _block_reading(i); + } + + RETURN_IF_ERROR(_submit_spill_task(state, spill_stream)); + DCHECK_EQ(_multi_cast_blocks.size(), 0); + + for (auto& pos : _sender_pos_to_read) { + pos = _multi_cast_blocks.end(); + } + _cumulative_mem_size = 0; + *triggered = true; + } + } + + return Status::OK(); +} + +Status MultiCastDataStreamer::_submit_spill_task(RuntimeState* state, + vectorized::SpillStreamSPtr spill_stream) { + std::vector blocks; + for (auto& block : _multi_cast_blocks) { + DCHECK_GT(block._block->rows(), 0); + blocks.emplace_back(std::move(*block._block)); + } + + _multi_cast_blocks.clear(); + + auto spill_func = [state, blocks = std::move(blocks), + spill_stream = std::move(spill_stream)]() 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)); + } + VLOG_DEBUG << "Query: " << print_id(state->query_id()) << " multi cast write " + << blocks_count << " blocks"; + return spill_stream->spill_eof(); + }; + + auto exception_catch_func = [spill_func = std::move(spill_func), + query_id = print_id(state->query_id()), this]() mutable { + auto status = [&]() { RETURN_IF_CATCH_EXCEPTION(return spill_func()); }(); + _write_dependency->set_ready(); + + if (!status.ok()) { + LOG(WARNING) << "Query: " << query_id + << " multi cast write failed: " << status.to_string() + << ", dependency: " << (void*)_spill_dependency.get(); + } else { + for (int i = 0; i < _sender_pos_to_read.size(); ++i) { + _set_ready_for_read(i); + } + } + return status; + }; + + auto spill_runnable = std::make_shared( + state, nullptr, _spill_dependency, _sink_profile, _shared_state->shared_from_this(), + exception_catch_func); + + _spill_dependency->block(); + + auto* thread_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(); + return thread_pool->submit(std::move(spill_runnable)); } Status MultiCastDataStreamer::push(RuntimeState* state, doris::vectorized::Block* block, bool eos) { @@ -74,20 +289,60 @@ Status MultiCastDataStreamer::push(RuntimeState* state, doris::vectorized::Block COUNTER_UPDATE(_process_rows, rows); const auto block_mem_size = block->allocated_bytes(); - _cumulative_mem_size += block_mem_size; - COUNTER_SET(_peak_mem_usage, std::max(_cumulative_mem_size, _peak_mem_usage->value())); { std::lock_guard l(_mutex); - _multi_cast_blocks.emplace_back(block, _cast_sender_count, block_mem_size); - // last elem - auto end = std::prev(_multi_cast_blocks.end()); - for (int i = 0; i < _sender_pos_to_read.size(); ++i) { - if (_sender_pos_to_read[i] == _multi_cast_blocks.end()) { - _sender_pos_to_read[i] = end; - _set_ready_for_read(i); + + if (_pending_block) { + DCHECK_GT(_pending_block->rows(), 0); + const auto pending_size = _pending_block->allocated_bytes(); + _cumulative_mem_size += pending_size; + _multi_cast_blocks.emplace_back(_pending_block.get(), _cast_sender_count, pending_size); + _pending_block.reset(); + + auto end = std::prev(_multi_cast_blocks.end()); + for (int i = 0; i < _sender_pos_to_read.size(); ++i) { + if (_sender_pos_to_read[i] == _multi_cast_blocks.end()) { + _sender_pos_to_read[i] = end; + _set_ready_for_read(i); + } } } + + _cumulative_mem_size += block_mem_size; + COUNTER_SET(_peak_mem_usage, + std::max(_cumulative_mem_size.load(), _peak_mem_usage->value())); + + if (rows > 0) { + if (!eos) { + bool spilled = false; + RETURN_IF_ERROR(_trigger_spill_if_need(state, &spilled)); + if (spilled) { + _pending_block = vectorized::Block::create_unique( + block->get_columns_with_type_and_name()); + block->clear(); + return Status::OK(); + } + } + + _multi_cast_blocks.emplace_back(block, _cast_sender_count, block_mem_size); + + // last elem + auto end = std::prev(_multi_cast_blocks.end()); + for (int i = 0; i < _sender_pos_to_read.size(); ++i) { + if (_sender_pos_to_read[i] == _multi_cast_blocks.end()) { + _sender_pos_to_read[i] = end; + _set_ready_for_read(i); + } + } + } else if (eos) { + for (int i = 0; i < _sender_pos_to_read.size(); ++i) { + if (_sender_pos_to_read[i] == _multi_cast_blocks.end()) { + _set_ready_for_read(i); + } + } + } + _eos = eos; } @@ -117,4 +372,38 @@ void MultiCastDataStreamer::_block_reading(int sender_idx) { dep->block(); } +std::string MultiCastDataStreamer::debug_string() { + size_t read_ready_count = 0; + size_t read_spill_ready_count = 0; + size_t pos_at_end_count = 0; + size_t blocks_count = 0; + { + std::unique_lock l(_mutex); + blocks_count = _multi_cast_blocks.size(); + for (int32_t i = 0; i != _cast_sender_count; ++i) { + if (!_dependencies[i]->is_blocked_by()) { + read_ready_count++; + } + + if (!_spill_read_dependencies[i]->is_blocked_by()) { + read_spill_ready_count++; + } + + if (_sender_pos_to_read[i] == _multi_cast_blocks.end()) { + pos_at_end_count++; + } + } + } + + fmt::memory_buffer debug_string_buffer; + fmt::format_to( + debug_string_buffer, + "MemSize: {}, blocks: {}, sender count: {}, pos_at_end_count: {}, copying_count: {} " + "read_ready_count: {}, read_spill_ready_count: {}, write spill dependency blocked: {}", + PrettyPrinter::print_bytes(_cumulative_mem_size), blocks_count, _cast_sender_count, + pos_at_end_count, _copying_count.load(), read_ready_count, read_spill_ready_count, + (_spill_dependency->is_blocked_by() != nullptr)); + return fmt::to_string(debug_string_buffer); +} + } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/multi_cast_data_streamer.h b/be/src/pipeline/exec/multi_cast_data_streamer.h index 380538d0ac0805..8fd11a325bd3f2 100644 --- a/be/src/pipeline/exec/multi_cast_data_streamer.h +++ b/be/src/pipeline/exec/multi_cast_data_streamer.h @@ -17,12 +17,23 @@ #pragma once +#include +#include +#include +#include + +#include "pipeline/dependency.h" +#include "util/runtime_profile.h" +#include "vec/core/block.h" #include "vec/sink/vdata_stream_sender.h" +#include "vec/spill/spill_stream.h" namespace doris::pipeline { #include "common/compile_check_begin.h" class Dependency; +struct MultiCastSharedState; + struct MultiCastBlock { MultiCastBlock(vectorized::Block* block, int need_copy, size_t mem_size); @@ -33,30 +44,49 @@ struct MultiCastBlock { size_t _mem_size; }; +struct SpillingReader { + vectorized::SpillReaderUPtr reader; + vectorized::SpillStreamSPtr stream; + int64_t block_offset {0}; + bool all_data_read {false}; +}; + // TDOD: MultiCastDataStreamer same as the data queue, maybe rethink union and refactor the // code class MultiCastDataStreamer { public: - MultiCastDataStreamer(const RowDescriptor& row_desc, ObjectPool* pool, int cast_sender_count, + MultiCastDataStreamer(const RowDescriptor& row_desc, MultiCastSharedState* shared_state, + ObjectPool* pool, int cast_sender_count, int32_t node_id, bool with_dependencies = false) : _row_desc(row_desc), + _shared_state(shared_state), _profile(pool->add(new RuntimeProfile("MultiCastDataStreamSink"))), - _cast_sender_count(cast_sender_count) { + _cached_blocks(cast_sender_count), + _cast_sender_count(cast_sender_count), + _node_id(node_id), + _spill_readers(cast_sender_count), + _source_profiles(cast_sender_count) { _sender_pos_to_read.resize(cast_sender_count, _multi_cast_blocks.end()); if (with_dependencies) { _dependencies.resize(cast_sender_count, nullptr); } + _spill_dependency = Dependency::create_shared(_node_id, _node_id, + "MultiCastDataStreamerDependency", true); + + for (int i = 0; i != cast_sender_count; ++i) { + _spill_read_dependencies.emplace_back(Dependency::create_shared( + node_id, node_id, "MultiCastReadSpillDependency", true)); + } _peak_mem_usage = ADD_COUNTER(profile(), "PeakMemUsage", TUnit::BYTES); _process_rows = ADD_COUNTER(profile(), "ProcessRows", TUnit::UNIT); }; ~MultiCastDataStreamer() = default; - Status pull(int sender_idx, vectorized::Block* block, bool* eos); + Status pull(RuntimeState* state, int sender_idx, vectorized::Block* block, bool* eos); Status push(RuntimeState* state, vectorized::Block* block, bool eos); - const RowDescriptor& row_desc() { return _row_desc; } RuntimeProfile* profile() { return _profile; } @@ -66,24 +96,60 @@ class MultiCastDataStreamer { _block_reading(sender_idx); } + void set_write_dependency(Dependency* dependency) { _write_dependency = dependency; } + + Dependency* get_spill_dependency() const { return _spill_dependency.get(); } + + Dependency* get_spill_read_dependency(int sender_idx) const { + return _spill_read_dependencies[sender_idx].get(); + } + + void set_sink_profile(RuntimeProfile* profile) { _sink_profile = profile; } + + void set_source_profile(int sender_idx, RuntimeProfile* profile) { + _source_profiles[sender_idx] = profile; + } + + std::string debug_string(); + private: void _set_ready_for_read(int sender_idx); void _block_reading(int sender_idx); - void _copy_block(vectorized::Block* block, int& un_finish_copy); + Status _copy_block(RuntimeState* state, int32_t sender_idx, vectorized::Block* block, + MultiCastBlock& multi_cast_block); + + Status _submit_spill_task(RuntimeState* state, vectorized::SpillStreamSPtr spill_stream); + + Status _trigger_spill_if_need(RuntimeState* state, bool* triggered); const RowDescriptor& _row_desc; + MultiCastSharedState* _shared_state; RuntimeProfile* _profile = nullptr; std::list _multi_cast_blocks; + std::list _spilling_blocks; + std::vector> _cached_blocks; std::vector::iterator> _sender_pos_to_read; std::mutex _mutex; bool _eos = false; int _cast_sender_count = 0; - int64_t _cumulative_mem_size = 0; - + int _node_id; + std::atomic_int64_t _cumulative_mem_size = 0; + std::atomic_int64_t _copying_count = 0; RuntimeProfile::Counter* _process_rows = nullptr; RuntimeProfile::Counter* _peak_mem_usage = nullptr; + Dependency* _write_dependency; std::vector _dependencies; + std::shared_ptr _spill_dependency; + + vectorized::BlockUPtr _pending_block; + + std::vector>> _spill_readers; + + std::vector> _spill_read_dependencies; + + RuntimeProfile* _sink_profile; + std::vector _source_profiles; }; #include "common/compile_check_end.h" } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp index c9642bb9b3bd20..3c76ade1f27f6a 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp @@ -480,6 +480,7 @@ Status NestedLoopJoinProbeOperatorX::push(doris::RuntimeState* state, vectorized auto& local_state = get_local_state(state); COUNTER_UPDATE(local_state._probe_rows_counter, block->rows()); COUNTER_SET(local_state._memory_used_counter, block->allocated_bytes()); + SCOPED_PEAK_MEM(&local_state.estimate_memory_usage()); local_state._cur_probe_row_visited_flags.resize(block->rows()); std::fill(local_state._cur_probe_row_visited_flags.begin(), local_state._cur_probe_row_visited_flags.end(), 0); @@ -507,10 +508,12 @@ Status NestedLoopJoinProbeOperatorX::pull(RuntimeState* state, vectorized::Block bool* eos) const { auto& local_state = get_local_state(state); if (_is_output_left_side_only) { + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); RETURN_IF_ERROR(local_state._build_output_block(local_state._child_block.get(), block)); *eos = local_state._shared_state->left_side_eos; local_state._need_more_input_data = !local_state._shared_state->left_side_eos; } else { + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); *eos = ((_match_all_build || _is_right_semi_anti) ? local_state._output_null_idx_build_side == local_state._shared_state->build_blocks.size() && @@ -522,9 +525,10 @@ Status NestedLoopJoinProbeOperatorX::pull(RuntimeState* state, vectorized::Block local_state.add_tuple_is_null_column(&local_state._join_block); { SCOPED_TIMER(local_state._join_filter_timer); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block( - local_state._conjuncts, &local_state._join_block, - local_state._join_block.columns())); + + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, + &local_state._join_block, + local_state._join_block.columns())); } RETURN_IF_ERROR( local_state._build_output_block(&local_state._join_block, block, false)); @@ -540,6 +544,7 @@ Status NestedLoopJoinProbeOperatorX::pull(RuntimeState* state, vectorized::Block state, join_op_variants); }; SCOPED_TIMER(local_state._loop_join_timer); + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); RETURN_IF_ERROR(std::visit( func, local_state._shared_state->join_op_variants, vectorized::make_bool_variant(_match_all_build || _is_right_semi_anti), diff --git a/be/src/pipeline/exec/operator.cpp b/be/src/pipeline/exec/operator.cpp index bb254aae72b8a7..76aba5d30b7c27 100644 --- a/be/src/pipeline/exec/operator.cpp +++ b/be/src/pipeline/exec/operator.cpp @@ -250,6 +250,14 @@ void PipelineXLocalStateBase::clear_origin_block() { _origin_block.clear_column_data(_parent->intermediate_row_desc().num_materialized_slots()); } +Status PipelineXLocalStateBase::filter_block(const vectorized::VExprContextSPtrs& expr_contexts, + vectorized::Block* block, size_t column_to_keep) { + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(expr_contexts, block, column_to_keep)); + + _estimate_memory_usage += vectorized::VExprContext::get_memory_usage(expr_contexts); + return Status::OK(); +} + Status OperatorXBase::do_projections(RuntimeState* state, vectorized::Block* origin_block, vectorized::Block* output_block) const { auto* local_state = state->get_local_state(operator_id()); @@ -262,11 +270,14 @@ Status OperatorXBase::do_projections(RuntimeState* state, vectorized::Block* ori vectorized::Block input_block = *origin_block; std::vector result_column_ids; + size_t bytes_usage = 0; for (const auto& projections : local_state->_intermediate_projections) { result_column_ids.resize(projections.size()); for (int i = 0; i < projections.size(); i++) { RETURN_IF_ERROR(projections[i]->execute(&input_block, &result_column_ids[i])); } + + bytes_usage += input_block.allocated_bytes(); input_block.shuffle_columns(result_column_ids); } @@ -277,12 +288,14 @@ Status OperatorXBase::do_projections(RuntimeState* state, vectorized::Block* ori auto& null_column = reinterpret_cast(*to); null_column.get_nested_column().insert_range_from(*from, 0, rows); null_column.get_null_map_column().get_data().resize_fill(rows, 0); + bytes_usage += null_column.allocated_bytes(); } else { to = make_nullable(from, false)->assume_mutable(); } } else { if (_keep_origin || !from->is_exclusive()) { to->insert_range_from(*from, 0, rows); + bytes_usage += from->allocated_bytes(); } else { to = from->assume_mutable(); } @@ -295,18 +308,24 @@ Status OperatorXBase::do_projections(RuntimeState* state, vectorized::Block* ori *_output_row_descriptor); if (rows != 0) { auto& mutable_columns = mutable_block.mutable_columns(); + const size_t origin_columns_count = input_block.columns(); DCHECK(mutable_columns.size() == local_state->_projections.size()); for (int i = 0; i < mutable_columns.size(); ++i) { auto result_column_id = -1; RETURN_IF_ERROR(local_state->_projections[i]->execute(&input_block, &result_column_id)); auto column_ptr = input_block.get_by_position(result_column_id) .column->convert_to_full_column_if_const(); + if (result_column_id >= origin_columns_count) { + bytes_usage += column_ptr->allocated_bytes(); + } insert_column_datas(mutable_columns[i], column_ptr, rows); } DCHECK(mutable_block.rows() == rows); output_block->set_columns(std::move(mutable_columns)); } + local_state->_estimate_memory_usage += bytes_usage; + return Status::OK(); } @@ -381,7 +400,7 @@ std::string DataSinkOperatorXBase::debug_string(RuntimeState* state, int indenta Status DataSinkOperatorXBase::init(const TDataSink& tsink) { std::string op_name = "UNKNOWN_SINK"; - std::map::const_iterator it = _TDataSinkType_VALUES_TO_NAMES.find(tsink.type); + auto it = _TDataSinkType_VALUES_TO_NAMES.find(tsink.type); if (it != _TDataSinkType_VALUES_TO_NAMES.end()) { op_name = it->second; diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h index df6e9c913b6b4c..84327f8b5a6456 100644 --- a/be/src/pipeline/exec/operator.h +++ b/be/src/pipeline/exec/operator.h @@ -20,6 +20,7 @@ #include #include +#include #include #include #include @@ -31,9 +32,12 @@ #include "common/status.h" #include "pipeline/dependency.h" #include "pipeline/exec/operator.h" +#include "pipeline/exec/spill_utils.h" #include "pipeline/local_exchange/local_exchanger.h" +#include "runtime/memory/mem_tracker.h" #include "runtime/query_context.h" #include "runtime/runtime_state.h" +#include "runtime/thread_context.h" #include "util/runtime_profile.h" #include "vec/core/block.h" #include "vec/runtime/vdata_stream_recvr.h" @@ -109,7 +113,11 @@ class OperatorBase { virtual size_t revocable_mem_size(RuntimeState* state) const { return 0; } - virtual Status revoke_memory(RuntimeState* state) { return Status::OK(); } + virtual Status revoke_memory(RuntimeState* state, + const std::shared_ptr& spill_context) { + return Status::OK(); + } + [[nodiscard]] virtual bool require_data_distribution() const { return false; } OperatorPtr child() { return _child; } [[nodiscard]] bool followed_by_shuffled_operator() const { @@ -185,6 +193,15 @@ class PipelineXLocalStateBase { std::shared_ptr get_query_statistics_ptr() { return _query_statistics; } + Status filter_block(const vectorized::VExprContextSPtrs& expr_contexts, + vectorized::Block* block, size_t column_to_keep); + + int64_t& estimate_memory_usage() { return _estimate_memory_usage; } + + void reset_estimate_memory_usage() { _estimate_memory_usage = 0; } + + bool low_memory_mode() { return _state->get_query_ctx()->low_memory_mode(); } + protected: friend class OperatorXBase; template @@ -192,6 +209,7 @@ class PipelineXLocalStateBase { ObjectPool* _pool = nullptr; int64_t _num_rows_returned {0}; + int64_t _estimate_memory_usage {0}; std::unique_ptr _runtime_profile; @@ -273,32 +291,137 @@ class PipelineXSpillLocalState : public PipelineXLocalState { Status init(RuntimeState* state, LocalStateInfo& info) override { RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); - _spill_counters = ADD_LABEL_COUNTER_WITH_LEVEL(Base::profile(), "Spill", 1); - _spill_recover_time = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillRecoverTime", "Spill", 1); - _spill_read_data_time = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillReadDataTime", "Spill", 1); - _spill_deserialize_time = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillDeserializeTime", "Spill", 1); - _spill_read_bytes = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillReadDataSize", - TUnit::BYTES, "Spill", 1); - _spill_wait_in_queue_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillWaitInQueueTime", "Spill", 1); - _spill_write_wait_io_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteWaitIOTime", "Spill", 1); - _spill_read_wait_io_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillReadWaitIOTime", "Spill", 1); + + _spill_total_timer = ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillTotalTime", 1); + init_spill_read_counters(); + return Status::OK(); } - RuntimeProfile::Counter* _spill_counters = nullptr; - RuntimeProfile::Counter* _spill_recover_time; - RuntimeProfile::Counter* _spill_read_data_time; - RuntimeProfile::Counter* _spill_deserialize_time; - RuntimeProfile::Counter* _spill_read_bytes; - RuntimeProfile::Counter* _spill_write_wait_io_timer = nullptr; - RuntimeProfile::Counter* _spill_read_wait_io_timer = nullptr; - RuntimeProfile::Counter* _spill_wait_in_queue_timer = nullptr; + void init_spill_write_counters() { + _spill_write_timer = ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteTime", 1); + + _spill_write_wait_in_queue_task_count = ADD_COUNTER_WITH_LEVEL( + Base::profile(), "SpillWriteTaskWaitInQueueCount", TUnit::UNIT, 1); + _spill_writing_task_count = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteTaskCount", TUnit::UNIT, 1); + _spill_write_wait_in_queue_timer = + ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteTaskWaitInQueueTime", 1); + + _spill_write_file_timer = ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteFileTime", 1); + + _spill_write_serialize_block_timer = + ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteSerializeBlockTime", 1); + _spill_write_block_count = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteBlockCount", TUnit::UNIT, 1); + _spill_write_block_data_size = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteBlockBytes", TUnit::BYTES, 1); + _spill_write_file_total_size = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteFileBytes", TUnit::BYTES, 1); + _spill_write_rows_count = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteRows", TUnit::UNIT, 1); + _spill_file_total_count = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteFileTotalCount", TUnit::UNIT, 1); + } + + void init_spill_read_counters() { + // Spill read counters + _spill_recover_time = ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillRecoverTime", 1); + + _spill_read_wait_in_queue_task_count = ADD_COUNTER_WITH_LEVEL( + Base::profile(), "SpillReadTaskWaitInQueueCount", TUnit::UNIT, 1); + _spill_reading_task_count = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillReadTaskCount", TUnit::UNIT, 1); + _spill_read_wait_in_queue_timer = + ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillReadTaskWaitInQueueTime", 1); + + _spill_read_file_time = ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillReadFileTime", 1); + _spill_read_derialize_block_timer = + ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillReadDerializeBlockTime", 1); + + _spill_read_block_count = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillReadBlockCount", TUnit::UNIT, 1); + _spill_read_block_data_size = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillReadBlockBytes", TUnit::BYTES, 1); + _spill_read_file_size = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillReadFileBytes", TUnit::BYTES, 1); + _spill_read_rows_count = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillReadRows", TUnit::UNIT, 1); + _spill_read_file_count = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "SpillReadFileCount", TUnit::UNIT, 1); + + _spill_file_current_size = ADD_COUNTER_WITH_LEVEL( + Base::profile(), "SpillWriteFileCurrentBytes", TUnit::BYTES, 1); + _spill_file_current_count = ADD_COUNTER_WITH_LEVEL( + Base::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::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; + // Original count of spilled Blocks + // One Big Block maybe split into multiple small Blocks when actually written to disk file. + RuntimeProfile::Counter* _spill_write_block_count = nullptr; + // Total bytes of spill data in Block format(in memory format) + RuntimeProfile::Counter* _spill_write_block_data_size = nullptr; + // 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_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_derialize_block_timer = nullptr; + RuntimeProfile::Counter* _spill_read_block_count = nullptr; + // Total bytes of read data in Block format(in memory format) + RuntimeProfile::Counter* _spill_read_block_data_size = nullptr; + // Total bytes of spill data read from disk file + 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; @@ -345,6 +468,7 @@ class PipelineXSinkLocalStateBase { RuntimeProfile::Counter* rows_input_counter() { return _rows_input_counter; } RuntimeProfile::Counter* exec_time_counter() { return _exec_timer; } RuntimeProfile::Counter* memory_used_counter() { return _memory_used_counter; } + virtual std::vector dependencies() const { return {nullptr}; } // override in exchange sink , AsyncWriterSink @@ -417,6 +541,7 @@ class PipelineXSinkLocalState : public PipelineXSinkLocalStateBase { protected: Dependency* _dependency = nullptr; + std::shared_ptr _spill_dependency; SharedStateType* _shared_state = nullptr; private: @@ -427,13 +552,13 @@ class PipelineXSinkLocalState : public PipelineXSinkLocalStateBase { class DataSinkOperatorXBase : public OperatorBase { public: DataSinkOperatorXBase(const int operator_id, const int node_id) - : OperatorBase(), _operator_id(operator_id), _node_id(node_id), _dests_id({1}) {} + : _operator_id(operator_id), _node_id(node_id), _dests_id({1}) {} DataSinkOperatorXBase(const int operator_id, const int node_id, const int dest_id) - : OperatorBase(), _operator_id(operator_id), _node_id(node_id), _dests_id({dest_id}) {} + : _operator_id(operator_id), _node_id(node_id), _dests_id({dest_id}) {} DataSinkOperatorXBase(const int operator_id, const int node_id, std::vector& sources) - : OperatorBase(), _operator_id(operator_id), _node_id(node_id), _dests_id(sources) {} + : _operator_id(operator_id), _node_id(node_id), _dests_id(sources) {} ~DataSinkOperatorXBase() override = default; @@ -461,6 +586,12 @@ class DataSinkOperatorXBase : public OperatorBase { [[nodiscard]] virtual Status setup_local_state(RuntimeState* state, LocalSinkStateInfo& info) = 0; + [[nodiscard]] virtual size_t get_reserve_mem_size(RuntimeState* state, bool eos) { + return state->minimum_operator_memory_required_bytes(); + } + + [[nodiscard]] bool is_spillable() const { return _spillable; } + template TARGET& cast() { DCHECK(dynamic_cast(this)) @@ -523,11 +654,9 @@ class DataSinkOperatorXBase : public OperatorBase { const int _operator_id; const int _node_id; int _nereids_id = -1; + bool _spillable = false; std::vector _dests_id; std::string _name; - - // Maybe this will be transferred to BufferControlBlock. - std::shared_ptr _query_statistics; }; template @@ -563,34 +692,87 @@ class PipelineXSpillSinkLocalState : public PipelineXSinkLocalState dependencies() const override { + auto dependencies = Base::dependencies(); + return dependencies; + } + + void update_max_min_rows_counter() { + int64_t max_rows = 0; + int64_t min_rows = std::numeric_limits::max(); + + for (auto rows : _rows_in_partitions) { + if (rows > max_rows) { + max_rows = rows; + } + if (rows < min_rows) { + min_rows = rows; + } + } + + COUNTER_SET(_spill_max_rows_of_partition, max_rows); + COUNTER_SET(_spill_min_rows_of_partition, min_rows); + } + + std::vector _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_timer = nullptr; + RuntimeProfile::Counter* _spill_write_serialize_block_timer = nullptr; + // Original count of spilled Blocks + // One Big Block maybe split into multiple small Blocks when actually written to disk file. + RuntimeProfile::Counter* _spill_write_block_count = nullptr; + // Total bytes of spill data in Block format(in memory format) + RuntimeProfile::Counter* _spill_write_block_data_size = nullptr; + RuntimeProfile::Counter* _spill_write_rows_count = nullptr; + // Spilled file total size + RuntimeProfile::Counter* _spill_file_total_size = nullptr; + + RuntimeProfile::Counter* _spill_max_rows_of_partition = nullptr; + RuntimeProfile::Counter* _spill_min_rows_of_partition = nullptr; }; class OperatorXBase : public OperatorBase { @@ -670,6 +852,15 @@ 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 + [[nodiscard]] virtual size_t get_reserve_mem_size(RuntimeState* state) { + return state->minimum_operator_memory_required_bytes(); + } + virtual std::string debug_string(int indentation_level = 0) const; virtual std::string debug_string(RuntimeState* state, int indentation_level = 0) const; @@ -724,6 +915,8 @@ class OperatorXBase : public OperatorBase { // To keep compatibility with older FE void set_serial_operator() { _is_serial_operator = true; } + virtual void reset_reserve_mem_size(RuntimeState* state) {} + protected: template friend class PipelineXLocalState; @@ -755,6 +948,7 @@ class OperatorXBase : public OperatorBase { int64_t _limit; // -1: no limit uint32_t _debug_point_count = 0; + std::atomic_uint32_t _bytes_per_row = 0; std::string _op_name; int _parallel_tasks = 0; @@ -779,6 +973,29 @@ class OperatorX : public OperatorXBase { [[nodiscard]] LocalState& get_local_state(RuntimeState* state) const { return state->get_local_state(operator_id())->template cast(); } + + 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); + } + } }; /** diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.cpp b/be/src/pipeline/exec/partition_sort_sink_operator.cpp index cdc4b31c76d7e0..7c4506e3919872 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_sink_operator.cpp @@ -177,6 +177,19 @@ Status PartitionSortSinkOperatorX::_split_block_by_partition( return Status::OK(); } +size_t PartitionSortSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + auto rows = state->batch_size(); + size_t reserve_mem_size = std::visit( + vectorized::Overload {[&](std::monostate&) -> size_t { return 0; }, + [&](auto& agg_method) -> size_t { + return agg_method.hash_table->estimate_memory(rows); + }}, + local_state._partitioned_data->method_variant); + reserve_mem_size += rows * sizeof(size_t); // hash values + return reserve_mem_size; +} + Status PartitionSortSinkOperatorX::_emplace_into_hash_table( const vectorized::ColumnRawPtrs& key_columns, vectorized::Block* input_block, PartitionSortSinkLocalState& local_state, bool eos) { diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.h b/be/src/pipeline/exec/partition_sort_sink_operator.h index 32bbf38202713f..025845682c741b 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.h +++ b/be/src/pipeline/exec/partition_sort_sink_operator.h @@ -87,6 +87,8 @@ class PartitionSortSinkOperatorX final : public DataSinkOperatorXempty()) { //if buffer have no data and sink not eos, block reading and wait for signal again - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, - output_block->columns())); + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, output_block, + output_block->columns())); local_state._num_rows_returned += output_block->rows(); } return Status::OK(); diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp index 3d763ce9f6309a..982c13e60b5cec 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp @@ -18,23 +18,25 @@ #include "partitioned_aggregation_sink_operator.h" #include +#include #include #include "aggregation_sink_operator.h" #include "common/status.h" +#include "pipeline/dependency.h" #include "pipeline/exec/spill_utils.h" +#include "pipeline/pipeline_task.h" #include "runtime/fragment_mgr.h" +#include "util/pretty_printer.h" +#include "util/runtime_profile.h" +#include "vec/spill/spill_stream.h" #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { #include "common/compile_check_begin.h" PartitionedAggSinkLocalState::PartitionedAggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : Base(parent, state) { - _finish_dependency = - std::make_shared(parent->operator_id(), parent->node_id(), - parent->get_name() + "_SPILL_DEPENDENCY", true); -} + : Base(parent, state) {} Status PartitionedAggSinkLocalState::init(doris::RuntimeState* state, doris::pipeline::LocalSinkStateInfo& info) { @@ -46,7 +48,7 @@ Status PartitionedAggSinkLocalState::init(doris::RuntimeState* state, _init_counters(); auto& parent = Base::_parent->template cast(); - Base::_shared_state->init_spill_params(parent._spill_partition_count_bits); + Base::_shared_state->init_spill_params(parent._spill_partition_count); RETURN_IF_ERROR(setup_in_memory_agg_op(state)); @@ -59,13 +61,19 @@ Status PartitionedAggSinkLocalState::init(doris::RuntimeState* state, value_columns_.emplace_back(aggregate_evaluator->function()->create_serialize_column()); } - _finish_dependency->block(); + _rows_in_partitions.assign(Base::_shared_state->partition_count, 0); + + _spill_dependency = Dependency::create_shared(parent.operator_id(), parent.node_id(), + "AggSinkSpillDependency", true); + state->get_task()->add_spill_dependency(_spill_dependency.get()); + return Status::OK(); } Status PartitionedAggSinkLocalState::open(RuntimeState* state) { SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_open_timer); + _shared_state->setup_shared_profile(_profile); return Base::open(state); } @@ -97,10 +105,16 @@ void PartitionedAggSinkLocalState::_init_counters() { _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); _max_row_size_counter = ADD_COUNTER(Base::profile(), "MaxRowSizeInBytes", TUnit::UNIT); + _memory_usage_container = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "MemoryUsageContainer", TUnit::BYTES, 1); + _memory_usage_arena = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "MemoryUsageArena", TUnit::BYTES, 1); + _memory_usage_reserved = + ADD_COUNTER_WITH_LEVEL(Base::profile(), "MemoryUsageReserved", TUnit::BYTES, 1); COUNTER_SET(_max_row_size_counter, (int64_t)0); _spill_serialize_hash_table_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillSerializeHashTableTime", "Spill", 1); + ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillSerializeHashTableTime", 1); } #define UPDATE_PROFILE(counter, name) \ do { \ @@ -123,6 +137,10 @@ void PartitionedAggSinkLocalState::update_profile(RuntimeProfile* child_profile) UPDATE_PROFILE(_hash_table_emplace_timer, "HashTableEmplaceTime"); UPDATE_PROFILE(_hash_table_input_counter, "HashTableInputCount"); UPDATE_PROFILE(_max_row_size_counter, "MaxRowSizeInBytes"); + UPDATE_PROFILE(_memory_usage_container, "MemoryUsageContainer"); + UPDATE_PROFILE(_memory_usage_arena, "MemoryUsageArena"); + + update_max_min_rows_counter(); } PartitionedAggSinkOperatorX::PartitionedAggSinkOperatorX(ObjectPool* pool, int operator_id, @@ -132,14 +150,13 @@ PartitionedAggSinkOperatorX::PartitionedAggSinkOperatorX(ObjectPool* pool, int o : DataSinkOperatorX(operator_id, tnode.node_id) { _agg_sink_operator = std::make_unique(pool, operator_id, tnode, descs, require_bucket_distribution); + _spillable = true; } Status PartitionedAggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(DataSinkOperatorX::init(tnode, state)); _name = "PARTITIONED_AGGREGATION_SINK_OPERATOR"; - if (state->query_options().__isset.external_agg_partition_bits) { - _spill_partition_count_bits = state->query_options().external_agg_partition_bits; - } + _spill_partition_count = state->spill_aggregation_partition_count(); _agg_sink_operator->set_dests_id(DataSinkOperatorX::dests_id()); RETURN_IF_ERROR( @@ -157,27 +174,40 @@ Status PartitionedAggSinkOperatorX::sink(doris::RuntimeState* state, vectorized: local_state.inc_running_big_mem_op_num(state); SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); - RETURN_IF_ERROR(local_state.Base::_shared_state->sink_status); local_state._eos = eos; auto* runtime_state = local_state._runtime_state.get(); 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; if (eos) { + revocable_size = revocable_mem_size(state); + query_mem_limit = state->get_query_ctx()->get_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)); + 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)); } local_state._dependency->set_ready_to_read(); - local_state._finish_dependency->set_ready(); } } else { local_state._dependency->set_ready_to_read(); - local_state._finish_dependency->set_ready(); + } + } else if (local_state._shared_state->is_spilled) { + if (revocable_mem_size(state) >= vectorized::SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { + return revoke_memory(state, nullptr); } } if (local_state._runtime_state) { @@ -186,16 +216,14 @@ Status PartitionedAggSinkOperatorX::sink(doris::RuntimeState* state, vectorized: } return Status::OK(); } -Status PartitionedAggSinkOperatorX::revoke_memory(RuntimeState* state) { +Status PartitionedAggSinkOperatorX::revoke_memory( + RuntimeState* state, const std::shared_ptr& spill_context) { auto& local_state = get_local_state(state); - return local_state.revoke_memory(state); + return local_state.revoke_memory(state, spill_context); } size_t PartitionedAggSinkOperatorX::revocable_mem_size(RuntimeState* state) const { auto& local_state = get_local_state(state); - if (!local_state.Base::_shared_state->sink_status.ok()) { - return UINT64_MAX; - } auto* runtime_state = local_state._runtime_state.get(); auto size = _agg_sink_operator->get_revocable_mem_size(runtime_state); return size; @@ -229,11 +257,23 @@ Status PartitionedAggSinkLocalState::setup_in_memory_agg_op(RuntimeState* state) return sink_local_state->open(state); } -Status PartitionedAggSinkLocalState::revoke_memory(RuntimeState* state) { - VLOG_DEBUG << "query " << print_id(state->query_id()) << " agg node " - << Base::_parent->node_id() << " revoke_memory" - << ", eos: " << _eos; - RETURN_IF_ERROR(Base::_shared_state->sink_status); +size_t PartitionedAggSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + auto* runtime_state = local_state._runtime_state.get(); + auto size = _agg_sink_operator->get_reserve_mem_size(runtime_state, eos); + COUNTER_SET(local_state._memory_usage_reserved, int64_t(size)); + return size; +} + +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 " + "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))); if (!_shared_state->is_spilled) { _shared_state->is_spilled = true; profile()->add_info_string("Spilled", "true"); @@ -241,79 +281,78 @@ Status PartitionedAggSinkLocalState::revoke_memory(RuntimeState* state) { // TODO: spill thread may set_ready before the task::execute thread put the task to blocked state if (!_eos) { - Base::_dependency->Dependency::block(); + Base::_spill_dependency->Dependency::block(); } auto& parent = Base::_parent->template cast(); Status status; Defer defer {[&]() { if (!status.ok()) { if (!_eos) { - Base::_dependency->Dependency::set_ready(); + Base::_spill_dependency->Dependency::set_ready(); } } }}; auto query_id = state->query_id(); - MonotonicStopWatch submit_timer; - submit_timer.start(); DBUG_EXECUTE_IF("fault_inject::partitioned_agg_sink::revoke_memory_submit_func", { status = Status::Error( "fault_inject partitioned_agg_sink revoke_memory submit_func failed"); return status; }); - auto spill_runnable = std::make_shared( - state, _shared_state->shared_from_this(), - [this, &parent, state, query_id, submit_timer] { + state->get_query_ctx()->increase_revoking_tasks_count(); + + auto spill_runnable = std::make_shared( + state, spill_context, _spill_dependency, _profile, _shared_state->shared_from_this(), + [this, &parent, state, query_id, size_to_revoke] { + Status status; DBUG_EXECUTE_IF("fault_inject::partitioned_agg_sink::revoke_memory_cancel", { - auto st = Status::InternalError( + status = Status::InternalError( "fault_inject partitioned_agg_sink " "revoke_memory canceled"); - ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_id, st); - return st; + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_id, status); + return status; }); - _spill_wait_in_queue_timer->update(submit_timer.elapsed_time()); - SCOPED_TIMER(Base::_spill_timer); Defer defer {[&]() { - if (!_shared_state->sink_status.ok() || state->is_cancelled()) { - if (!_shared_state->sink_status.ok()) { - LOG(WARNING) - << "query " << print_id(query_id) << " agg node " - << Base::_parent->node_id() - << " revoke_memory error: " << Base::_shared_state->sink_status; + 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 { - VLOG_DEBUG << "query " << print_id(query_id) << " agg node " - << Base::_parent->node_id() << " revoke_memory finish" - << ", eos: " << _eos; + 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(); - _finish_dependency->set_ready(); - } else { - Base::_dependency->Dependency::set_ready(); } + state->get_query_ctx()->decrease_revoking_tasks_count(); }}; auto* runtime_state = _runtime_state.get(); auto* agg_data = parent._agg_sink_operator->get_agg_data(runtime_state); - Base::_shared_state->sink_status = - std::visit(vectorized::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, _eos)); - }}, - agg_data->method_variant); - RETURN_IF_ERROR(Base::_shared_state->sink_status); - Base::_shared_state->sink_status = - parent._agg_sink_operator->reset_hash_table(runtime_state); - return Base::_shared_state->sink_status; + status = std::visit( + vectorized::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 ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool()->submit( diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h index 63cd95534dc9f5..b0315e31f5d243 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.h @@ -16,10 +16,16 @@ // under the License. #pragma once +#include +#include + #include "aggregation_sink_operator.h" +#include "pipeline/dependency.h" #include "pipeline/exec/operator.h" +#include "util/pretty_printer.h" #include "vec/exprs/vectorized_agg_fn.h" #include "vec/exprs/vexpr.h" +#include "vec/spill/spill_stream.h" #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { @@ -40,9 +46,8 @@ class PartitionedAggSinkLocalState Status init(RuntimeState* state, LocalSinkStateInfo& info) override; Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; - Dependency* finishdependency() override { return _finish_dependency.get(); } - Status revoke_memory(RuntimeState* state); + Status revoke_memory(RuntimeState* state, const std::shared_ptr& spill_context); Status setup_in_memory_agg_op(RuntimeState* state); @@ -55,7 +60,7 @@ class PartitionedAggSinkLocalState }; template Status _spill_hash_table(RuntimeState* state, HashTableCtxType& context, - HashTableType& hash_table, bool eos) { + HashTableType& hash_table, const size_t size_to_revoke, bool eos) { Status status; Defer defer {[&]() { if (!status.ok()) { @@ -67,8 +72,21 @@ class PartitionedAggSinkLocalState Base::_shared_state->in_mem_shared_state->aggregate_data_container->init_once(); - static int spill_batch_rows = 4096; - int row_count = 0; + const auto total_rows = + Base::_shared_state->in_mem_shared_state->aggregate_data_container->total_count(); + + 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`) + const auto spill_batch_rows = std::min( + 1024 * 1024, + std::max(4096, vectorized::SpillStream::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; + size_t row_count = 0; std::vector> spill_infos( Base::_shared_state->partition_count); @@ -85,6 +103,7 @@ class PartitionedAggSinkLocalState for (int i = 0; i < Base::_shared_state->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); @@ -100,6 +119,7 @@ class PartitionedAggSinkLocalState auto spill_null_key_data = (hash_null_key_data && i == Base::_shared_state->partition_count - 1); if (spill_infos[i].keys_.size() > 0 || spill_null_key_data) { + _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_, spill_null_key_data @@ -130,10 +150,6 @@ class PartitionedAggSinkLocalState auto status = spill_partition->get_spill_stream(state, Base::_parent->node_id(), Base::profile(), spill_stream); RETURN_IF_ERROR(status); - spill_stream->set_write_counters(Base::_spill_serialize_block_timer, - Base::_spill_block_count, Base::_spill_data_size, - Base::_spill_write_disk_timer, - Base::_spill_write_wait_io_timer); status = to_block(context, keys, values, null_key_data); RETURN_IF_ERROR(status); @@ -148,14 +164,9 @@ class PartitionedAggSinkLocalState keys.clear(); values.clear(); } - status = spill_stream->prepare_spill(); + status = spill_stream->spill_block(state, block_, false); RETURN_IF_ERROR(status); - { - SCOPED_TIMER(_spill_write_disk_timer); - status = spill_stream->spill_block(state, block_, false); - } - RETURN_IF_ERROR(status); status = spill_partition->flush_if_full(); _reset_tmp_data(); return status; @@ -260,8 +271,6 @@ class PartitionedAggSinkLocalState std::unique_ptr _runtime_state; - std::shared_ptr _finish_dependency; - // temp structures during spilling vectorized::MutableColumns key_columns_; vectorized::MutableColumns value_columns_; @@ -282,6 +291,9 @@ class PartitionedAggSinkLocalState RuntimeProfile::Counter* _deserialize_data_timer = nullptr; RuntimeProfile::Counter* _max_row_size_counter = nullptr; RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; + RuntimeProfile::Counter* _memory_usage_container = nullptr; + RuntimeProfile::Counter* _memory_usage_arena = nullptr; + RuntimeProfile::Counter* _memory_usage_reserved = nullptr; RuntimeProfile::HighWaterMarkCounter* _serialize_key_arena_memory_usage = nullptr; RuntimeProfile::Counter* _spill_serialize_hash_table_timer = nullptr; @@ -317,13 +329,16 @@ class PartitionedAggSinkOperatorX : public DataSinkOperatorX& spill_context) 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_bits = 4; + size_t _spill_partition_count = 32; }; #include "common/compile_check_end.h" } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp index 55a06f98add1db..cd9a0a72bb1421 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp @@ -17,15 +17,20 @@ #include "partitioned_aggregation_source_operator.h" +#include + #include #include "aggregation_source_operator.h" #include "common/exception.h" +#include "common/logging.h" #include "common/status.h" #include "pipeline/exec/operator.h" #include "pipeline/exec/spill_utils.h" +#include "pipeline/pipeline_task.h" #include "runtime/fragment_mgr.h" #include "util/runtime_profile.h" +#include "vec/spill/spill_stream.h" #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { @@ -39,6 +44,10 @@ Status PartitionedAggLocalState::init(RuntimeState* state, LocalStateInfo& info) SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); _init_counters(); + _spill_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), + "AggSourceSpillDependency", true); + state->get_task()->add_spill_dependency(_spill_dependency.get()); + return Status::OK(); } @@ -60,13 +69,20 @@ void PartitionedAggLocalState::_init_counters() { _hash_table_iterate_timer = ADD_TIMER(profile(), "HashTableIterateTime"); _insert_keys_to_column_timer = ADD_TIMER(profile(), "InsertKeysToColumnTime"); _serialize_data_timer = ADD_TIMER(profile(), "SerializeDataTime"); - _hash_table_size_counter = ADD_COUNTER(profile(), "HashTableSize", TUnit::UNIT); + _hash_table_size_counter = ADD_COUNTER_WITH_LEVEL(profile(), "HashTableSize", TUnit::UNIT, 1); _merge_timer = ADD_TIMER(profile(), "MergeTime"); _deserialize_data_timer = ADD_TIMER(profile(), "DeserializeAndMergeTime"); _hash_table_compute_timer = ADD_TIMER(profile(), "HashTableComputeTime"); _hash_table_emplace_timer = ADD_TIMER(profile(), "HashTableEmplaceTime"); - _hash_table_input_counter = ADD_COUNTER(profile(), "HashTableInputCount", TUnit::UNIT); + _hash_table_input_counter = + ADD_COUNTER_WITH_LEVEL(profile(), "HashTableInputCount", TUnit::UNIT, 1); + _hash_table_memory_usage = + ADD_COUNTER_WITH_LEVEL(profile(), "HashTableMemoryUsage", TUnit::BYTES, 1); + + _memory_usage_container = + ADD_COUNTER_WITH_LEVEL(profile(), "MemoryUsageContainer", TUnit::BYTES, 1); + _memory_usage_arena = ADD_COUNTER_WITH_LEVEL(profile(), "MemoryUsageArena", TUnit::BYTES, 1); } #define UPDATE_PROFILE(counter, name) \ @@ -84,6 +100,9 @@ void PartitionedAggLocalState::update_profile(RuntimeProfile* child_profile) { UPDATE_PROFILE(_insert_keys_to_column_timer, "InsertKeysToColumnTime"); UPDATE_PROFILE(_serialize_data_timer, "SerializeDataTime"); UPDATE_PROFILE(_hash_table_size_counter, "HashTableSize"); + UPDATE_PROFILE(_hash_table_memory_usage, "HashTableMemoryUsage"); + UPDATE_PROFILE(_memory_usage_container, "MemoryUsageContainer"); + UPDATE_PROFILE(_memory_usage_arena, "MemoryUsageArena"); } Status PartitionedAggLocalState::close(RuntimeState* state) { @@ -126,30 +145,52 @@ bool PartitionedAggSourceOperatorX::is_serial_operator() const { Status PartitionedAggSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { auto& local_state = get_local_state(state); + local_state.copy_shared_spill_profile(); + Status status; Defer defer {[&]() { - if (!local_state._status.ok() || *eos) { + if (!status.ok() || *eos) { local_state._shared_state->close(); } }}; local_state.inc_running_big_mem_op_num(state); SCOPED_TIMER(local_state.exec_time_counter()); - RETURN_IF_ERROR(local_state._status); - - if (local_state._shared_state->is_spilled) { - local_state._status = local_state.initiate_merge_spill_partition_agg_data(state); - RETURN_IF_ERROR(local_state._status); - /// When `_is_merging` is true means we are reading spilled data and merging the data into hash table. - if (local_state._is_merging) { + 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; 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._need_to_merge_data_for_current_partition = false; } // not spilled in sink or current partition still has data auto* runtime_state = local_state._runtime_state.get(); - local_state._status = _agg_source_operator->get_block(runtime_state, block, eos); - RETURN_IF_ERROR(local_state._status); + 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 (local_state._runtime_state) { auto* source_local_state = local_state._runtime_state->get_local_state(_agg_source_operator->operator_id()); @@ -158,6 +199,10 @@ Status PartitionedAggSourceOperatorX::get_block(RuntimeState* state, vectorized: if (*eos) { if (local_state._shared_state->is_spilled && !local_state._shared_state->spill_partitions.empty()) { + local_state._current_partition_eos = false; + local_state._need_to_merge_data_for_current_partition = true; + status = local_state._shared_state->in_mem_shared_state->reset_hash_table(); + RETURN_IF_ERROR(status); *eos = false; } } @@ -191,110 +236,117 @@ Status PartitionedAggLocalState::setup_in_memory_agg_op(RuntimeState* state) { return source_local_state->open(state); } -Status PartitionedAggLocalState::initiate_merge_spill_partition_agg_data(RuntimeState* state) { - DCHECK(!_is_merging); - Base::_shared_state->in_mem_shared_state->aggregate_data_container->init_once(); - if (Base::_shared_state->in_mem_shared_state->aggregate_data_container->iterator != - Base::_shared_state->in_mem_shared_state->aggregate_data_container->end() || - _shared_state->spill_partitions.empty()) { +Status PartitionedAggLocalState::recover_blocks_from_disk(RuntimeState* state, bool& has_data) { + const auto query_id = state->query_id(); + + if (_shared_state->spill_partitions.empty()) { + _shared_state->close(); + has_data = false; return Status::OK(); } - _is_merging = true; - VLOG_DEBUG << "query " << print_id(state->query_id()) << " agg node " << _parent->node_id() - << " merge spilled agg data"; - - RETURN_IF_ERROR(Base::_shared_state->in_mem_shared_state->reset_hash_table()); - _dependency->Dependency::block(); - - auto query_id = state->query_id(); - - MonotonicStopWatch submit_timer; - submit_timer.start(); - auto spill_func = [this, state, query_id, submit_timer] { - _spill_wait_in_queue_timer->update(submit_timer.elapsed_time()); + has_data = true; + auto spill_func = [this, state, query_id] { + Status status; Defer defer {[&]() { - if (!_status.ok() || state->is_cancelled()) { - if (!_status.ok()) { - LOG(WARNING) << "query " << print_id(query_id) << " agg node " - << _parent->node_id() - << " merge spilled agg data error: " << _status; + 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(); - } else if (_shared_state->spill_partitions.empty()) { - VLOG_DEBUG << "query " << print_id(query_id) << " agg node " << _parent->node_id() - << " merge spilled agg data finish"; } - Base::_shared_state->in_mem_shared_state->aggregate_data_container->init_once(); - _is_merging = false; - _dependency->Dependency::set_ready(); }}; bool has_agg_data = false; - auto& parent = Base::_parent->template cast(); + size_t accumulated_blocks_size = 0; while (!state->is_cancelled() && !has_agg_data && !_shared_state->spill_partitions.empty()) { - for (auto& stream : _shared_state->spill_partitions[0]->spill_streams_) { - stream->set_read_counters(Base::_spill_read_data_time, - Base::_spill_deserialize_time, Base::_spill_read_bytes, - Base::_spill_read_wait_io_timer); + 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(profile()); vectorized::Block block; bool eos = false; while (!eos && !state->is_cancelled()) { { - SCOPED_TIMER(Base::_spill_recover_time); DBUG_EXECUTE_IF("fault_inject::partitioned_agg_source::recover_spill_data", { - _status = Status::Error( + status = Status::Error( "fault_inject partitioned_agg_source " "recover_spill_data failed"); }); - if (_status.ok()) { - _status = stream->read_next_block_sync(&block, &eos); + if (status.ok()) { + status = stream->read_next_block_sync(&block, &eos); } } - RETURN_IF_ERROR(_status); + RETURN_IF_ERROR(status); if (!block.empty()) { has_agg_data = true; - _status = parent._agg_source_operator - ->merge_with_serialized_key_helper( - _runtime_state.get(), &block); - RETURN_IF_ERROR(_status); + accumulated_blocks_size += block.allocated_bytes(); + _blocks.emplace_back(std::move(block)); + + if (accumulated_blocks_size >= + vectorized::SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { + break; + } } } - (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + + _current_partition_eos = eos; + + if (_current_partition_eos) { + (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); + _shared_state->spill_partitions[0]->spill_streams_.pop_front(); + } + } + + if (_shared_state->spill_partitions[0]->spill_streams_.empty()) { + _shared_state->spill_partitions.pop_front(); } - _shared_state->spill_partitions.pop_front(); - } - if (_shared_state->spill_partitions.empty()) { - _shared_state->close(); } - return _status; + + VLOG_DEBUG << fmt::format( + "Query:{}, agg probe:{}, task:{}, recover partitioned finished, partitions " + "left:{}, bytes read:{}, spill dep:{}", + print_id(query_id), _parent->node_id(), state->task_id(), + _shared_state->spill_partitions.size(), accumulated_blocks_size, + (void*)(_spill_dependency.get())); + return status; }; - auto exception_catch_func = [spill_func, query_id, this]() { + auto exception_catch_func = [this, state, spill_func, 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"); ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_id, st); - return; + return st; }); auto status = [&]() { RETURN_IF_CATCH_EXCEPTION({ return spill_func(); }); }(); - - if (!status.ok()) { - _status = status; - } + 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; }; DBUG_EXECUTE_IF("fault_inject::partitioned_agg_source::submit_func", { return Status::Error( "fault_inject partitioned_agg_source submit_func failed"); }); + _spill_dependency->block(); + + VLOG_DEBUG << fmt::format( + "Query:{}, agg probe:{}, task:{}, begin to recover, partitions left:{}, " + "_spill_dependency:{}", + print_id(query_id), _parent->node_id(), state->task_id(), + _shared_state->spill_partitions.size(), (void*)(_spill_dependency.get())); return ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool()->submit( - std::make_shared(state, _shared_state->shared_from_this(), - exception_catch_func)); + std::make_shared(state, _spill_dependency, _runtime_profile.get(), + _shared_state->shared_from_this(), + exception_catch_func)); } #include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h index 6fb0ecaba01e20..c990e3d90ea7d9 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_source_operator.h +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.h @@ -42,7 +42,7 @@ class PartitionedAggLocalState final : public PipelineXSpillLocalState _runtime_state; bool _opened = false; - Status _status; std::unique_ptr> _spill_merge_promise; std::future _spill_merge_future; bool _current_partition_eos = true; - bool _is_merging = false; + bool _need_to_merge_data_for_current_partition = true; + + std::shared_ptr _spill_dependency; + std::vector _blocks; std::unique_ptr _internal_runtime_profile; RuntimeProfile::Counter* _get_results_timer = nullptr; @@ -73,6 +75,10 @@ class PartitionedAggLocalState final : public PipelineXSpillLocalState { diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp index 3e7f95374f53d2..0588afcca0adcb 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp @@ -17,9 +17,21 @@ #include "partitioned_hash_join_probe_operator.h" +#include +#include + +#include +#include + +#include "common/exception.h" +#include "common/logging.h" +#include "common/status.h" #include "pipeline/pipeline_task.h" #include "runtime/fragment_mgr.h" #include "util/mem_info.h" +#include "util/runtime_profile.h" +#include "vec/core/block.h" +#include "vec/spill/spill_stream.h" #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { @@ -32,6 +44,8 @@ PartitionedHashJoinProbeLocalState::PartitionedHashJoinProbeLocalState(RuntimeSt Status PartitionedHashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXSpillLocalState::init(state, info)); + init_spill_write_counters(); + SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); _internal_runtime_profile.reset(new RuntimeProfile("internal_profile")); @@ -40,32 +54,30 @@ Status PartitionedHashJoinProbeLocalState::init(RuntimeState* state, LocalStateI _partitioned_blocks.resize(p._partition_count); _probe_spilling_streams.resize(p._partition_count); - _spill_and_partition_label = ADD_LABEL_COUNTER(profile(), "Partition"); - _partition_timer = ADD_CHILD_TIMER(profile(), "PartitionTime", "Partition"); - _partition_shuffle_timer = ADD_CHILD_TIMER(profile(), "PartitionShuffleTime", "Partition"); - _spill_build_rows = ADD_CHILD_COUNTER(profile(), "SpillBuildRows", TUnit::UNIT, "Spill"); - _spill_build_timer = ADD_CHILD_TIMER_WITH_LEVEL(profile(), "SpillBuildTime", "Spill", 1); - _recovery_build_rows = ADD_CHILD_COUNTER(profile(), "RecoveryBuildRows", TUnit::UNIT, "Spill"); - _recovery_build_timer = ADD_CHILD_TIMER_WITH_LEVEL(profile(), "RecoveryBuildTime", "Spill", 1); - _spill_probe_rows = ADD_CHILD_COUNTER(profile(), "SpillProbeRows", TUnit::UNIT, "Spill"); - _recovery_probe_rows = ADD_CHILD_COUNTER(profile(), "RecoveryProbeRows", TUnit::UNIT, "Spill"); - _spill_build_blocks = ADD_CHILD_COUNTER(profile(), "SpillBuildBlocks", TUnit::UNIT, "Spill"); - _recovery_build_blocks = - ADD_CHILD_COUNTER(profile(), "RecoveryBuildBlocks", TUnit::UNIT, "Spill"); - _spill_probe_blocks = ADD_CHILD_COUNTER(profile(), "SpillProbeBlocks", TUnit::UNIT, "Spill"); - _spill_probe_timer = ADD_CHILD_TIMER_WITH_LEVEL(profile(), "SpillProbeTime", "Spill", 1); - _recovery_probe_blocks = - ADD_CHILD_COUNTER(profile(), "RecoveryProbeBlocks", TUnit::UNIT, "Spill"); - _recovery_probe_timer = ADD_CHILD_TIMER_WITH_LEVEL(profile(), "RecoveryProbeTime", "Spill", 1); - - _spill_serialize_block_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillSerializeBlockTime", "Spill", 1); - _spill_write_disk_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteDiskTime", "Spill", 1); - _spill_data_size = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteDataSize", - TUnit::BYTES, "Spill", 1); - _spill_block_count = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteBlockCount", - TUnit::UNIT, "Spill", 1); + _spill_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), + "HashJoinProbeSpillDependency", true); + state->get_task()->add_spill_dependency(_spill_dependency.get()); + + _partition_timer = ADD_TIMER(profile(), "SpillPartitionTime"); + _partition_shuffle_timer = ADD_TIMER(profile(), "SpillPartitionShuffleTime"); + _spill_build_rows = ADD_COUNTER(profile(), "SpillBuildRows", TUnit::UNIT); + _spill_build_timer = ADD_TIMER_WITH_LEVEL(profile(), "SpillBuildTime", 1); + _recovery_build_rows = ADD_COUNTER(profile(), "SpillRecoveryBuildRows", TUnit::UNIT); + _recovery_build_timer = ADD_TIMER_WITH_LEVEL(profile(), "SpillRecoveryBuildTime", 1); + _spill_probe_rows = ADD_COUNTER(profile(), "SpillProbeRows", TUnit::UNIT); + _recovery_probe_rows = ADD_COUNTER(profile(), "SpillRecoveryProbeRows", TUnit::UNIT); + _spill_build_blocks = ADD_COUNTER(profile(), "SpillBuildBlocks", TUnit::UNIT); + _recovery_build_blocks = ADD_COUNTER(profile(), "SpillRecoveryBuildBlocks", TUnit::UNIT); + _spill_probe_blocks = ADD_COUNTER(profile(), "SpillProbeBlocks", TUnit::UNIT); + _spill_probe_timer = ADD_TIMER_WITH_LEVEL(profile(), "SpillProbeTime", 1); + _recovery_probe_blocks = ADD_COUNTER(profile(), "SpillRecoveryProbeBlocks", TUnit::UNIT); + _recovery_probe_timer = ADD_TIMER_WITH_LEVEL(profile(), "SpillRecoveryProbeTime", 1); + _get_child_next_timer = ADD_TIMER_WITH_LEVEL(profile(), "GetChildNextTime", 1); + + _memory_usage_reserved = + ADD_COUNTER_WITH_LEVEL(profile(), "MemoryUsageReserved", TUnit::UNIT, 1); + + _probe_blocks_bytes = ADD_COUNTER_WITH_LEVEL(profile(), "ProbeBlocksBytes", TUnit::BYTES, 1); // Build phase _build_phase_label = ADD_LABEL_COUNTER(profile(), "BuildPhase"); @@ -81,6 +93,10 @@ Status PartitionedHashJoinProbeLocalState::init(RuntimeState* state, LocalStateI _build_expr_call_timer = ADD_CHILD_TIMER(profile(), "BuildExprCallTime", "BuildPhase"); _build_side_compute_hash_timer = ADD_CHILD_TIMER(profile(), "BuildSideHashComputingTime", "BuildPhase"); + + _hash_table_memory_usage = + ADD_COUNTER_WITH_LEVEL(profile(), "MemoryUsageHashTable", TUnit::BYTES, 1); + _allocate_resource_timer = ADD_CHILD_TIMER(profile(), "AllocateResourceTime", "BuildPhase"); // Probe phase @@ -161,21 +177,27 @@ Status PartitionedHashJoinProbeLocalState::spill_probe_blocks(RuntimeState* stat auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(); auto query_id = state->query_id(); - MonotonicStopWatch submit_timer; - submit_timer.start(); - auto spill_func = [query_id, state, submit_timer, this] { - _spill_wait_in_queue_timer->update(submit_timer.elapsed_time()); + auto spill_func = [query_id, state, this] { 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 && partitioned_block->allocated_bytes() >= - vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { - blocks.emplace_back(partitioned_block->to_block()); - partitioned_block.reset(); + if (partitioned_block) { + const auto size = partitioned_block->allocated_bytes(); + if (size >= vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + blocks.emplace_back(partitioned_block->to_block()); + partitioned_block.reset(); + } else { + not_revoked_size += size; + } + } + + if (blocks.empty()) { + continue; } auto& spilling_stream = _probe_spilling_streams[partition_index]; @@ -184,133 +206,110 @@ Status PartitionedHashJoinProbeLocalState::spill_probe_blocks(RuntimeState* stat state, spilling_stream, print_id(state->query_id()), "hash_probe", _parent->node_id(), std::numeric_limits::max(), std::numeric_limits::max(), _runtime_profile.get())); - RETURN_IF_ERROR(spilling_stream->prepare_spill()); - spilling_stream->set_write_counters( - _spill_serialize_block_timer, _spill_block_count, _spill_data_size, - _spill_write_disk_timer, _spill_write_wait_io_timer); } - COUNTER_UPDATE(_spill_probe_blocks, blocks.size()); + auto merged_block = vectorized::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"); }); - RETURN_IF_ERROR(spilling_stream->spill_block(state, block, false)); - COUNTER_UPDATE(_spill_probe_rows, block.rows()); + } + + 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); } } - VLOG_DEBUG << "query: " << print_id(query_id) - << " hash probe revoke done, node: " << p.node_id() - << ", task: " << state->task_id(); + + 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(); }; - auto exception_catch_func = [query_id, spill_func, this]() { + auto exception_catch_func = [query_id, spill_func]() { DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::spill_probe_blocks_cancel", { - ExecEnv::GetInstance()->fragment_mgr()->cancel_query( - query_id, Status::InternalError("fault_inject partitioned_hash_join_probe " - "spill_probe_blocks canceled")); - return; + auto status = Status::InternalError( + "fault_inject partitioned_hash_join_probe " + "spill_probe_blocks canceled"); + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_id, status); + return status; }); auto status = [&]() { RETURN_IF_CATCH_EXCEPTION({ return spill_func(); }); }(); - - if (!status.ok()) { - _spill_status_ok = false; - _spill_status = std::move(status); - } - _dependency->set_ready(); + return status; }; - _dependency->block(); + _spill_dependency->block(); 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"); }); - auto spill_runnable = std::make_shared(state, _shared_state->shared_from_this(), - exception_catch_func); + auto spill_runnable = std::make_shared( + state, _spill_dependency, _runtime_profile.get(), _shared_state->shared_from_this(), + exception_catch_func); return spill_io_pool->submit(std::move(spill_runnable)); } Status PartitionedHashJoinProbeLocalState::finish_spilling(uint32_t partition_index) { - auto& build_spilling_stream = _shared_state->spilled_streams[partition_index]; - if (build_spilling_stream) { - RETURN_IF_ERROR(build_spilling_stream->spill_eof()); - build_spilling_stream->set_read_counters(_spill_read_data_time, _spill_deserialize_time, - _spill_read_bytes, _spill_read_wait_io_timer); - } - 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(_spill_read_data_time, _spill_deserialize_time, - _spill_read_bytes, _spill_read_wait_io_timer); + probe_spilling_stream->set_read_counters(profile()); } return Status::OK(); } -Status PartitionedHashJoinProbeLocalState::recovery_build_blocks_from_disk(RuntimeState* state, - uint32_t partition_index, - bool& has_data) { - VLOG_DEBUG << "query: " << print_id(state->query_id()) << ", node: " << _parent->node_id() - << ", task id: " << state->task_id() << ", partition: " << partition_index - << " recovery_build_blocks_from_disk"; +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(); } - - auto& mutable_block = _shared_state->partitioned_build_blocks[partition_index]; - if (!mutable_block) { - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(spilled_stream); - spilled_stream.reset(); - return Status::OK(); - } - - std::weak_ptr shared_state_holder = - _shared_state->shared_from_this(); + spilled_stream->set_read_counters(profile()); auto query_id = state->query_id(); - MonotonicStopWatch submit_timer; - submit_timer.start(); - - auto read_func = [this, query_id, state, spilled_stream = spilled_stream, &mutable_block, - shared_state_holder, submit_timer, partition_index] { - auto shared_state_sptr = shared_state_holder.lock(); - if (!shared_state_sptr || state->is_cancelled()) { - LOG(INFO) << "query: " << print_id(query_id) - << " execution_context released, maybe query was cancelled."; - return; - } - - _spill_wait_in_queue_timer->update(submit_timer.elapsed_time()); + auto read_func = [this, query_id, state, spilled_stream = spilled_stream, partition_index] { SCOPED_TIMER(_recovery_build_timer); bool eos = false; - VLOG_DEBUG << "query: " << print_id(state->query_id()) << ", node: " << _parent->node_id() - << ", task id: " << state->task_id() << ", partition: " << partition_index - << ", recoverying build data"; + 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) { vectorized::Block block; - Status st; DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recover_build_blocks", { - st = Status::Error( + status = Status::Error( "fault_inject partitioned_hash_join_probe recover_build_blocks failed"); }); - if (st.ok()) { - st = spilled_stream->read_next_block_sync(&block, &eos); + if (status.ok()) { + status = spilled_stream->read_next_block_sync(&block, &eos); } - if (!st.ok()) { - _spill_status_ok = false; - _spill_status = std::move(st); + if (!status.ok()) { break; } COUNTER_UPDATE(_recovery_build_rows, block.rows()); @@ -321,63 +320,62 @@ Status PartitionedHashJoinProbeLocalState::recovery_build_blocks_from_disk(Runti } if (UNLIKELY(state->is_cancelled())) { - LOG(INFO) << "recovery build block when canceled."; + 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 (mutable_block->empty()) { - *mutable_block = std::move(block); + if (!_recovered_build_block) { + _recovered_build_block = vectorized::MutableBlock::create_unique(std::move(block)); } else { - DCHECK_EQ(mutable_block->columns(), block.columns()); - st = mutable_block->merge(std::move(block)); - if (!st.ok()) { - _spill_status_ok = false; - _spill_status = std::move(st); + 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() >= + vectorized::SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { + break; + } } - ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(spilled_stream); - shared_state_sptr->spilled_streams[partition_index].reset(); - VLOG_DEBUG << "query: " << print_id(state->query_id()) << ", node: " << _parent->node_id() - << ", task id: " << state->task_id() << ", partition: " << partition_index - << ", recovery build data done"; + 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); + } + return status; }; - auto exception_catch_func = [read_func, query_id, this]() { + auto exception_catch_func = [read_func, query_id]() { DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recover_build_blocks_cancel", { - ExecEnv::GetInstance()->fragment_mgr()->cancel_query( - query_id, Status::InternalError("fault_inject partitioned_hash_join_probe " - "recover_build_blocks canceled")); - return; + auto status = Status::InternalError( + "fault_inject partitioned_hash_join_probe " + "recover_build_blocks canceled"); + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_id, status); + return status; }); auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION(read_func()); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(read_func()); return Status::OK(); }(); - if (!status.ok()) { - _spill_status_ok = false; - _spill_status = std::move(status); - } - _dependency->set_ready(); + return status; }; auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(); has_data = true; - _dependency->block(); - { - auto* pipeline_task = state->get_task(); - if (pipeline_task) { - auto& p = _parent->cast(); - VLOG_DEBUG << "query: " << print_id(state->query_id()) << ", node: " << p.node_id() - << ", task id: " << state->task_id() << ", partition: " << partition_index - << ", dependency: " << _dependency - << ", task debug_string: " << pipeline_task->debug_string(); - } - } + _spill_dependency->block(); DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recovery_build_blocks_submit_func", { @@ -385,11 +383,10 @@ Status PartitionedHashJoinProbeLocalState::recovery_build_blocks_from_disk(Runti "fault_inject partitioned_hash_join_probe " "recovery_build_blocks submit_func failed"); }); - auto spill_runnable = std::make_shared(state, _shared_state->shared_from_this(), - exception_catch_func); - VLOG_DEBUG << "query: " << print_id(state->query_id()) << ", node: " << _parent->node_id() - << ", task id: " << state->task_id() << ", partition: " << partition_index - << " recovery_build_blocks_from_disk submit func"; + + auto spill_runnable = std::make_shared( + state, _spill_dependency, _runtime_profile.get(), _shared_state->shared_from_this(), + exception_catch_func); return spill_io_pool->submit(std::move(spill_runnable)); } @@ -398,40 +395,39 @@ std::string PartitionedHashJoinProbeLocalState::debug_string(int indentation_lev bool need_more_input_data; if (_shared_state->need_to_spill) { need_more_input_data = !_child_eos; - } else if (_runtime_state) { - need_more_input_data = p._inner_probe_operator->need_more_input_data(_runtime_state.get()); + } 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()); } else { need_more_input_data = true; } fmt::memory_buffer debug_string_buffer; fmt::format_to(debug_string_buffer, "{}, short_circuit_for_probe: {}, need_to_spill: {}, child_eos: {}, " - "_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->need_to_spill, _child_eos, _runtime_state != nullptr, - need_more_input_data); + _shared_state->need_to_spill, _child_eos, + _shared_state->inner_runtime_state != nullptr, need_more_input_data); return fmt::to_string(debug_string_buffer); } -Status PartitionedHashJoinProbeLocalState::recovery_probe_blocks_from_disk(RuntimeState* state, - uint32_t partition_index, - bool& has_data) { +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) { return Status::OK(); } + spilled_stream->set_read_counters(profile()); auto& blocks = _probe_blocks[partition_index]; auto query_id = state->query_id(); - MonotonicStopWatch submit_timer; - submit_timer.start(); - auto read_func = [this, query_id, &spilled_stream, &blocks, submit_timer] { - _spill_wait_in_queue_timer->update(submit_timer.elapsed_time()); + auto read_func = [this, query_id, partition_index, &spilled_stream, &blocks] { SCOPED_TIMER(_recovery_probe_timer); vectorized::Block block; @@ -441,49 +437,54 @@ Status PartitionedHashJoinProbeLocalState::recovery_probe_blocks_from_disk(Runti st = Status::Error( "fault_inject partitioned_hash_join_probe recover_probe_blocks failed"); }); - if (st.ok()) { + + size_t read_size = 0; + while (!eos && !_state->is_cancelled() && st.ok()) { st = spilled_stream->read_next_block_sync(&block, &eos); - } - if (!st.ok()) { - _spill_status_ok = false; - _spill_status = std::move(st); - } else { - COUNTER_UPDATE(_recovery_probe_rows, block.rows()); - COUNTER_UPDATE(_recovery_probe_blocks, 1); - blocks.emplace_back(std::move(block)); - } + if (!st.ok()) { + break; + } else { + COUNTER_UPDATE(_recovery_probe_rows, block.rows()); + COUNTER_UPDATE(_recovery_probe_blocks, 1); + read_size += block.allocated_bytes(); + blocks.emplace_back(std::move(block)); + } + if (read_size >= vectorized::SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { + break; + } + } if (eos) { - VLOG_DEBUG << "query: " << print_id(query_id) - << ", recovery probe data done: " << spilled_stream->get_spill_dir(); + 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(); } + return st; }; - auto exception_catch_func = [read_func, query_id, this]() { + auto exception_catch_func = [read_func, query_id]() { DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recover_probe_blocks_cancel", { - ExecEnv::GetInstance()->fragment_mgr()->cancel_query( - query_id, Status::InternalError("fault_inject partitioned_hash_join_probe " - "recover_probe_blocks canceled")); - return; + auto status = Status::InternalError( + "fault_inject partitioned_hash_join_probe " + "recover_probe_blocks canceled"); + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_id, status); + return status; }); auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION(read_func()); + RETURN_IF_ERROR_OR_CATCH_EXCEPTION(read_func()); return Status::OK(); }(); - if (!status.ok()) { - _spill_status_ok = false; - _spill_status = std::move(status); - } - _dependency->set_ready(); + return status; }; auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(); DCHECK(spill_io_pool != nullptr); - _dependency->block(); + _spill_dependency->block(); has_data = true; DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_probe::recovery_probe_blocks_submit_func", { @@ -491,8 +492,9 @@ Status PartitionedHashJoinProbeLocalState::recovery_probe_blocks_from_disk(Runti "fault_inject partitioned_hash_join_probe " "recovery_probe_blocks submit_func failed"); }); - return spill_io_pool->submit(std::make_shared( - state, _shared_state->shared_from_this(), exception_catch_func)); + return spill_io_pool->submit(std::make_shared( + state, _spill_dependency, _runtime_profile.get(), _shared_state->shared_from_this(), + exception_catch_func)); } PartitionedHashJoinProbeOperatorX::PartitionedHashJoinProbeOperatorX(ObjectPool* pool, @@ -516,7 +518,7 @@ Status PartitionedHashJoinProbeOperatorX::init(const TPlanNode& tnode, RuntimeSt auto tnode_ = _tnode; tnode_.runtime_filters.clear(); - for (auto& conjunct : tnode.hash_join_node.eq_join_conjuncts) { + for (const auto& conjunct : tnode.hash_join_node.eq_join_conjuncts) { _probe_exprs.emplace_back(conjunct.left); } _partitioner = std::make_unique(_partition_count); @@ -568,6 +570,7 @@ Status PartitionedHashJoinProbeOperatorX::push(RuntimeState* state, vectorized:: } 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(); if (UNLIKELY(count == 0)) { @@ -585,38 +588,52 @@ Status PartitionedHashJoinProbeOperatorX::push(RuntimeState* state, vectorized:: (eos && partitioned_blocks[i]->rows() > 0)) { local_state._probe_blocks[i].emplace_back(partitioned_blocks[i]->to_block()); 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(); } } + COUNTER_SET(local_state._probe_blocks_bytes, bytes_of_blocks); + return Status::OK(); } Status PartitionedHashJoinProbeOperatorX::_setup_internal_operator_for_non_spill( PartitionedHashJoinProbeLocalState& local_state, RuntimeState* state) { DCHECK(local_state._shared_state->inner_runtime_state); - local_state._runtime_state = std::move(local_state._shared_state->inner_runtime_state); local_state._in_mem_shared_state_sptr = std::move(local_state._shared_state->inner_shared_state); + + auto* sink_state = local_state._shared_state->inner_runtime_state->get_sink_local_state(); + if (sink_state != nullptr) { + COUNTER_SET(local_state._hash_table_memory_usage, + sink_state->profile()->get_counter("MemoryUsageHashTable")->value()); + } return Status::OK(); } Status PartitionedHashJoinProbeOperatorX::_setup_internal_operators( PartitionedHashJoinProbeLocalState& local_state, RuntimeState* state) const { - if (local_state._runtime_state) { + if (local_state._shared_state->inner_runtime_state) { _update_profile_from_internal_states(local_state); } - local_state._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._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._runtime_state->set_be_number(state->be_number()); + local_state._shared_state->inner_runtime_state->set_be_number(state->be_number()); - local_state._runtime_state->set_desc_tbl(&state->desc_tbl()); - local_state._runtime_state->resize_op_id_to_local_state(-1); - local_state._runtime_state->set_runtime_filter_mgr(state->local_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(); @@ -624,23 +641,23 @@ Status PartitionedHashJoinProbeOperatorX::_setup_internal_operators( LocalSinkStateInfo info {0, local_state._internal_runtime_profile.get(), -1, local_state._in_mem_shared_state_sptr.get(), {}, {}}; - RETURN_IF_ERROR( - _inner_sink_operator->setup_local_state(local_state._runtime_state.get(), info)); + RETURN_IF_ERROR(_inner_sink_operator->setup_local_state( + local_state._shared_state->inner_runtime_state.get(), info)); LocalStateInfo state_info {local_state._internal_runtime_profile.get(), {}, local_state._in_mem_shared_state_sptr.get(), {}, 0}; - RETURN_IF_ERROR( - _inner_probe_operator->setup_local_state(local_state._runtime_state.get(), state_info)); + RETURN_IF_ERROR(_inner_probe_operator->setup_local_state( + local_state._shared_state->inner_runtime_state.get(), state_info)); - auto* sink_local_state = local_state._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._runtime_state->get_local_state(_inner_probe_operator->operator_id()); + 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)); @@ -655,32 +672,50 @@ Status PartitionedHashJoinProbeOperatorX::_setup_internal_operators( return Status::Error( "fault_inject partitioned_hash_join_probe sink failed"); }); - RETURN_IF_ERROR(_inner_sink_operator->sink(local_state._runtime_state.get(), &block, true)); - VLOG_DEBUG << "query: " << print_id(state->query_id()) - << ", internal build operator finished, node id: " << node_id() - << ", task id: " << state->task_id() - << ", partition: " << local_state._partition_cursor; + + RETURN_IF_ERROR(_inner_sink_operator->sink(local_state._shared_state->inner_runtime_state.get(), + &block, 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(), + _inner_sink_operator->get_memory_usage( + local_state._shared_state->inner_runtime_state.get())); + + COUNTER_SET(local_state._hash_table_memory_usage, + sink_local_state->profile()->get_counter("MemoryUsageHashTable")->value()); return Status::OK(); } Status PartitionedHashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) const { auto& local_state = get_local_state(state); - if (!local_state._spill_status_ok) { - DCHECK_NE(local_state._spill_status.code(), 0); - return local_state._spill_status; - } 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(); + } + } + if (local_state._need_to_setup_internal_operators) { - *eos = false; bool has_data = false; - RETURN_IF_ERROR(local_state.recovery_build_blocks_from_disk( + RETURN_IF_ERROR(local_state.recover_build_blocks_from_disk( state, local_state._partition_cursor, has_data)); if (has_data) { 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]; @@ -689,19 +724,20 @@ Status PartitionedHashJoinProbeOperatorX::pull(doris::RuntimeState* state, } } bool in_mem_eos = false; - auto* runtime_state = local_state._runtime_state.get(); + auto* runtime_state = local_state._shared_state->inner_runtime_state.get(); 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.recovery_probe_blocks_from_disk(state, partition_index, has_data)); + local_state.recover_probe_blocks_from_disk(state, partition_index, has_data)); if (!has_data) { vectorized::Block block; RETURN_IF_ERROR(_inner_probe_operator->push(runtime_state, &block, true)); - VLOG_DEBUG << "query: " << print_id(state->query_id()) << ", node: " << node_id() - << ", task: " << state->task_id() << "partition: " << partition_index - << " has no data to recovery"; + 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); break; } else { return Status::OK(); @@ -715,19 +751,20 @@ Status PartitionedHashJoinProbeOperatorX::pull(doris::RuntimeState* state, } } - RETURN_IF_ERROR(_inner_probe_operator->pull(local_state._runtime_state.get(), output_block, - &in_mem_eos)); + RETURN_IF_ERROR(_inner_probe_operator->pull( + local_state._shared_state->inner_runtime_state.get(), output_block, &in_mem_eos)); *eos = false; if (in_mem_eos) { - VLOG_DEBUG << "query: " << print_id(state->query_id()) << ", node: " << node_id() - << ", task: " << state->task_id() - << ", partition: " << local_state._partition_cursor; + VLOG_DEBUG << fmt::format( + "Query:{}, hash join probe:{}, task:{}," + " partition:{}, probe done", + print_id(state->query_id()), node_id(), state->task_id(), + local_state._partition_cursor); local_state._partition_cursor++; if (local_state._partition_cursor == _partition_count) { *eos = true; } else { - RETURN_IF_ERROR(local_state.finish_spilling(local_state._partition_cursor)); local_state._need_to_setup_internal_operators = true; } } @@ -739,8 +776,9 @@ bool PartitionedHashJoinProbeOperatorX::need_more_input_data(RuntimeState* state auto& local_state = get_local_state(state); if (local_state._shared_state->need_to_spill) { return !local_state._child_eos; - } else if (local_state._runtime_state) { - return _inner_probe_operator->need_more_input_data(local_state._runtime_state.get()); + } 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()); } else { return true; } @@ -748,6 +786,22 @@ bool PartitionedHashJoinProbeOperatorX::need_more_input_data(RuntimeState* state size_t PartitionedHashJoinProbeOperatorX::revocable_mem_size(RuntimeState* state) const { auto& local_state = get_local_state(state); + if (local_state._child_eos) { + 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 ? vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM + : vectorized::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) { @@ -758,7 +812,7 @@ size_t PartitionedHashJoinProbeOperatorX::revocable_mem_size(RuntimeState* state auto& partitioned_block = local_state._partitioned_blocks[i]; if (partitioned_block) { auto block_bytes = partitioned_block->allocated_bytes(); - if (block_bytes >= vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + if (block_bytes >= spill_size_threshold) { mem_size += block_bytes; } } @@ -766,10 +820,39 @@ size_t PartitionedHashJoinProbeOperatorX::revocable_mem_size(RuntimeState* state return mem_size; } +size_t PartitionedHashJoinProbeOperatorX::get_reserve_mem_size(RuntimeState* state) { + auto& local_state = get_local_state(state); + const auto need_to_spill = local_state._shared_state->need_to_spill; + if (!need_to_spill || !local_state._child_eos) { + return Base::get_reserve_mem_size(state); + } + + size_t size_to_reserve = vectorized::SpillStream::MAX_SPILL_WRITE_BATCH_MEM; + + 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 = JoinHashTable::calc_bucket_size(rows); + + size_to_reserve += bucket_size * sizeof(uint32_t); // JoinHashTable::first + size_to_reserve += rows * sizeof(uint32_t); // JoinHashTable::next + + if (_join_op == TJoinOp::FULL_OUTER_JOIN || _join_op == TJoinOp::RIGHT_OUTER_JOIN || + _join_op == TJoinOp::RIGHT_ANTI_JOIN || _join_op == TJoinOp::RIGHT_SEMI_JOIN) { + size_to_reserve += rows * sizeof(uint8_t); // JoinHashTable::visited + } + } + + 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 << "query: " << print_id(state->query_id()) << ", hash probe node: " << node_id() - << ", task: " << state->task_id(); + VLOG_DEBUG << fmt::format("Query:{}, hash join probe:{}, task:{}, revoke_memory", + print_id(state->query_id()), node_id(), state->task_id()); RETURN_IF_ERROR(local_state.spill_probe_blocks(state)); return Status::OK(); @@ -777,24 +860,30 @@ Status PartitionedHashJoinProbeOperatorX::_revoke_memory(RuntimeState* state) { bool PartitionedHashJoinProbeOperatorX::_should_revoke_memory(RuntimeState* state) const { auto& local_state = get_local_state(state); - const auto revocable_size = revocable_mem_size(state); - if (PipelineTask::should_revoke_memory(state, revocable_size)) { - return true; - } if (local_state._shared_state->need_to_spill) { - const auto min_revocable_size = state->min_revocable_mem(); - return revocable_size > min_revocable_size; + const auto revocable_size = _revocable_mem_size(state); + const auto min_revocable_size = state->spill_min_revocable_mem(); + + if (state->get_query_ctx()->low_memory_mode()) { + return revocable_size > + std::min(min_revocable_size, + static_cast( + vectorized::SpillStream::MAX_SPILL_WRITE_BATCH_MEM)); + } else { + return vectorized::SpillStream::MAX_SPILL_WRITE_BATCH_MEM; + } } return false; } void PartitionedHashJoinProbeOperatorX::_update_profile_from_internal_states( PartitionedHashJoinProbeLocalState& local_state) const { - if (local_state._runtime_state) { - auto* sink_local_state = local_state._runtime_state->get_sink_local_state(); + if (local_state._shared_state->inner_runtime_state) { + auto* sink_local_state = + local_state._shared_state->inner_runtime_state->get_sink_local_state(); local_state.update_build_profile(sink_local_state->profile()); - auto* probe_local_state = - local_state._runtime_state->get_local_state(_inner_probe_operator->operator_id()); + auto* probe_local_state = local_state._shared_state->inner_runtime_state->get_local_state( + _inner_probe_operator->operator_id()); local_state.update_probe_profile(probe_local_state->profile()); } } @@ -803,46 +892,50 @@ Status PartitionedHashJoinProbeOperatorX::get_block(RuntimeState* state, vectori bool* eos) { *eos = false; auto& local_state = get_local_state(state); - SCOPED_TIMER(local_state.exec_time_counter()); + local_state.copy_shared_spill_profile(); const auto need_to_spill = local_state._shared_state->need_to_spill; #ifndef NDEBUG Defer eos_check_defer([&] { if (*eos) { - LOG(INFO) << "query: " << print_id(state->query_id()) - << ", hash probe node: " << node_id() << ", task: " << state->task_id() - << ", eos with child eos: " << local_state._child_eos - << ", need spill: " << need_to_spill; + 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, need_to_spill); } }); #endif - if (need_more_input_data(state)) { - if (need_to_spill && _should_revoke_memory(state)) { - return _revoke_memory(state); - } - RETURN_IF_ERROR(_child->get_block_after_projects(state, local_state._child_block.get(), - &local_state._child_eos)); + Defer defer([&]() { + COUNTER_SET(local_state._memory_usage_reserved, + int64_t(local_state.estimate_memory_usage())); + }); - if (need_to_spill && local_state._child_eos) { - RETURN_IF_ERROR(local_state.finish_spilling(0)); + if (need_more_input_data(state)) { + { + SCOPED_TIMER(local_state._get_child_next_timer); + RETURN_IF_ERROR(_child->get_block_after_projects(state, local_state._child_block.get(), + &local_state._child_eos)); } + SCOPED_TIMER(local_state.exec_time_counter()); if (local_state._child_block->rows() == 0 && !local_state._child_eos) { return Status::OK(); } - Defer defer([&] { local_state._child_block->clear_column_data(); }); + Defer clear_defer([&] { local_state._child_block->clear_column_data(); }); if (need_to_spill) { - SCOPED_TIMER(local_state.exec_time_counter()); RETURN_IF_ERROR(push(state, local_state._child_block.get(), local_state._child_eos)); + if (_should_revoke_memory(state)) { + return _revoke_memory(state); + } } else { - if (UNLIKELY(!local_state._runtime_state)) { + if (UNLIKELY(!local_state._shared_state->inner_runtime_state)) { RETURN_IF_ERROR(_setup_internal_operator_for_non_spill(local_state, state)); } - RETURN_IF_ERROR(_inner_probe_operator->push(local_state._runtime_state.get(), - local_state._child_block.get(), - local_state._child_eos)); + RETURN_IF_ERROR(_inner_probe_operator->push( + local_state._shared_state->inner_runtime_state.get(), + local_state._child_block.get(), local_state._child_eos)); } } @@ -851,15 +944,16 @@ Status PartitionedHashJoinProbeOperatorX::get_block(RuntimeState* state, vectori if (need_to_spill) { RETURN_IF_ERROR(pull(state, block, eos)); } else { - RETURN_IF_ERROR( - _inner_probe_operator->pull(local_state._runtime_state.get(), block, eos)); + RETURN_IF_ERROR(_inner_probe_operator->pull( + local_state._shared_state->inner_runtime_state.get(), block, eos)); if (*eos) { _update_profile_from_internal_states(local_state); - local_state._runtime_state.reset(); + local_state._shared_state->inner_runtime_state.reset(); } } local_state.add_num_rows_returned(block->rows()); + COUNTER_UPDATE(local_state._blocks_returned_counter, 1); if (*eos) { _update_profile_from_internal_states(local_state); } diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h index a19e88d7203e62..226116ef7f42e2 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.h @@ -18,9 +18,11 @@ #pragma once #include +#include #include "common/status.h" #include "operator.h" +#include "pipeline/dependency.h" #include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/join_build_sink_operator.h" @@ -48,10 +50,10 @@ class PartitionedHashJoinProbeLocalState final Status spill_probe_blocks(RuntimeState* state); - Status recovery_build_blocks_from_disk(RuntimeState* state, uint32_t partition_index, - bool& has_data); - Status recovery_probe_blocks_from_disk(RuntimeState* state, uint32_t partition_index, - bool& has_data); + Status 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); @@ -72,24 +74,19 @@ class PartitionedHashJoinProbeLocalState final std::unique_ptr _child_block; bool _child_eos {false}; - std::mutex _spill_lock; - Status _spill_status; - - std::atomic _spilling_task_count {0}; - std::atomic _spill_status_ok {true}; - std::vector> _partitioned_blocks; + std::unique_ptr _recovered_build_block; std::map> _probe_blocks; std::vector _probe_spilling_streams; std::unique_ptr _partitioner; - std::unique_ptr _runtime_state; std::unique_ptr _internal_runtime_profile; bool _need_to_setup_internal_operators {true}; - RuntimeProfile::Counter* _spill_and_partition_label = nullptr; + std::shared_ptr _spill_dependency; + RuntimeProfile::Counter* _partition_timer = nullptr; RuntimeProfile::Counter* _partition_shuffle_timer = nullptr; RuntimeProfile::Counter* _spill_build_rows = nullptr; @@ -105,11 +102,6 @@ class PartitionedHashJoinProbeLocalState final RuntimeProfile::Counter* _recovery_probe_blocks = nullptr; RuntimeProfile::Counter* _recovery_probe_timer = nullptr; - RuntimeProfile::Counter* _spill_serialize_block_timer = nullptr; - RuntimeProfile::Counter* _spill_write_disk_timer = nullptr; - RuntimeProfile::Counter* _spill_data_size = nullptr; - RuntimeProfile::Counter* _spill_block_count = nullptr; - RuntimeProfile::Counter* _build_phase_label = nullptr; RuntimeProfile::Counter* _build_rows_counter = nullptr; RuntimeProfile::Counter* _publish_runtime_filter_timer = nullptr; @@ -121,6 +113,9 @@ class PartitionedHashJoinProbeLocalState final RuntimeProfile::Counter* _build_side_compute_hash_timer = nullptr; RuntimeProfile::Counter* _build_side_merge_block_timer = nullptr; + RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; + RuntimeProfile::Counter* _probe_blocks_bytes = nullptr; + RuntimeProfile::Counter* _allocate_resource_timer = nullptr; RuntimeProfile::Counter* _probe_phase_label = nullptr; @@ -136,6 +131,9 @@ class PartitionedHashJoinProbeLocalState final RuntimeProfile::Counter* _probe_rows_counter = nullptr; RuntimeProfile::Counter* _join_filter_timer = nullptr; RuntimeProfile::Counter* _build_output_block_timer = nullptr; + RuntimeProfile::Counter* _memory_usage_reserved = nullptr; + + RuntimeProfile::Counter* _get_child_next_timer = nullptr; }; class PartitionedHashJoinProbeOperatorX final @@ -172,6 +170,8 @@ class PartitionedHashJoinProbeOperatorX final size_t revocable_mem_size(RuntimeState* state) const override; + size_t get_reserve_mem_size(RuntimeState* state) override; + void set_inner_operators(const std::shared_ptr& sink_operator, const std::shared_ptr& probe_operator) { _inner_sink_operator = sink_operator; @@ -184,6 +184,8 @@ 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, diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp index 852dccae71ca3b..8d100dd453039e 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp @@ -17,9 +17,21 @@ #include "partitioned_hash_join_sink_operator.h" +#include + +#include +#include +#include + +#include "common/logging.h" #include "pipeline/exec/operator.h" +#include "pipeline/exec/spill_utils.h" +#include "pipeline/pipeline_task.h" #include "runtime/fragment_mgr.h" #include "util/mem_info.h" +#include "util/pretty_printer.h" +#include "util/runtime_profile.h" +#include "vec/spill/spill_stream.h" #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { @@ -34,12 +46,20 @@ Status PartitionedHashJoinSinkLocalState::init(doris::RuntimeState* state, _shared_state->partitioned_build_blocks.resize(p._partition_count); _shared_state->spilled_streams.resize(p._partition_count); + _rows_in_partitions.assign(p._partition_count, 0); + + _spill_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), + "HashJoinBuildSpillDependency", true); + state->get_task()->add_spill_dependency(_spill_dependency.get()); + _internal_runtime_profile.reset(new RuntimeProfile("internal_profile")); - _partition_timer = ADD_CHILD_TIMER_WITH_LEVEL(profile(), "PartitionTime", "Spill", 1); - _partition_shuffle_timer = - ADD_CHILD_TIMER_WITH_LEVEL(profile(), "PartitionShuffleTime", "Spill", 1); - _spill_build_timer = ADD_CHILD_TIMER_WITH_LEVEL(profile(), "SpillBuildTime", "Spill", 1); + _partition_timer = ADD_TIMER_WITH_LEVEL(profile(), "SpillPartitionTime", 1); + _partition_shuffle_timer = ADD_TIMER_WITH_LEVEL(profile(), "SpillPartitionShuffleTime", 1); + _spill_build_timer = ADD_TIMER_WITH_LEVEL(profile(), "SpillBuildTime", 1); + _in_mem_rows_counter = ADD_COUNTER_WITH_LEVEL(profile(), "SpillInMemRow", TUnit::UNIT, 1); + _memory_usage_reserved = + ADD_COUNTER_WITH_LEVEL(profile(), "MemoryUsageReserved", TUnit::BYTES, 1); return Status::OK(); } @@ -47,6 +67,7 @@ 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(_profile); RETURN_IF_ERROR(PipelineXSpillSinkLocalState::open(state)); auto& p = _parent->cast(); for (uint32_t i = 0; i != p._partition_count; ++i) { @@ -55,10 +76,6 @@ Status PartitionedHashJoinSinkLocalState::open(RuntimeState* state) { 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(), _profile)); - RETURN_IF_ERROR(spilling_stream->prepare_spill()); - spilling_stream->set_write_counters(_spill_serialize_block_timer, _spill_block_count, - _spill_data_size, _spill_write_disk_timer, - _spill_write_wait_io_timer); } return p._partitioner->clone(state, _partitioner); } @@ -77,9 +94,9 @@ size_t PartitionedHashJoinSinkLocalState::revocable_mem_size(RuntimeState* state /// If no need to spill, all rows were sunk into the `_inner_sink_operator` without partitioned. if (!_shared_state->need_to_spill) { if (_shared_state->inner_shared_state) { - auto inner_sink_state_ = _shared_state->inner_runtime_state->get_sink_local_state(); + auto* inner_sink_state_ = _shared_state->inner_runtime_state->get_sink_local_state(); if (inner_sink_state_) { - auto inner_sink_state = + auto* inner_sink_state = assert_cast(inner_sink_state_); return inner_sink_state->_build_blocks_memory_usage->value(); } @@ -100,29 +117,85 @@ size_t PartitionedHashJoinSinkLocalState::revocable_mem_size(RuntimeState* state return mem_size; } -Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block(RuntimeState* state) { +void PartitionedHashJoinSinkLocalState::update_memory_usage() { + if (!_shared_state->need_to_spill) { + 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_); + COUNTER_SET(_memory_used_counter, inner_sink_state->_memory_used_counter->value()); + } + } + return; + } + + int64_t mem_size = 0; + auto& partitioned_blocks = _shared_state->partitioned_build_blocks; + for (auto& block : partitioned_blocks) { + if (block) { + mem_size += block->allocated_bytes(); + } + } + COUNTER_SET(_memory_used_counter, mem_size); +} + +size_t PartitionedHashJoinSinkLocalState::get_reserve_mem_size(RuntimeState* state, bool eos) { + size_t size_to_reserve = 0; auto& p = _parent->cast(); + if (_shared_state->need_to_spill) { + size_to_reserve = p._partition_count * vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM; + } else { + if (_shared_state->inner_runtime_state) { + size_to_reserve = p._inner_sink_operator->get_reserve_mem_size( + _shared_state->inner_runtime_state.get(), eos); + } + } + + COUNTER_SET(_memory_usage_reserved, int64_t(size_to_reserve)); + return size_to_reserve; +} + +Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block( + RuntimeState* state, const std::shared_ptr& spill_context) { + auto& p = _parent->cast(); + HashJoinBuildSinkLocalState* inner_sink_state {nullptr}; + if (auto* tmp_sink_state = _shared_state->inner_runtime_state->get_sink_local_state()) { + inner_sink_state = assert_cast(tmp_sink_state); + } _shared_state->inner_shared_state->hash_table_variants.reset(); + if (inner_sink_state) { + COUNTER_UPDATE(_memory_used_counter, + -(inner_sink_state->_hash_table_memory_usage->value() + + inner_sink_state->_build_arena_memory_usage->value())); + } auto row_desc = p._child->row_desc(); const auto num_slots = row_desc.num_slots(); vectorized::Block build_block; - 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_); + int64_t block_old_mem = 0; + if (inner_sink_state) { build_block = inner_sink_state->_build_side_mutable_block.to_block(); + block_old_mem = build_block.allocated_bytes(); } if (build_block.rows() <= 1) { - LOG(WARNING) << "has no data to revoke, node: " << _parent->node_id() - << ", task: " << state->task_id(); + LOG(WARNING) << fmt::format( + "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(); } if (build_block.columns() > num_slots) { - build_block.erase(num_slots); + vectorized::Block::erase_useless_column(&build_block, num_slots); + COUNTER_UPDATE(_memory_used_counter, build_block.allocated_bytes() - block_old_mem); } auto spill_func = [build_block = std::move(build_block), state, this]() mutable { + Defer defer1 {[&]() { update_memory_usage(); }}; auto& p = _parent->cast(); auto& partitioned_blocks = _shared_state->partitioned_build_blocks; std::vector> partitions_indexes(p._partition_count); @@ -131,21 +204,6 @@ Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block(RuntimeSta std::for_each(partitions_indexes.begin(), partitions_indexes.end(), [](std::vector& indices) { indices.reserve(reserved_size); }); - auto flush_rows = [&state, this](std::unique_ptr& partition_block, - vectorized::SpillStreamSPtr& spilling_stream) { - auto block = partition_block->to_block(); - auto status = spilling_stream->spill_block(state, block, false); - - if (!status.ok()) { - std::unique_lock lock(_spill_lock); - _spill_status = status; - _spill_status_ok = false; - _dependency->set_ready(); - return false; - } - return true; - }; - size_t total_rows = build_block.rows(); size_t offset = 1; while (offset < total_rows) { @@ -156,6 +214,10 @@ Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block(RuntimeSta 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; @@ -181,52 +243,57 @@ Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block(RuntimeSta vectorized::MutableBlock::create_unique(build_block.clone_empty()); } + int64_t old_mem = partition_block->allocated_bytes(); { SCOPED_TIMER(_partition_shuffle_timer); - Status st = partition_block->add_rows(&sub_block, begin, end); - if (!st.ok()) { - std::unique_lock lock(_spill_lock); - _spill_status = st; - _spill_status_ok = false; - _dependency->set_ready(); - return; - } + 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) { - if (!flush_rows(partition_block, spilling_stream)) { - return; - } + auto block = partition_block->to_block(); + RETURN_IF_ERROR(spilling_stream->spill_block(state, block, false)); partition_block = vectorized::MutableBlock::create_unique(build_block.clone_empty()); + COUNTER_UPDATE(_memory_used_counter, -new_mem); + } else { + COUNTER_UPDATE(_memory_used_counter, new_mem - old_mem); } } } - _dependency->set_ready(); + Status status; + if (_child_eos) { + std::for_each(_shared_state->partitioned_build_blocks.begin(), + _shared_state->partitioned_build_blocks.end(), [&](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; }; - auto exception_catch_func = [spill_func, this]() mutable { - auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION(spill_func()); - return Status::OK(); - }(); - - if (!status.ok()) { - std::unique_lock lock(_spill_lock); - _spill_status = status; - _spill_status_ok = false; - _dependency->set_ready(); - } + auto exception_catch_func = [spill_func]() mutable { + auto status = [&]() { RETURN_IF_CATCH_EXCEPTION(return spill_func()); }(); + return status; }; - auto spill_runnable = std::make_shared(state, _shared_state->shared_from_this(), - exception_catch_func); + auto spill_runnable = std::make_shared( + state, spill_context, _spill_dependency, _profile, _shared_state->shared_from_this(), + exception_catch_func); auto* thread_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(); - _dependency->block(); + _spill_dependency->block(); DBUG_EXECUTE_IF( "fault_inject::partitioned_hash_join_sink::revoke_unpartitioned_block_submit_func", { return Status::Error( @@ -236,90 +303,91 @@ Status PartitionedHashJoinSinkLocalState::_revoke_unpartitioned_block(RuntimeSta return thread_pool->submit(std::move(spill_runnable)); } -Status PartitionedHashJoinSinkLocalState::revoke_memory(RuntimeState* state) { - LOG(INFO) << "hash join sink " << _parent->node_id() << " revoke_memory" - << ", eos: " << _child_eos; - DCHECK_EQ(_spilling_streams_count, 0); +Status PartitionedHashJoinSinkLocalState::revoke_memory( + RuntimeState* state, const std::shared_ptr& spill_context) { + SCOPED_TIMER(_spill_total_timer); + 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); + CHECK_EQ(_spill_dependency->is_blocked_by(nullptr), nullptr); if (!_shared_state->need_to_spill) { profile()->add_info_string("Spilled", "true"); _shared_state->need_to_spill = true; - return _revoke_unpartitioned_block(state); + return _revoke_unpartitioned_block(state, spill_context); } - _spilling_streams_count = cast_set(_shared_state->partitioned_build_blocks.size()); - - auto query_id = state->query_id(); - - for (int i = 0; i != _shared_state->partitioned_build_blocks.size(); ++i) { - vectorized::SpillStreamSPtr& spilling_stream = _shared_state->spilled_streams[i]; - auto& mutable_block = _shared_state->partitioned_build_blocks[i]; + const auto query_id = state->query_id(); + auto* spill_io_pool = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(); + DCHECK(spill_io_pool != nullptr); - if (!mutable_block || - mutable_block->allocated_bytes() < vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { - --_spilling_streams_count; - continue; + auto spill_fin_cb = [this, state, query_id, 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::for_each(_shared_state->partitioned_build_blocks.begin(), + _shared_state->partitioned_build_blocks.end(), [&](auto& block) { + if (block) { + COUNTER_UPDATE(_in_mem_rows_counter, block->rows()); + } + }); + status = _finish_spilling(); + _dependency->set_ready_to_read(); } - DCHECK(spilling_stream != nullptr); - - auto* spill_io_pool = - ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool(); - DCHECK(spill_io_pool != nullptr); - - MonotonicStopWatch submit_timer; - submit_timer.start(); + if (spill_context) { + spill_context->on_task_finished(); + } - Status st; - DBUG_EXECUTE_IF("fault_inject::partitioned_hash_join_sink::revoke_memory_submit_func", { - st = Status::Error( - "fault_inject partitioned_hash_join_sink revoke_memory submit_func failed"); - }); + _spill_dependency->set_ready(); + return status; + }; - auto spill_runnable = std::make_shared( - state, _shared_state->shared_from_this(), - [this, query_id, spilling_stream, i, submit_timer] { - DBUG_EXECUTE_IF( - "fault_inject::partitioned_hash_join_sink::revoke_memory_cancel", { - ExecEnv::GetInstance()->fragment_mgr()->cancel_query( - query_id, Status::InternalError( - "fault_inject partitioned_hash_join_sink " - "revoke_memory canceled")); - return; - }); - _spill_wait_in_queue_timer->update(submit_timer.elapsed_time()); - SCOPED_TIMER(_spill_build_timer); + auto spill_runnable = std::make_shared( + state, nullptr, nullptr, _profile, _shared_state->shared_from_this(), + [this, query_id] { + 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"); + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_id, status); + return status; + }); + SCOPED_TIMER(_spill_build_timer); + + for (size_t i = 0; i != _shared_state->partitioned_build_blocks.size(); ++i) { + vectorized::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() < + vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + continue; + } auto status = [&]() { - RETURN_IF_CATCH_EXCEPTION(_spill_to_disk(i, spilling_stream)); - return Status::OK(); + RETURN_IF_CATCH_EXCEPTION( + return _spill_to_disk(static_cast(i), spilling_stream)); }(); - if (!status.ok()) { - std::unique_lock lock(_spill_lock); - _dependency->set_ready(); - _spill_status_ok = false; - _spill_status = std::move(status); - } - }); - if (st.ok()) { - st = spill_io_pool->submit(std::move(spill_runnable)); - } + RETURN_IF_ERROR(status); + } + return Status::OK(); + }, + spill_fin_cb); - if (!st.ok()) { - --_spilling_streams_count; - return st; - } - } + _spill_dependency->block(); + return spill_io_pool->submit(std::move(spill_runnable)); +} - if (_spilling_streams_count > 0) { - std::unique_lock lock(_spill_lock); - if (_spilling_streams_count > 0) { - _dependency->block(); - } else if (_child_eos) { - LOG(INFO) << "hash join sink " << _parent->node_id() << " set_ready_to_read" - << ", task id: " << state->task_id(); - _dependency->set_ready_to_read(); +Status PartitionedHashJoinSinkLocalState::_finish_spilling() { + for (auto& stream : _shared_state->spilled_streams) { + if (stream) { + RETURN_IF_ERROR(stream->spill_eof()); } } return Status::OK(); @@ -332,6 +400,7 @@ Status PartitionedHashJoinSinkLocalState::_partition_block(RuntimeState* state, if (!rows) { return Status::OK(); } + Defer defer {[&]() { update_memory_usage(); }}; { /// TODO: DO NOT execute build exprs twice(when partition and building hash table) SCOPED_TIMER(_partition_timer); @@ -360,37 +429,26 @@ Status PartitionedHashJoinSinkLocalState::_partition_block(RuntimeState* state, } RETURN_IF_ERROR(partitioned_blocks[i]->add_rows(in_block, partition_indexes[i].data(), partition_indexes[i].data() + count)); + _rows_in_partitions[i] += count; } + update_max_min_rows_counter(); + return Status::OK(); } -void PartitionedHashJoinSinkLocalState::_spill_to_disk( +Status PartitionedHashJoinSinkLocalState::_spill_to_disk( uint32_t partition_index, const vectorized::SpillStreamSPtr& spilling_stream) { auto& partitioned_block = _shared_state->partitioned_build_blocks[partition_index]; - if (_spill_status_ok) { + 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 = vectorized::MutableBlock::create_unique(block.clone_empty()); - auto st = spilling_stream->spill_block(state(), block, false); - if (!st.ok()) { - _spill_status_ok = false; - std::lock_guard l(_spill_status_lock); - _spill_status = st; - } - } - - auto num = _spilling_streams_count.fetch_sub(1); - DCHECK_GE(_spilling_streams_count, 0); - - if (num == 1) { - std::unique_lock lock(_spill_lock); - _dependency->set_ready(); - if (_child_eos) { - LOG(INFO) << "hash join sink " << _parent->node_id() << " set_ready_to_read" - << ", task id: " << state()->task_id(); - _dependency->set_ready_to_read(); - } + return spilling_stream->spill_block(state(), block, false); + } else { + return _state->cancel_reason(); } } @@ -408,7 +466,9 @@ PartitionedHashJoinSinkOperatorX::PartitionedHashJoinSinkOperatorX(ObjectPool* p : std::vector {}), _tnode(tnode), _descriptor_tbl(descs), - _partition_count(partition_count) {} + _partition_count(partition_count) { + _spillable = true; +} Status PartitionedHashJoinSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(JoinBuildSinkOperatorX::init(tnode, state)); @@ -477,38 +537,86 @@ Status PartitionedHashJoinSinkOperatorX::_setup_internal_operator(RuntimeState* return Status::OK(); } +// 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, vectorized::Block* in_block, bool eos) { auto& local_state = get_local_state(state); + CHECK_EQ(local_state._spill_dependency->is_blocked_by(nullptr), nullptr); local_state.inc_running_big_mem_op_num(state); SCOPED_TIMER(local_state.exec_time_counter()); - if (!local_state._spill_status_ok) { - DCHECK_NE(local_state._spill_status.code(), 0); - return local_state._spill_status; - } local_state._child_eos = eos; const auto rows = in_block->rows(); const auto need_to_spill = local_state._shared_state->need_to_spill; + 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()->get_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(), need_to_spill, + PrettyPrinter::print_bytes(query_mem_limit), + PrettyPrinter::print_bytes(revocable_size)); + } + if (rows == 0) { if (eos) { - LOG(INFO) << "hash join sink " << node_id() << " sink eos, set_ready_to_read" - << ", task id: " << state->task_id(); - - if (!need_to_spill) { + if (need_to_spill) { + return revoke_memory(state, nullptr); + } else { if (UNLIKELY(!local_state._shared_state->inner_runtime_state)) { RETURN_IF_ERROR(_setup_internal_operator(state)); } + 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)); + + 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::for_each(local_state._shared_state->partitioned_build_blocks.begin(), + local_state._shared_state->partitioned_build_blocks.end(), + [&](auto& block) { + if (block) { + COUNTER_UPDATE(local_state._in_mem_rows_counter, block->rows()); + } + }); local_state._dependency->set_ready_to_read(); } return Status::OK(); @@ -517,10 +625,10 @@ Status PartitionedHashJoinSinkOperatorX::sink(RuntimeState* state, vectorized::B COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); if (need_to_spill) { RETURN_IF_ERROR(local_state._partition_block(state, in_block, 0, rows)); - - const auto revocable_size = revocable_mem_size(state); - if (revocable_size > state->min_revocable_mem()) { - return local_state.revoke_memory(state); + if (eos) { + return revoke_memory(state, nullptr); + } else if (revocable_mem_size(state) > vectorized::SpillStream::MAX_SPILL_WRITE_BATCH_MEM) { + return revoke_memory(state, nullptr); } } else { if (UNLIKELY(!local_state._shared_state->inner_runtime_state)) { @@ -531,14 +639,28 @@ Status PartitionedHashJoinSinkOperatorX::sink(RuntimeState* state, vectorized::B "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); + } + } RETURN_IF_ERROR(_inner_sink_operator->sink( local_state._shared_state->inner_runtime_state.get(), in_block, eos)); - } - - if (eos) { - LOG(INFO) << "hash join sink " << node_id() << " sink eos, set_ready_to_read" - << ", task id: " << state->task_id(); - local_state._dependency->set_ready_to_read(); + local_state.update_memory_usage(); + 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(); + } } return Status::OK(); @@ -550,10 +672,16 @@ size_t PartitionedHashJoinSinkOperatorX::revocable_mem_size(RuntimeState* state) return local_state.revocable_mem_size(state); } -Status PartitionedHashJoinSinkOperatorX::revoke_memory(RuntimeState* state) { +Status PartitionedHashJoinSinkOperatorX::revoke_memory( + RuntimeState* state, const std::shared_ptr& spill_context) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); - return local_state.revoke_memory(state); + return local_state.revoke_memory(state, spill_context); +} + +size_t PartitionedHashJoinSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + return local_state.get_reserve_mem_size(state, eos); } #include "common/compile_check_end.h" diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h index e16e52dcaf9453..e1a76fa17de19f 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.h @@ -19,12 +19,15 @@ #include +#include + #include "common/status.h" #include "operator.h" #include "pipeline/exec/hashjoin_build_sink.h" #include "pipeline/exec/hashjoin_probe_operator.h" #include "pipeline/exec/join_build_sink_operator.h" #include "pipeline/exec/spill_utils.h" +#include "vec/core/block.h" #include "vec/runtime/partitioner.h" namespace doris { @@ -44,32 +47,30 @@ 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); + Status revoke_memory(RuntimeState* state, const std::shared_ptr& spill_context); size_t revocable_mem_size(RuntimeState* state) const; + [[nodiscard]] size_t get_reserve_mem_size(RuntimeState* state, bool eos); + void update_memory_usage(); protected: PartitionedHashJoinSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : PipelineXSpillSinkLocalState(parent, state) {} - void _spill_to_disk(uint32_t partition_index, - const vectorized::SpillStreamSPtr& spilling_stream); + Status _spill_to_disk(uint32_t partition_index, + const vectorized::SpillStreamSPtr& spilling_stream); Status _partition_block(RuntimeState* state, vectorized::Block* in_block, size_t begin, size_t end); - Status _revoke_unpartitioned_block(RuntimeState* state); + Status _revoke_unpartitioned_block(RuntimeState* state, + const std::shared_ptr& spill_context); - friend class PartitionedHashJoinSinkOperatorX; + Status _finish_spilling(); - std::atomic_int _spilling_streams_count {0}; - std::atomic _spill_status_ok {true}; - std::mutex _spill_lock; + friend class PartitionedHashJoinSinkOperatorX; bool _child_eos {false}; - Status _spill_status; - std::mutex _spill_status_lock; - std::unique_ptr _partitioner; std::unique_ptr _internal_runtime_profile; @@ -77,6 +78,8 @@ class PartitionedHashJoinSinkLocalState 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; }; class PartitionedHashJoinSinkOperatorX @@ -100,7 +103,10 @@ class PartitionedHashJoinSinkOperatorX size_t revocable_mem_size(RuntimeState* state) const override; - Status revoke_memory(RuntimeState* state) override; + Status revoke_memory(RuntimeState* state, + const std::shared_ptr& spill_context) override; + + size_t get_reserve_mem_size(RuntimeState* state, bool eos) override; DataDistribution required_data_distribution() const override { if (_join_op == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN) { diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp index 48131e0d96e4c6..c3649a1f8c0201 100644 --- a/be/src/pipeline/exec/repeat_operator.cpp +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -183,6 +183,7 @@ Status RepeatOperatorX::push(RuntimeState* state, vectorized::Block* input_block auto& expr_ctxs = local_state._expr_ctxs; DCHECK(!intermediate_block || intermediate_block->rows() == 0); if (input_block->rows() > 0) { + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); intermediate_block = vectorized::Block::create_unique(); for (auto& expr : expr_ctxs) { @@ -208,6 +209,9 @@ Status RepeatOperatorX::pull(doris::RuntimeState* state, vectorized::Block* outp auto& _child_eos = local_state._child_eos; auto& _intermediate_block = local_state._intermediate_block; RETURN_IF_CANCELLED(state); + + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); + DCHECK(_repeat_id_idx >= 0); for (const std::vector& v : _grouping_list) { DCHECK(_repeat_id_idx <= (int)v.size()); @@ -241,11 +245,13 @@ Status RepeatOperatorX::pull(doris::RuntimeState* state, vectorized::Block* outp _child_block.clear_column_data(_child->row_desc().num_materialized_slots()); } } + { SCOPED_TIMER(local_state._filter_timer); RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, output_block->columns())); } + *eos = _child_eos && _child_block.rows() == 0; local_state.reached_limit(output_block, eos); return Status::OK(); diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index a7802f33e249db..58f9c9b755cea3 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -1288,6 +1288,35 @@ Status ScanOperatorX::get_block(RuntimeState* state, vectorized: return Status::OK(); } +template +size_t ScanOperatorX::get_reserve_mem_size(RuntimeState* state) { + auto& local_state = get_local_state(state); + if (!local_state._opened || local_state._closed || !local_state._scanner_ctx) { + return config::doris_scanner_row_bytes; + } + + if (local_state.low_memory_mode()) { + return local_state._scanner_ctx->low_memory_mode_scan_bytes_per_scanner() * + local_state._scanner_ctx->low_memory_mode_scanners(); + } else { + const auto peak_usage = local_state._memory_used_counter->value(); + const auto block_usage = local_state._scanner_ctx->block_memory_usage(); + if (peak_usage > 0) { + // It is only a safty check, to avoid some counter not right. + if (peak_usage > block_usage) { + return peak_usage - block_usage; + } else { + return config::doris_scanner_row_bytes; + } + } else { + // If the scan operator is first time to run, then we think it will occupy doris_scanner_row_bytes. + // It maybe a little smaller than actual usage. + return config::doris_scanner_row_bytes; + // return local_state._scanner_ctx->max_bytes_in_queue(); + } + } +} + template class ScanOperatorX; template class ScanLocalState; template class ScanOperatorX; diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h index c6c9cdf405d5a4..d099ccdd12f28f 100644 --- a/be/src/pipeline/exec/scan_operator.h +++ b/be/src/pipeline/exec/scan_operator.h @@ -372,6 +372,8 @@ class ScanOperatorX : public OperatorX { [[nodiscard]] virtual bool is_file_scan_operator() const { return false; } + [[nodiscard]] size_t get_reserve_mem_size(RuntimeState* state) override; + const std::vector& runtime_filter_descs() override { return _runtime_filter_descs; } diff --git a/be/src/pipeline/exec/schema_scan_operator.cpp b/be/src/pipeline/exec/schema_scan_operator.cpp index 2e2f80f5e24838..15b72693e5a73f 100644 --- a/be/src/pipeline/exec/schema_scan_operator.cpp +++ b/be/src/pipeline/exec/schema_scan_operator.cpp @@ -260,8 +260,8 @@ Status SchemaScanOperatorX::get_block(RuntimeState* state, vectorized::Block* bl *src_block.get_by_name(dest_slot_desc->col_name()).column, 0, src_block.rows()); } - RETURN_IF_ERROR(vectorized::VExprContext::filter_block( - local_state._conjuncts, block, _dest_tuple_desc->slots().size())); + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, block, + _dest_tuple_desc->slots().size())); src_block.clear(); } } while (block->rows() == 0 && !*eos); diff --git a/be/src/pipeline/exec/select_operator.h b/be/src/pipeline/exec/select_operator.h index 584a6f74308903..f033c7c0de81a8 100644 --- a/be/src/pipeline/exec/select_operator.h +++ b/be/src/pipeline/exec/select_operator.h @@ -47,8 +47,7 @@ class SelectOperatorX final : public StreamingOperatorX { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); RETURN_IF_CANCELLED(state); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, block, block->columns())); local_state.reached_limit(block, eos); return Status::OK(); } diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index db487b0f9e7252..cbd5b9b44b2c9c 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -69,6 +69,7 @@ Status SetProbeSinkOperatorX::sink(RuntimeState* state, vectorized 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()); + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); uint32_t probe_rows = cast_set(in_block->rows()); if (probe_rows > 0) { @@ -202,6 +203,12 @@ void SetProbeSinkOperatorX::_finalize_probe( } } +template +size_t SetProbeSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + return local_state._estimate_memory_usage; +} + template void SetProbeSinkOperatorX::_refresh_hash_table( SetProbeSinkLocalState& local_state) { diff --git a/be/src/pipeline/exec/set_probe_sink_operator.h b/be/src/pipeline/exec/set_probe_sink_operator.h index 6b764c1e509951..45c0cbce430eb0 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.h +++ b/be/src/pipeline/exec/set_probe_sink_operator.h @@ -56,6 +56,8 @@ class SetProbeSinkLocalState final : public PipelineXSinkLocalState friend struct vectorized::HashTableProbe; + int64_t _estimate_memory_usage = 0; + //record insert column id during probe std::vector _probe_column_inserted_id; vectorized::ColumnRawPtrs _probe_columns; @@ -102,6 +104,8 @@ class SetProbeSinkOperatorX final : public DataSinkOperatorX create_shared_state() const override { return nullptr; } + size_t get_reserve_mem_size(RuntimeState* state, bool eos) override; + private: void _finalize_probe(SetProbeSinkLocalState& local_state); Status _extract_probe_column(SetProbeSinkLocalState& local_state, diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index 539134e53e7fe2..82bf523c60a26c 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -211,6 +211,27 @@ Status SetSinkOperatorX::init(const TPlanNode& tnode, RuntimeState return Status::OK(); } +template +size_t SetSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + size_t size_to_reserve = std::visit( + [&](auto&& arg) -> size_t { + using HashTableCtxType = std::decay_t; + if constexpr (std::is_same_v) { + return 0; + } else { + return arg.hash_table->estimate_memory(state->batch_size()); + } + }, + local_state._shared_state->hash_table_variants->method_variant); + + size_to_reserve += local_state._mutable_block.allocated_bytes(); + for (auto& _child_expr : _child_exprs) { + size_to_reserve += _child_expr->root()->estimate_memory(state->batch_size()); + } + return size_to_reserve; +} + template Status SetSinkOperatorX::open(RuntimeState* state) { RETURN_IF_ERROR(Base::open(state)); diff --git a/be/src/pipeline/exec/set_sink_operator.h b/be/src/pipeline/exec/set_sink_operator.h index ba387d97b41360..5a18ef79643513 100644 --- a/be/src/pipeline/exec/set_sink_operator.h +++ b/be/src/pipeline/exec/set_sink_operator.h @@ -95,6 +95,8 @@ class SetSinkOperatorX final : public DataSinkOperatorX friend struct HashTableBuild; diff --git a/be/src/pipeline/exec/set_source_operator.cpp b/be/src/pipeline/exec/set_source_operator.cpp index b31ec157bb877e..bfe67b82d8b0b2 100644 --- a/be/src/pipeline/exec/set_source_operator.cpp +++ b/be/src/pipeline/exec/set_source_operator.cpp @@ -77,6 +77,8 @@ Status SetSourceOperatorX::get_block(RuntimeState* state, vectoriz RETURN_IF_CANCELLED(state); auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); + _create_mutable_cols(local_state, block); { SCOPED_TIMER(local_state._get_data_timer); diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index 6bec42ac62d192..f7b7510e24c4a2 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -156,6 +156,11 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in return Status::OK(); } +size_t SortSinkOperatorX::get_reserve_mem_size_for_next_sink(RuntimeState* state, bool eos) { + auto& local_state = get_local_state(state); + return local_state._shared_state->sorter->get_reserve_mem_size(state, eos); +} + size_t SortSinkOperatorX::get_revocable_mem_size(RuntimeState* state) const { auto& local_state = get_local_state(state); return local_state._shared_state->sorter->data_size(); diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index 766c6c0ffc9a59..534f2a806d9eaa 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -37,6 +37,8 @@ class SortSinkLocalState : public PipelineXSinkLocalState { Status init(RuntimeState* state, LocalSinkStateInfo& info) override; Status open(RuntimeState* state) override; + [[nodiscard]] size_t get_reserve_mem_size(RuntimeState* state, bool eos); + private: friend class SortSinkOperatorX; @@ -80,6 +82,8 @@ class SortSinkOperatorX final : public DataSinkOperatorX { size_t get_revocable_mem_size(RuntimeState* state) const; + size_t get_reserve_mem_size_for_next_sink(RuntimeState* state, bool eos); + Status prepare_for_spill(RuntimeState* state); Status merge_sort_read_for_spill(RuntimeState* state, doris::vectorized::Block* block, diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp index 2fb09d7278fda8..dc422c64dc6fff 100644 --- a/be/src/pipeline/exec/sort_source_operator.cpp +++ b/be/src/pipeline/exec/sort_source_operator.cpp @@ -56,6 +56,8 @@ Status SortSourceOperatorX::open(RuntimeState* state) { Status SortSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); + RETURN_IF_ERROR(local_state._shared_state->sorter->get_next(state, block, eos)); local_state.reached_limit(block, eos); return Status::OK(); diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.cpp b/be/src/pipeline/exec/spill_sort_sink_operator.cpp index 6071301c1d7bcc..943d9a7c8d20fa 100644 --- a/be/src/pipeline/exec/spill_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/spill_sort_sink_operator.cpp @@ -19,16 +19,14 @@ #include "pipeline/exec/sort_sink_operator.h" #include "pipeline/exec/spill_utils.h" +#include "pipeline/pipeline_task.h" #include "runtime/fragment_mgr.h" #include "vec/spill/spill_stream_manager.h" namespace doris::pipeline { SpillSortSinkLocalState::SpillSortSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) - : Base(parent, state) { - _finish_dependency = std::make_shared(parent->operator_id(), parent->node_id(), - parent->get_name() + "_SPILL_DEPENDENCY"); -} + : Base(parent, state) {} Status SpillSortSinkLocalState::init(doris::RuntimeState* state, doris::pipeline::LocalSinkStateInfo& info) { @@ -38,13 +36,23 @@ Status SpillSortSinkLocalState::init(doris::RuntimeState* state, _init_counters(); + _spill_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), + "SortSinkSpillDependency", true); + state->get_task()->add_spill_dependency(_spill_dependency.get()); + RETURN_IF_ERROR(setup_in_memory_sort_op(state)); Base::_shared_state->in_mem_shared_state->sorter->set_enable_spill(); - _finish_dependency->block(); return Status::OK(); } +Status SpillSortSinkLocalState::open(RuntimeState* state) { + SCOPED_TIMER(Base::exec_time_counter()); + SCOPED_TIMER(Base::_open_timer); + _shared_state->setup_shared_profile(_profile); + return Base::open(state); +} + void SpillSortSinkLocalState::_init_counters() { _internal_runtime_profile = std::make_unique("internal_profile"); @@ -52,12 +60,7 @@ void SpillSortSinkLocalState::_init_counters() { _merge_block_timer = ADD_TIMER(_profile, "MergeBlockTime"); _sort_blocks_memory_usage = ADD_COUNTER_WITH_LEVEL(_profile, "MemoryUsageSortBlocks", TUnit::BYTES, 1); - - _spill_merge_sort_timer = - ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillMergeSortTime", "Spill", 1); - - _spill_wait_in_queue_timer = - ADD_CHILD_TIMER_WITH_LEVEL(profile(), "SpillWaitInQueueTime", "Spill", 1); + _spill_merge_sort_timer = ADD_TIMER_WITH_LEVEL(_profile, "SpillMergeSortTime", 1); } #define UPDATE_PROFILE(counter, name) \ do { \ @@ -75,7 +78,7 @@ void SpillSortSinkLocalState::update_profile(RuntimeProfile* child_profile) { Status SpillSortSinkLocalState::close(RuntimeState* state, Status execsink_status) { dec_running_big_mem_op_num(state); - return Status::OK(); + return Base::close(state, execsink_status); } Status SpillSortSinkLocalState::setup_in_memory_sort_op(RuntimeState* state) { @@ -111,6 +114,7 @@ SpillSortSinkOperatorX::SpillSortSinkOperatorX(ObjectPool* pool, int operator_id const TPlanNode& tnode, const DescriptorTbl& descs, bool require_bucket_distribution) : DataSinkOperatorX(operator_id, tnode.node_id) { + _spillable = true; _sort_sink_operator = std::make_unique(pool, operator_id, tnode, descs, require_bucket_distribution); } @@ -129,16 +133,18 @@ Status SpillSortSinkOperatorX::open(RuntimeState* state) { return _sort_sink_operator->open(state); } -Status SpillSortSinkOperatorX::revoke_memory(RuntimeState* state) { +size_t SpillSortSinkOperatorX::get_reserve_mem_size(RuntimeState* state, bool eos) { + 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) { auto& local_state = get_local_state(state); - return local_state.revoke_memory(state); + return local_state.revoke_memory(state, spill_context); } size_t SpillSortSinkOperatorX::revocable_mem_size(RuntimeState* state) const { auto& local_state = get_local_state(state); - if (!local_state.Base::_shared_state->sink_status.ok()) { - return UINT64_MAX; - } return _sort_sink_operator->get_revocable_mem_size(local_state._runtime_state.get()); } @@ -147,10 +153,9 @@ Status SpillSortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Bloc auto& local_state = get_local_state(state); local_state.inc_running_big_mem_op_num(state); SCOPED_TIMER(local_state.exec_time_counter()); - RETURN_IF_ERROR(local_state.Base::_shared_state->sink_status); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); if (in_block->rows() > 0) { - local_state._shared_state->update_spill_block_batch_row_count(in_block); + local_state._shared_state->update_spill_block_batch_row_count(state, in_block); } local_state._eos = eos; DBUG_EXECUTE_IF("fault_inject::spill_sort_sink::sink", @@ -164,88 +169,79 @@ Status SpillSortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Bloc if (eos) { if (local_state._shared_state->is_spilled) { if (revocable_mem_size(state) > 0) { - RETURN_IF_ERROR(revoke_memory(state)); + RETURN_IF_ERROR(revoke_memory(state, nullptr)); } else { local_state._dependency->set_ready_to_read(); - local_state._finish_dependency->set_ready(); } } else { RETURN_IF_ERROR( local_state._shared_state->in_mem_shared_state->sorter->prepare_for_read()); local_state._dependency->set_ready_to_read(); - local_state._finish_dependency->set_ready(); } } return Status::OK(); } -Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state) { +size_t SpillSortSinkLocalState::get_reserve_mem_size(RuntimeState* state, bool eos) { + auto& parent = Base::_parent->template cast(); + return parent._sort_sink_operator->get_reserve_mem_size_for_next_sink(_runtime_state.get(), + eos); +} + +Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state, + const std::shared_ptr& spill_context) { if (!_shared_state->is_spilled) { _shared_state->is_spilled = true; profile()->add_info_string("Spilled", "true"); } - VLOG_DEBUG << "query " << print_id(state->query_id()) << " sort node " - << Base::_parent->node_id() << " revoke_memory" - << ", eos: " << _eos; - RETURN_IF_ERROR(Base::_shared_state->sink_status); + VLOG_DEBUG << fmt::format("Query:{}, sort sink:{}, task:{}, revoke_memory, eos:{}", + print_id(state->query_id()), _parent->node_id(), state->task_id(), + _eos); auto status = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( state, _spilling_stream, print_id(state->query_id()), "sort", _parent->node_id(), - _shared_state->spill_block_batch_row_count, - SpillSortSharedState::SORT_BLOCK_SPILL_BATCH_BYTES, profile()); + _shared_state->spill_block_batch_row_count, state->spill_sort_batch_bytes(), profile()); RETURN_IF_ERROR(status); - _spilling_stream->set_write_counters( - Base::_spill_serialize_block_timer, Base::_spill_block_count, Base::_spill_data_size, - Base::_spill_write_disk_timer, Base::_spill_write_wait_io_timer); - - status = _spilling_stream->prepare_spill(); - RETURN_IF_ERROR(status); _shared_state->sorted_streams.emplace_back(_spilling_stream); auto& parent = Base::_parent->template cast(); // TODO: spill thread may set_ready before the task::execute thread put the task to blocked state if (!_eos) { - Base::_dependency->Dependency::block(); + Base::_spill_dependency->Dependency::block(); } auto query_id = state->query_id(); - MonotonicStopWatch submit_timer; - submit_timer.start(); - - auto spill_func = [this, state, query_id, &parent, submit_timer] { - _spill_wait_in_queue_timer->update(submit_timer.elapsed_time()); + auto spill_func = [this, state, query_id, &parent] { + Status status; Defer defer {[&]() { - if (!_shared_state->sink_status.ok() || state->is_cancelled()) { - if (!_shared_state->sink_status.ok()) { - LOG(WARNING) << "query " << print_id(query_id) << " sort node " - << _parent->node_id() - << " revoke memory error: " << _shared_state->sink_status; + 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 << "query " << print_id(query_id) << " sort node " << _parent->node_id() - << " revoke memory finish"; + VLOG_DEBUG << fmt::format("Query:{}, sort sink:{}, task:{}, revoke memory finish", + print_id(query_id), _parent->node_id(), state->task_id()); } - if (!_shared_state->sink_status.ok()) { + if (!status.ok()) { _shared_state->close(); } _spilling_stream.reset(); + state->get_query_ctx()->decrease_revoking_tasks_count(); if (_eos) { _dependency->set_ready_to_read(); - _finish_dependency->set_ready(); - } else { - _dependency->Dependency::set_ready(); } }}; - _shared_state->sink_status = - parent._sort_sink_operator->prepare_for_spill(_runtime_state.get()); - RETURN_IF_ERROR(_shared_state->sink_status); + status = parent._sort_sink_operator->prepare_for_spill(_runtime_state.get()); + RETURN_IF_ERROR(status); auto* sink_local_state = _runtime_state->get_sink_local_state(); update_profile(sink_local_state->profile()); @@ -255,16 +251,13 @@ Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state) { while (!eos && !state->is_cancelled()) { { SCOPED_TIMER(_spill_merge_sort_timer); - _shared_state->sink_status = parent._sort_sink_operator->merge_sort_read_for_spill( + status = parent._sort_sink_operator->merge_sort_read_for_spill( _runtime_state.get(), &block, _shared_state->spill_block_batch_row_count, &eos); } - RETURN_IF_ERROR(_shared_state->sink_status); - { - SCOPED_TIMER(Base::_spill_timer); - _shared_state->sink_status = _spilling_stream->spill_block(state, block, eos); - } - RETURN_IF_ERROR(_shared_state->sink_status); + RETURN_IF_ERROR(status); + status = _spilling_stream->spill_block(state, block, eos); + RETURN_IF_ERROR(status); block.clear_column_data(); } parent._sort_sink_operator->reset(_runtime_state.get()); @@ -272,17 +265,18 @@ Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state) { return Status::OK(); }; - auto exception_catch_func = [this, query_id, spill_func]() { + auto exception_catch_func = [query_id, spill_func]() { DBUG_EXECUTE_IF("fault_inject::spill_sort_sink::revoke_memory_cancel", { - ExecEnv::GetInstance()->fragment_mgr()->cancel_query( - query_id, Status::InternalError("fault_inject spill_sort_sink " - "revoke_memory canceled")); - return; + auto status = Status::InternalError( + "fault_inject spill_sort_sink " + "revoke_memory canceled"); + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_id, status); + return status; }); - _shared_state->sink_status = [&]() { - RETURN_IF_CATCH_EXCEPTION({ return spill_func(); }); - }(); + auto status = [&]() { RETURN_IF_CATCH_EXCEPTION({ return spill_func(); }); }(); + + return status; }; DBUG_EXECUTE_IF("fault_inject::spill_sort_sink::revoke_memory_submit_func", { @@ -291,15 +285,19 @@ Status SpillSortSinkLocalState::revoke_memory(RuntimeState* state) { "revoke_memory submit_func failed"); }); if (status.ok()) { + state->get_query_ctx()->increase_revoking_tasks_count(); + status = ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool()->submit( - std::make_shared(state, _shared_state->shared_from_this(), - exception_catch_func)); + std::make_shared(state, spill_context, _spill_dependency, + _profile, _shared_state->shared_from_this(), + exception_catch_func)); } if (!status.ok()) { if (!_eos) { - Base::_dependency->Dependency::set_ready(); + Base::_spill_dependency->Dependency::set_ready(); } } return status; } + } // namespace doris::pipeline \ No newline at end of file diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.h b/be/src/pipeline/exec/spill_sort_sink_operator.h index 2c820d9fa09daf..3d6ccdcc4ce359 100644 --- a/be/src/pipeline/exec/spill_sort_sink_operator.h +++ b/be/src/pipeline/exec/spill_sort_sink_operator.h @@ -17,6 +17,8 @@ #pragma once +#include + #include "operator.h" #include "sort_sink_operator.h" @@ -34,11 +36,12 @@ class SpillSortSinkLocalState : public PipelineXSpillSinkLocalState& spill_context); private: void _init_counters(); @@ -55,7 +58,6 @@ class SpillSortSinkLocalState : public PipelineXSpillSinkLocalState _finish_dependency; }; class SpillSortSinkOperatorX final : public DataSinkOperatorX { @@ -83,9 +85,12 @@ class SpillSortSinkOperatorX final : public DataSinkOperatorXset_child(child); } + size_t get_reserve_mem_size(RuntimeState* state, bool eos) override; + size_t revocable_mem_size(RuntimeState* state) const override; - Status revoke_memory(RuntimeState* state) override; + Status revoke_memory(RuntimeState* state, + const std::shared_ptr& spill_context) override; using DataSinkOperatorX::node_id; using DataSinkOperatorX::operator_id; diff --git a/be/src/pipeline/exec/spill_sort_source_operator.cpp b/be/src/pipeline/exec/spill_sort_source_operator.cpp index 69ed816fa9142d..d344b22e08bbb4 100644 --- a/be/src/pipeline/exec/spill_sort_source_operator.cpp +++ b/be/src/pipeline/exec/spill_sort_source_operator.cpp @@ -17,8 +17,11 @@ #include "spill_sort_source_operator.h" +#include + #include "common/status.h" #include "pipeline/exec/spill_utils.h" +#include "pipeline/pipeline_task.h" #include "runtime/fragment_mgr.h" #include "sort_source_operator.h" #include "util/runtime_profile.h" @@ -26,29 +29,20 @@ namespace doris::pipeline { SpillSortLocalState::SpillSortLocalState(RuntimeState* state, OperatorXBase* parent) - : Base(state, parent) { - if (state->external_sort_bytes_threshold() > 0) { - _external_sort_bytes_threshold = state->external_sort_bytes_threshold(); - } -} + : Base(state, parent) {} + Status SpillSortLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); + init_spill_write_counters(); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + + _spill_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), + "SortSourceSpillDependency", true); + state->get_task()->add_spill_dependency(_spill_dependency.get()); + _internal_runtime_profile = std::make_unique("internal_profile"); - _spill_timer = ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillMergeSortTime", "Spill", 1); - _spill_merge_sort_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillMergeSortTime", "Spill", 1); - _spill_serialize_block_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillSerializeBlockTime", "Spill", 1); - _spill_write_disk_timer = - ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillWriteDiskTime", "Spill", 1); - _spill_data_size = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteDataSize", - TUnit::BYTES, "Spill", 1); - _spill_block_count = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "SpillWriteBlockCount", - TUnit::UNIT, "Spill", 1); - _spill_wait_in_queue_timer = - ADD_CHILD_TIMER_WITH_LEVEL(profile(), "SpillWaitInQueueTime", "Spill", 1); + _spill_merge_sort_timer = ADD_TIMER_WITH_LEVEL(Base::profile(), "SpillMergeSortTime", 1); return Status::OK(); } @@ -58,6 +52,7 @@ Status SpillSortLocalState::open(RuntimeState* state) { if (_opened) { return Status::OK(); } + RETURN_IF_ERROR(setup_in_memory_sort_op(state)); return Base::open(state); } @@ -69,29 +64,27 @@ Status SpillSortLocalState::close(RuntimeState* state) { dec_running_big_mem_op_num(state); return Base::close(state); } -int SpillSortLocalState::_calc_spill_blocks_to_merge() const { - int count = _external_sort_bytes_threshold / SpillSortSharedState::SORT_BLOCK_SPILL_BATCH_BYTES; +int SpillSortLocalState::_calc_spill_blocks_to_merge(RuntimeState* state) const { + int count = state->spill_sort_mem_limit() / state->spill_sort_batch_bytes(); return std::max(2, count); } Status SpillSortLocalState::initiate_merge_sort_spill_streams(RuntimeState* state) { auto& parent = Base::_parent->template cast(); - VLOG_DEBUG << "query " << print_id(state->query_id()) << " sort node " << _parent->node_id() - << " merge spill data"; - _dependency->Dependency::block(); + VLOG_DEBUG << fmt::format("Query:{}, sort source:{}, task:{}, merge spill data", + print_id(state->query_id()), _parent->node_id(), state->task_id()); + _spill_dependency->Dependency::block(); auto query_id = state->query_id(); - MonotonicStopWatch submit_timer; - submit_timer.start(); - - auto spill_func = [this, state, query_id, &parent, submit_timer] { - _spill_wait_in_queue_timer->update(submit_timer.elapsed_time()); + auto spill_func = [this, state, query_id, &parent] { SCOPED_TIMER(_spill_merge_sort_timer); + Status status; Defer defer {[&]() { - if (!_status.ok() || state->is_cancelled()) { - if (!_status.ok()) { - LOG(WARNING) << "query " << print_id(query_id) << " sort node " - << _parent->node_id() << " merge spill data error: " << _status; + 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); } _shared_state->close(); for (auto& stream : _current_merging_streams) { @@ -99,26 +92,27 @@ Status SpillSortLocalState::initiate_merge_sort_spill_streams(RuntimeState* stat } _current_merging_streams.clear(); } else { - VLOG_DEBUG << "query " << print_id(query_id) << " sort node " << _parent->node_id() - << " merge spill data finish"; + VLOG_DEBUG << fmt::format( + "Query:{}, sort source:{}, task:{}, merge spill data finish", + print_id(query_id), _parent->node_id(), state->task_id()); } - _dependency->Dependency::set_ready(); }}; vectorized::Block merge_sorted_block; vectorized::SpillStreamSPtr tmp_stream; while (!state->is_cancelled()) { - int max_stream_count = _calc_spill_blocks_to_merge(); - VLOG_DEBUG << "query " << print_id(query_id) << " sort node " << _parent->node_id() - << " merge spill streams, streams count: " - << _shared_state->sorted_streams.size() - << ", curren merge max stream count: " << max_stream_count; + 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:{}", + 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( + status = _create_intermediate_merger( max_stream_count, parent._sort_source_operator->get_sort_description(_runtime_state.get())); } - RETURN_IF_ERROR(_status); + RETURN_IF_ERROR(status); // all the remaining streams can be merged in a run if (_shared_state->sorted_streams.empty()) { @@ -126,43 +120,38 @@ Status SpillSortLocalState::initiate_merge_sort_spill_streams(RuntimeState* stat } { - _status = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( + status = ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( state, tmp_stream, print_id(state->query_id()), "sort", _parent->node_id(), - _shared_state->spill_block_batch_row_count, - SpillSortSharedState::SORT_BLOCK_SPILL_BATCH_BYTES, profile()); - RETURN_IF_ERROR(_status); - _status = tmp_stream->prepare_spill(); - RETURN_IF_ERROR(_status); + _shared_state->spill_block_batch_row_count, state->spill_sort_batch_bytes(), + profile()); + RETURN_IF_ERROR(status); _shared_state->sorted_streams.emplace_back(tmp_stream); bool eos = false; - tmp_stream->set_write_counters(_spill_serialize_block_timer, _spill_block_count, - _spill_data_size, _spill_write_disk_timer, - _spill_write_wait_io_timer); 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( + status = Status::Error( "fault_inject spill_sort_source " "recover_spill_data failed"); }); - if (_status.ok()) { - _status = _merger->get_next(&merge_sorted_block, &eos); + 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); - if (_status.ok()) { + RETURN_IF_ERROR(status); + status = tmp_stream->spill_block(state, merge_sorted_block, eos); + if (status.ok()) { DBUG_EXECUTE_IF("fault_inject::spill_sort_source::spill_merged_data", { - _status = Status::Error( + status = Status::Error( "fault_inject spill_sort_source " "spill_merged_data failed"); }); } - RETURN_IF_ERROR(_status); + RETURN_IF_ERROR(status); } } for (auto& stream : _current_merging_streams) { @@ -173,8 +162,9 @@ Status SpillSortLocalState::initiate_merge_sort_spill_streams(RuntimeState* stat return Status::OK(); }; - auto exception_catch_func = [this, spill_func]() { - _status = [&]() { RETURN_IF_CATCH_EXCEPTION({ return spill_func(); }); }(); + auto exception_catch_func = [spill_func]() { + auto status = [&]() { RETURN_IF_CATCH_EXCEPTION({ return spill_func(); }); }(); + return status; }; DBUG_EXECUTE_IF("fault_inject::spill_sort_source::merge_sort_spill_data_submit_func", { @@ -182,9 +172,11 @@ Status SpillSortLocalState::initiate_merge_sort_spill_streams(RuntimeState* stat "fault_inject spill_sort_source " "merge_sort_spill_data submit_func failed"); }); + return ExecEnv::GetInstance()->spill_stream_mgr()->get_spill_io_thread_pool()->submit( - std::make_shared(state, _shared_state->shared_from_this(), - exception_catch_func)); + std::make_shared(state, _spill_dependency, _runtime_profile.get(), + _shared_state->shared_from_this(), + exception_catch_func)); } Status SpillSortLocalState::_create_intermediate_merger( @@ -198,8 +190,7 @@ Status SpillSortLocalState::_create_intermediate_merger( _current_merging_streams.clear(); for (int i = 0; i < num_blocks && !_shared_state->sorted_streams.empty(); ++i) { auto stream = _shared_state->sorted_streams.front(); - stream->set_read_counters(Base::_spill_read_data_time, Base::_spill_deserialize_time, - Base::_spill_read_bytes, Base::_spill_read_wait_io_timer); + stream->set_read_counters(profile()); _current_merging_streams.emplace_back(stream); child_block_suppliers.emplace_back( std::bind(std::mem_fn(&vectorized::SpillStream::read_next_block_sync), stream.get(), @@ -258,8 +249,10 @@ Status SpillSortSourceOperatorX::close(RuntimeState* state) { Status SpillSortSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { auto& local_state = get_local_state(state); + local_state.copy_shared_spill_profile(); + Status status; Defer defer {[&]() { - if (!local_state._status.ok() || *eos) { + if (!status.ok() || *eos) { local_state._shared_state->close(); for (auto& stream : local_state._current_merging_streams) { (void)ExecEnv::GetInstance()->spill_stream_mgr()->delete_spill_stream(stream); @@ -269,20 +262,19 @@ Status SpillSortSourceOperatorX::get_block(RuntimeState* state, vectorized::Bloc }}; local_state.inc_running_big_mem_op_num(state); SCOPED_TIMER(local_state.exec_time_counter()); - RETURN_IF_ERROR(local_state._status); if (local_state._shared_state->is_spilled) { if (!local_state._merger) { - local_state._status = local_state.initiate_merge_sort_spill_streams(state); - return local_state._status; + status = local_state.initiate_merge_sort_spill_streams(state); + return status; } else { - local_state._status = local_state._merger->get_next(block, eos); - RETURN_IF_ERROR(local_state._status); + SCOPED_TIMER(local_state._spill_total_timer); + status = local_state._merger->get_next(block, eos); + RETURN_IF_ERROR(status); } } else { - local_state._status = - _sort_source_operator->get_block(local_state._runtime_state.get(), block, eos); - RETURN_IF_ERROR(local_state._status); + status = _sort_source_operator->get_block(local_state._runtime_state.get(), block, eos); + RETURN_IF_ERROR(status); } local_state.reached_limit(block, eos); return Status::OK(); diff --git a/be/src/pipeline/exec/spill_sort_source_operator.h b/be/src/pipeline/exec/spill_sort_source_operator.h index 66d05e739d8c02..fae64e051f40fc 100644 --- a/be/src/pipeline/exec/spill_sort_source_operator.h +++ b/be/src/pipeline/exec/spill_sort_source_operator.h @@ -47,27 +47,22 @@ class SpillSortLocalState final : public PipelineXSpillLocalState _runtime_state; bool _opened = false; - Status _status; - int64_t _external_sort_bytes_threshold = 134217728; // 128M std::vector _current_merging_streams; std::unique_ptr _merger; + std::shared_ptr _spill_dependency; + std::unique_ptr _internal_runtime_profile; // counters for spill merge sort - RuntimeProfile::Counter* _spill_timer = nullptr; RuntimeProfile::Counter* _spill_merge_sort_timer = nullptr; - RuntimeProfile::Counter* _spill_serialize_block_timer = nullptr; - RuntimeProfile::Counter* _spill_write_disk_timer = nullptr; - RuntimeProfile::Counter* _spill_data_size = nullptr; - RuntimeProfile::Counter* _spill_block_count = nullptr; }; class SortSourceOperatorX; class SpillSortSourceOperatorX : public OperatorX { diff --git a/be/src/pipeline/exec/spill_utils.h b/be/src/pipeline/exec/spill_utils.h index 2ba6f22a60b10c..a3c51faca1f695 100644 --- a/be/src/pipeline/exec/spill_utils.h +++ b/be/src/pipeline/exec/spill_utils.h @@ -17,11 +17,20 @@ #pragma once +#include +#include + +#include +#include +#include + +#include "runtime/fragment_mgr.h" #include "runtime/memory/mem_tracker_limiter.h" #include "runtime/query_context.h" #include "runtime/runtime_state.h" #include "runtime/task_execution_context.h" #include "runtime/thread_context.h" +#include "util/runtime_profile.h" #include "util/threadpool.h" #include "vec/runtime/partitioner.h" @@ -29,18 +38,69 @@ namespace doris::pipeline { #include "common/compile_check_begin.h" using SpillPartitionerType = vectorized::Crc32HashPartitioner; +struct SpillContext { + std::atomic_int running_tasks_count; + TUniqueId query_id; + std::function all_tasks_finished_callback; + + SpillContext(int running_tasks_count_, TUniqueId query_id_, + std::function all_tasks_finished_callback_) + : running_tasks_count(running_tasks_count_), + query_id(std::move(query_id_)), + all_tasks_finished_callback(std::move(all_tasks_finished_callback_)) {} + + ~SpillContext() { + if (running_tasks_count.load() != 0) { + LOG(WARNING) << "Query: " << print_id(query_id) + << " not all spill tasks finished, remaining tasks: " + << running_tasks_count.load(); + } + } + + void on_task_finished() { + auto count = running_tasks_count.fetch_sub(1); + if (count == 1) { + all_tasks_finished_callback(this); + } + } +}; + class SpillRunnable : public Runnable { -public: - SpillRunnable(RuntimeState* state, const std::shared_ptr& shared_state, - std::function func) +protected: + SpillRunnable(RuntimeState* state, std::shared_ptr spill_context, + std::shared_ptr spill_dependency, RuntimeProfile* profile, + const std::shared_ptr& shared_state, bool is_write, + std::function spill_exec_func, + std::function spill_fin_cb = {}) : _state(state), + _profile(profile), + _spill_context(std::move(spill_context)), + _spill_dependency(std::move(spill_dependency)), + _is_write_task(is_write), _task_context_holder(state->get_task_execution_context()), _shared_state_holder(shared_state), - _func(std::move(func)) {} + _spill_exec_func(std::move(spill_exec_func)), + _spill_fin_cb(std::move(spill_fin_cb)) { + _exec_timer = profile->get_counter("ExecTime"); + _spill_total_timer = profile->get_counter("SpillTotalTime"); + + if (is_write) { + _spill_write_wait_in_queue_timer = + profile->get_counter("SpillWriteTaskWaitInQueueTime"); + _write_wait_in_queue_task_count = + profile->get_counter("SpillWriteTaskWaitInQueueCount"); + _writing_task_count = profile->get_counter("SpillWriteTaskCount"); + COUNTER_UPDATE(_write_wait_in_queue_task_count, 1); + } + + _submit_timer.start(); + } +public: ~SpillRunnable() override = default; void run() override { + const auto submit_elapsed_time = _submit_timer.elapsed_time(); // Should lock task context before scope task, because the _state maybe // destroyed when run is called. auto task_context_holder = _task_context_holder.lock(); @@ -48,9 +108,28 @@ class SpillRunnable : public Runnable { return; } SCOPED_ATTACH_TASK(_state); + + _exec_timer->update(submit_elapsed_time); + _spill_total_timer->update(submit_elapsed_time); + + SCOPED_TIMER(_exec_timer); + SCOPED_TIMER(_spill_total_timer); + + auto* spill_timer = _get_spill_timer(); + DCHECK(spill_timer != nullptr); + SCOPED_TIMER(spill_timer); + + _on_task_started(submit_elapsed_time); + Defer defer([&] { - std::function tmp; - std::swap(tmp, _func); + { + std::function tmp; + std::swap(tmp, _spill_exec_func); + } + { + std::function tmp; + std::swap(tmp, _spill_fin_cb); + } }); auto shared_state_holder = _shared_state_holder.lock(); @@ -61,14 +140,127 @@ class SpillRunnable : public Runnable { if (_state->is_cancelled()) { return; } - _func(); + auto status = _spill_exec_func(); + if (!status.ok()) { + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(_state->query_id(), status); + } + + _on_task_finished(); + if (_spill_fin_cb) { + auto status2 = _spill_fin_cb(); + if (!status2.ok()) { + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(_state->query_id(), status2); + } + } + + if (_spill_dependency) { + _spill_dependency->set_ready(); + } + } + +protected: + virtual void _on_task_finished() { + if (_spill_context) { + _spill_context->on_task_finished(); + } + } + + virtual RuntimeProfile::Counter* _get_spill_timer() { + return _profile->get_counter("SpillWriteTime"); + } + + virtual void _on_task_started(uint64_t submit_elapsed_time) { + VLOG_DEBUG << "Query: " << print_id(_state->query_id()) + << " spill task started, pipeline task id: " << _state->task_id() + << ", spill dep: " << (void*)(_spill_dependency.get()); + if (_is_write_task) { + COUNTER_UPDATE(_spill_write_wait_in_queue_timer, submit_elapsed_time); + COUNTER_UPDATE(_write_wait_in_queue_task_count, -1); + COUNTER_UPDATE(_writing_task_count, 1); + } } -private: RuntimeState* _state; + RuntimeProfile* _profile; + std::shared_ptr _spill_context; + std::shared_ptr _spill_dependency; + + bool _is_write_task; + +private: + MonotonicStopWatch _submit_timer; + + RuntimeProfile::Counter* _exec_timer = nullptr; + RuntimeProfile::Counter* _spill_total_timer; + + RuntimeProfile::Counter* _spill_write_wait_in_queue_timer = nullptr; + RuntimeProfile::Counter* _write_wait_in_queue_task_count = nullptr; + RuntimeProfile::Counter* _writing_task_count = nullptr; + std::weak_ptr _task_context_holder; - std::weak_ptr _shared_state_holder; - std::function _func; + std::weak_ptr _shared_state_holder; + std::function _spill_exec_func; + std::function _spill_fin_cb; +}; + +class SpillSinkRunnable : public SpillRunnable { +public: + SpillSinkRunnable(RuntimeState* state, std::shared_ptr spill_context, + std::shared_ptr spill_dependency, RuntimeProfile* profile, + const std::shared_ptr& shared_state, + std::function spill_exec_func, + std::function spill_fin_cb = {}) + : SpillRunnable(state, spill_context, spill_dependency, profile, shared_state, true, + spill_exec_func, spill_fin_cb) {} +}; + +class SpillNonSinkRunnable : public SpillRunnable { +public: + SpillNonSinkRunnable(RuntimeState* state, std::shared_ptr spill_dependency, + RuntimeProfile* profile, + const std::shared_ptr& shared_state, + std::function spill_exec_func, + std::function spill_fin_cb = {}) + : SpillRunnable(state, nullptr, spill_dependency, profile, shared_state, true, + spill_exec_func, spill_fin_cb) {} +}; + +class SpillRecoverRunnable : public SpillRunnable { +public: + SpillRecoverRunnable(RuntimeState* state, std::shared_ptr spill_dependency, + RuntimeProfile* profile, + const std::shared_ptr& shared_state, + std::function spill_exec_func, + std::function spill_fin_cb = {}) + : SpillRunnable(state, nullptr, spill_dependency, profile, shared_state, false, + spill_exec_func, spill_fin_cb) { + _spill_revover_timer = profile->get_counter("SpillRecoverTime"); + _spill_read_wait_in_queue_timer = profile->get_counter("SpillReadTaskWaitInQueueTime"); + _read_wait_in_queue_task_count = profile->get_counter("SpillReadTaskWaitInQueueCount"); + _reading_task_count = profile->get_counter("SpillReadTaskCount"); + + COUNTER_UPDATE(_read_wait_in_queue_task_count, 1); + } + +protected: + RuntimeProfile::Counter* _get_spill_timer() override { + return _profile->get_counter("SpillRecoverTime"); + } + + void _on_task_started(uint64_t submit_elapsed_time) override { + LOG(INFO) << "SpillRecoverRunnable, Query: " << print_id(_state->query_id()) + << " spill task started, pipeline task id: " << _state->task_id() + << ", spill dep: " << (void*)(_spill_dependency.get()); + COUNTER_UPDATE(_spill_read_wait_in_queue_timer, submit_elapsed_time); + COUNTER_UPDATE(_read_wait_in_queue_task_count, -1); + COUNTER_UPDATE(_reading_task_count, 1); + } + +private: + RuntimeProfile::Counter* _spill_revover_timer; + RuntimeProfile::Counter* _spill_read_wait_in_queue_timer = nullptr; + RuntimeProfile::Counter* _read_wait_in_queue_task_count = nullptr; + RuntimeProfile::Counter* _reading_task_count = nullptr; }; #include "common/compile_check_end.h" diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_operator.cpp index b6e5788a07c626..e0729db9da6717 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_operator.cpp @@ -497,8 +497,15 @@ Status StreamingAggLocalState::_init_hash_method(const vectorized::VExprContextS return Status::OK(); } +void StreamingAggLocalState::set_low_memory_mode() { + auto& p = Base::_parent->template cast(); + p._spill_streaming_agg_mem_limit = 1024 * 1024; +} Status StreamingAggLocalState::do_pre_agg(vectorized::Block* input_block, vectorized::Block* output_block) { + if (state()->get_query_ctx()->low_memory_mode()) { + set_low_memory_mode(); + } RETURN_IF_ERROR(_pre_agg_with_serialized_key(input_block, output_block)); // pre stream agg need use _num_row_return to decide whether to do pre stream agg @@ -626,8 +633,7 @@ Status StreamingAggLocalState::_pre_agg_with_serialized_key(doris::vectorized::B // to avoid wasting memory. // But for fixed hash map, it never need to expand bool ret_flag = false; - const auto spill_streaming_agg_mem_limit = - _parent->cast()._spill_streaming_agg_mem_limit; + const auto spill_streaming_agg_mem_limit = p._spill_streaming_agg_mem_limit; const bool used_too_much_memory = spill_streaming_agg_mem_limit > 0 && _memory_usage() > spill_streaming_agg_mem_limit; RETURN_IF_ERROR(std::visit( @@ -1147,7 +1153,7 @@ Status StreamingAggOperatorX::init(const TPlanNode& tnode, RuntimeState* state) _aggregate_evaluators.push_back(evaluator); } - if (state->enable_agg_spill()) { + if (state->enable_spill()) { // If spill enabled, the streaming agg should not occupy too much memory. _spill_streaming_agg_mem_limit = state->query_options().__isset.spill_streaming_agg_mem_limit @@ -1261,14 +1267,14 @@ Status StreamingAggLocalState::close(RuntimeState* state) { Status StreamingAggOperatorX::pull(RuntimeState* state, vectorized::Block* block, bool* eos) const { auto& local_state = get_local_state(state); + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); if (!local_state._pre_aggregated_block->empty()) { local_state._pre_aggregated_block->swap(*block); } else { RETURN_IF_ERROR(local_state._executor->get_result(&local_state, state, block, eos)); local_state.make_nullable_output_key(block); // dispose the having clause, should not be execute in prestreaming agg - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + RETURN_IF_ERROR(local_state.filter_block(local_state._conjuncts, block, block->columns())); } local_state.reached_limit(block, eos); @@ -1278,6 +1284,8 @@ Status StreamingAggOperatorX::pull(RuntimeState* state, vectorized::Block* block Status StreamingAggOperatorX::push(RuntimeState* state, vectorized::Block* in_block, bool eos) const { auto& local_state = get_local_state(state); + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); + local_state._input_num_rows += in_block->rows(); if (in_block->rows() > 0) { RETURN_IF_ERROR(local_state.do_pre_agg(in_block, local_state._pre_aggregated_block.get())); diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.h b/be/src/pipeline/exec/streaming_aggregation_operator.h index bd35cd940f2974..7d85d092d17e19 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_operator.h @@ -47,6 +47,7 @@ class StreamingAggLocalState final : public PipelineXLocalState Status close(RuntimeState* state) override; Status do_pre_agg(vectorized::Block* input_block, vectorized::Block* output_block); void make_nullable_output_key(vectorized::Block* block); + void set_low_memory_mode(); private: friend class StreamingAggOperatorX; diff --git a/be/src/pipeline/exec/union_sink_operator.cpp b/be/src/pipeline/exec/union_sink_operator.cpp index 56491b5258bc55..39ca43a9d1df76 100644 --- a/be/src/pipeline/exec/union_sink_operator.cpp +++ b/be/src/pipeline/exec/union_sink_operator.cpp @@ -91,6 +91,9 @@ Status UnionSinkOperatorX::open(RuntimeState* state) { Status UnionSinkOperatorX::sink(RuntimeState* state, vectorized::Block* in_block, bool eos) { auto& local_state = get_local_state(state); + if (state->get_query_ctx()->low_memory_mode()) { + local_state._shared_state->data_queue.set_low_memory_mode(); + } SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); if (local_state._output_block == nullptr) { diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index d13658488e2c9b..470914967aee31 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -91,7 +91,7 @@ std::string UnionSourceLocalState::debug_string(int indentation_level) const { if (_shared_state) { fmt::format_to(debug_string_buffer, ", data_queue: (is_all_finish = {}, has_data = {})", _shared_state->data_queue.is_all_finish(), - _shared_state->data_queue.remaining_has_data()); + _shared_state->data_queue.has_more_data()); } return fmt::to_string(debug_string_buffer); } @@ -131,7 +131,10 @@ Status UnionSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* b } block->swap(*output_block); output_block->clear_column_data(_row_descriptor.num_materialized_slots()); - local_state._shared_state->data_queue.push_free_block(std::move(output_block), child_idx); + if (!state->get_query_ctx()->low_memory_mode()) { + local_state._shared_state->data_queue.push_free_block(std::move(output_block), + child_idx); + } } local_state.reached_limit(block, eos); return Status::OK(); @@ -141,6 +144,9 @@ Status UnionSourceOperatorX::get_next_const(RuntimeState* state, vectorized::Blo DCHECK_EQ(state->per_fragment_instance_idx(), 0); auto& local_state = state->get_local_state(operator_id())->cast(); DCHECK_LT(local_state._const_expr_list_idx, _const_expr_lists.size()); + + SCOPED_PEAK_MEM(&local_state._estimate_memory_usage); + auto& _const_expr_list_idx = local_state._const_expr_list_idx; vectorized::MutableBlock mblock = vectorized::VectorizedUtils::build_mutable_mem_reuse_block(block, _row_descriptor); diff --git a/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp b/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp index b22ee9fd77e72f..084d33e456f1e5 100644 --- a/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp +++ b/be/src/pipeline/local_exchange/local_exchange_sink_operator.cpp @@ -28,7 +28,7 @@ LocalExchangeSinkLocalState::~LocalExchangeSinkLocalState() = default; std::vector LocalExchangeSinkLocalState::dependencies() const { auto deps = Base::dependencies(); - auto dep = _shared_state->get_sink_dep_by_channel_id(_channel_id); + auto* dep = _shared_state->get_sink_dep_by_channel_id(_channel_id); if (dep != nullptr) { deps.push_back(dep); } @@ -144,6 +144,12 @@ Status LocalExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* 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()); + + if (state->get_query_ctx()->low_memory_mode()) { + local_state._shared_state->set_low_memory_mode(state); + local_state._exchanger->set_low_memory_mode(); + } + RETURN_IF_ERROR(local_state._exchanger->sink( state, in_block, eos, {local_state._compute_hash_value_timer, local_state._distribute_timer, nullptr}, diff --git a/be/src/pipeline/local_exchange/local_exchanger.cpp b/be/src/pipeline/local_exchange/local_exchanger.cpp index f93bbf7eec0185..b31dbd7a308791 100644 --- a/be/src/pipeline/local_exchange/local_exchanger.cpp +++ b/be/src/pipeline/local_exchange/local_exchanger.cpp @@ -47,10 +47,13 @@ void Exchanger::_enqueue_data_and_set_ready(int channel_id, block->ref(1); allocated_bytes = block->data_block.allocated_bytes(); } + std::unique_lock l(*_m[channel_id]); + local_state->_shared_state->add_mem_usage(channel_id, allocated_bytes, !std::is_same_v && !std::is_same_v); + if (_data_queue[channel_id].enqueue(std::move(block))) { local_state->_shared_state->set_ready_to_read(channel_id); } else { @@ -85,10 +88,10 @@ bool Exchanger::_dequeue_data(LocalExchangeSourceLocalState* local_st local_state->_shared_state->sub_mem_usage(channel_id, block.first->data_block.allocated_bytes()); } else { - local_state->_shared_state->sub_mem_usage(channel_id, - block->data_block.allocated_bytes()); + const auto allocated_bytes = block->data_block.allocated_bytes(); + local_state->_shared_state->sub_mem_usage(channel_id, allocated_bytes); data_block->swap(block->data_block); - block->unref(local_state->_shared_state, data_block->allocated_bytes(), channel_id); + block->unref(local_state->_shared_state, allocated_bytes, channel_id); DCHECK_EQ(block->ref_value(), 0); } return true; @@ -163,6 +166,9 @@ Status ShuffleExchanger::sink(RuntimeState* state, vectorized::Block* in_block, in_block, *sink_info.channel_id, sink_info.local_state)); } + sink_info.local_state->_memory_used_counter->set( + sink_info.local_state->_shared_state->mem_usage); + return Status::OK(); } @@ -245,6 +251,7 @@ Status ShuffleExchanger::_split_rows(RuntimeState* state, const uint32_t* __rest if (new_block_wrapper->data_block.empty()) { return Status::OK(); } + local_state->_shared_state->add_total_mem_usage(new_block_wrapper->data_block.allocated_bytes(), channel_id); auto bucket_seq_to_instance_idx = @@ -349,6 +356,9 @@ Status PassthroughExchanger::sink(RuntimeState* state, vectorized::Block* in_blo auto channel_id = ((*sink_info.channel_id)++) % _num_partitions; _enqueue_data_and_set_ready(channel_id, sink_info.local_state, std::move(wrapper)); + sink_info.local_state->_memory_used_counter->set( + sink_info.local_state->_shared_state->mem_usage); + return Status::OK(); } @@ -395,6 +405,9 @@ Status PassToOneExchanger::sink(RuntimeState* state, vectorized::Block* in_block BlockWrapperSPtr wrapper = BlockWrapper::create_shared(std::move(new_block)); _enqueue_data_and_set_ready(0, sink_info.local_state, std::move(wrapper)); + sink_info.local_state->_memory_used_counter->set( + sink_info.local_state->_shared_state->mem_usage); + return Status::OK(); } @@ -426,6 +439,9 @@ Status LocalMergeSortExchanger::sink(RuntimeState* state, vectorized::Block* in_ sink_info.local_state->_shared_state->source_deps[*sink_info.channel_id] ->set_always_ready(); } + + sink_info.local_state->_memory_used_counter->set( + sink_info.local_state->_shared_state->mem_usage); return Status::OK(); } @@ -437,6 +453,15 @@ void ExchangerBase::finalize() { } } +void ExchangerBase::set_low_memory_mode() { + _free_block_limit = 0; + + vectorized::Block block; + while (_free_blocks.try_dequeue(block)) { + // do nothing + } +} + void LocalMergeSortExchanger::finalize() { BlockWrapperSPtr next_block; vectorized::Block block; @@ -567,6 +592,8 @@ Status AdaptivePassthroughExchanger::_passthrough_sink(RuntimeState* state, _enqueue_data_and_set_ready(channel_id, sink_info.local_state, BlockWrapper::create_shared(std::move(new_block))); + sink_info.local_state->_memory_used_counter->set( + sink_info.local_state->_shared_state->mem_usage); return Status::OK(); } @@ -586,7 +613,11 @@ Status AdaptivePassthroughExchanger::_shuffle_sink(RuntimeState* state, vectoriz std::iota(channel_ids.begin() + i, channel_ids.end(), 0); } } - return _split_rows(state, channel_ids.data(), block, std::move(sink_info)); + + sink_info.local_state->_memory_used_counter->set( + sink_info.local_state->_shared_state->mem_usage); + RETURN_IF_ERROR(_split_rows(state, channel_ids.data(), block, std::move(sink_info))); + return Status::OK(); } Status AdaptivePassthroughExchanger::_split_rows(RuntimeState* state, diff --git a/be/src/pipeline/local_exchange/local_exchanger.h b/be/src/pipeline/local_exchange/local_exchanger.h index d5d53e041700f6..3a4bccf1f482f5 100644 --- a/be/src/pipeline/local_exchange/local_exchanger.h +++ b/be/src/pipeline/local_exchange/local_exchanger.h @@ -91,6 +91,8 @@ class ExchangerBase { virtual std::string data_queue_debug_string(int i) = 0; + void set_low_memory_mode(); + protected: friend struct LocalExchangeSharedState; friend struct BlockWrapper; @@ -102,7 +104,7 @@ class ExchangerBase { const int _num_partitions; const int _num_senders; const int _num_sources; - const int _free_block_limit = 0; + int _free_block_limit = 0; moodycamel::ConcurrentQueue _free_blocks; }; @@ -219,10 +221,9 @@ struct BlockWrapper { if (ref_count.fetch_sub(1) == 1 && shared_state != nullptr) { DCHECK_GT(allocated_bytes, 0); shared_state->sub_total_mem_usage(allocated_bytes, channel_id); - if (shared_state->exchanger->_free_block_limit == 0 || - shared_state->exchanger->_free_blocks.size_approx() < - shared_state->exchanger->_free_block_limit * - shared_state->exchanger->_num_sources) { + if (shared_state->exchanger->_free_blocks.size_approx() < + shared_state->exchanger->_free_block_limit * + shared_state->exchanger->_num_sources) { data_block.clear_column_data(); // Free blocks is used to improve memory efficiency. Failure during pushing back // free block will not incur any bad result so just ignore the return value. diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index e2ad5e0d14cc42..7cf1a9631059ad 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -22,6 +22,7 @@ #include #include +#include #include // IWYU pragma: no_include #include @@ -190,6 +191,9 @@ void PipelineFragmentContext::cancel(const Status reason) { debug_string()); } + if (reason.is() || reason.is()) { + print_profile("cancel pipeline, reason: " + reason.to_string()); + } _query_ctx->cancel(reason, _fragment_id); if (reason.is()) { _is_report_on_cancel = false; @@ -1273,7 +1277,7 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo /// PartitionedAggSourceOperatorX does not support "group by limit opt(#29641)" yet. /// If `group_by_limit_opt` is true, then it might not need to spill at all. - const bool enable_spill = _runtime_state->enable_agg_spill() && + const bool enable_spill = _runtime_state->enable_spill() && !tnode.agg_node.grouping_exprs.empty() && !group_by_limit_opt; if (tnode.agg_node.aggregate_functions.empty() && !enable_spill && @@ -1367,12 +1371,12 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo case TPlanNodeType::HASH_JOIN_NODE: { const auto is_broadcast_join = tnode.hash_join_node.__isset.is_broadcast_join && tnode.hash_join_node.is_broadcast_join; - const auto enable_join_spill = _runtime_state->enable_join_spill(); - if (enable_join_spill && !is_broadcast_join) { + const auto enable_spill = _runtime_state->enable_spill(); + if (enable_spill && !is_broadcast_join) { auto tnode_ = tnode; /// TODO: support rf in partitioned hash join tnode_.runtime_filters.clear(); - const uint32_t partition_count = 32; + uint32_t partition_count = _runtime_state->spill_hash_join_partition_count(); auto inner_probe_operator = std::make_shared(pool, tnode_, 0, descs); auto inner_sink_operator = @@ -1482,7 +1486,7 @@ Status PipelineFragmentContext::_create_operator(ObjectPool* pool, const TPlanNo break; } case TPlanNodeType::SORT_NODE: { - const auto should_spill = _runtime_state->enable_sort_spill() && + const auto should_spill = _runtime_state->enable_spill() && tnode.sort_node.algorithm == TSortAlgorithm::FULL_SORT; if (should_spill) { op.reset(new SpillSortSourceOperatorX(pool, tnode, next_operator_id(), descs)); @@ -1701,6 +1705,21 @@ Status PipelineFragmentContext::submit() { } } +void PipelineFragmentContext::print_profile(const std::string& extra_info) { + if (_runtime_state->enable_profile()) { + std::stringstream ss; + for (auto runtime_profile_ptr : _runtime_state->pipeline_id_to_profile()) { + runtime_profile_ptr->pretty_print(&ss); + } + + if (_runtime_state->load_channel_profile()) { + _runtime_state->load_channel_profile()->pretty_print(&ss); + } + + LOG_INFO("Query {} fragment {} {}, profile, {}", print_id(this->_query_id), + this->_fragment_id, extra_info, ss.str()); + } +} // If all pipeline tasks binded to the fragment instance are finished, then we could // close the fragment instance. void PipelineFragmentContext::_close_fragment_instance() { @@ -1804,6 +1823,58 @@ Status PipelineFragmentContext::send_report(bool done) { req, std::dynamic_pointer_cast(shared_from_this())); } +size_t PipelineFragmentContext::get_revocable_size(bool* has_running_task) const { + size_t res = 0; + // _tasks will be cleared during ~PipelineFragmentContext, so that it's safe + // here to traverse the vector. + for (const auto& task_instances : _tasks) { + for (const auto& task : task_instances) { + if (task->is_running() || task->is_revoking()) { + LOG_EVERY_N(INFO, 50) << "Query: " << print_id(_query_id) + << " is running, task: " << (void*)task.get() + << ", is_revoking: " << task->is_revoking() + << ", is_running: " << task->is_running() + << ", task info: " << task->debug_string(); + *has_running_task = true; + return 0; + } + + size_t revocable_size = task->get_revocable_size(); + if (revocable_size > _runtime_state->spill_min_revocable_mem()) { + res += revocable_size; + } + } + } + return res; +} + +std::vector PipelineFragmentContext::get_revocable_tasks() const { + std::vector revocable_tasks; + for (const auto& task_instances : _tasks) { + for (const auto& task : task_instances) { + size_t revocable_size_ = task->get_revocable_size(); + if (revocable_size_ > _runtime_state->spill_min_revocable_mem() || + (revocable_size_ > 0 && _query_ctx->enable_force_spill())) { + revocable_tasks.emplace_back(task.get()); + } + } + } + return revocable_tasks; +} + +void PipelineFragmentContext::set_memory_sufficient(bool sufficient) { + for (const auto& task_instances : _tasks) { + for (const auto& task : task_instances) { + auto* dependency = task->get_memory_sufficient_dependency(); + if (sufficient) { + dependency->set_ready(); + } else { + dependency->block(); + } + } + } +} + std::string PipelineFragmentContext::debug_string() { fmt::memory_buffer debug_string_buffer; fmt::format_to(debug_string_buffer, "PipelineFragmentContext Info:\n"); diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h index 1674afa886d520..e9dc7d57130d84 100644 --- a/be/src/pipeline/pipeline_fragment_context.h +++ b/be/src/pipeline/pipeline_fragment_context.h @@ -69,6 +69,8 @@ class PipelineFragmentContext : public TaskExecutionContext { ~PipelineFragmentContext(); + void print_profile(const std::string& extra_info); + std::vector> collect_realtime_profile() const; std::shared_ptr collect_realtime_load_channel_profile() const; @@ -115,6 +117,26 @@ class PipelineFragmentContext : public TaskExecutionContext { [[nodiscard]] int next_sink_operator_id() { return _sink_operator_id--; } + [[nodiscard]] size_t get_revocable_size(bool* has_running_task) const; + + [[nodiscard]] std::vector get_revocable_tasks() const; + + void set_memory_sufficient(bool sufficient); + + void instance_ids(std::vector& ins_ids) const { + ins_ids.resize(_fragment_instance_ids.size()); + for (size_t i = 0; i < _fragment_instance_ids.size(); i++) { + ins_ids[i] = _fragment_instance_ids[i]; + } + } + + void instance_ids(std::vector& ins_ids) const { + ins_ids.resize(_fragment_instance_ids.size()); + for (size_t i = 0; i < _fragment_instance_ids.size(); i++) { + ins_ids[i] = print_id(_fragment_instance_ids[i]); + } + } + void clear_finished_tasks() { for (size_t j = 0; j < _tasks.size(); j++) { for (size_t i = 0; i < _tasks[j].size(); i++) { diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp index 5ed725010ec364..960b5a813ce7aa 100644 --- a/be/src/pipeline/pipeline_task.cpp +++ b/be/src/pipeline/pipeline_task.cpp @@ -17,28 +17,35 @@ #include "pipeline_task.h" +#include #include #include #include -#include -#include #include #include +#include "common/logging.h" #include "common/status.h" +#include "pipeline/dependency.h" #include "pipeline/exec/operator.h" #include "pipeline/exec/scan_operator.h" #include "pipeline/pipeline.h" #include "pipeline/pipeline_fragment_context.h" #include "pipeline/task_queue.h" +#include "pipeline/task_scheduler.h" #include "runtime/descriptors.h" +#include "runtime/exec_env.h" #include "runtime/query_context.h" #include "runtime/thread_context.h" +#include "runtime/workload_group/workload_group_manager.h" #include "util/container_util.hpp" #include "util/defer_op.h" #include "util/mem_info.h" #include "util/runtime_profile.h" +#include "util/uid_util.h" +#include "vec/core/block.h" +#include "vec/spill/spill_stream.h" namespace doris { class RuntimeState; @@ -72,6 +79,11 @@ PipelineTask::PipelineTask( if (shared_state) { _sink_shared_state = shared_state; } + + const auto dependency_name = + fmt::format("MemorySufficientDependency_{}_{}", _sink->node_id(), task_id); + _memory_sufficient_dependency = + pipeline::Dependency::create_unique(-1, -1, dependency_name, true); } Status PipelineTask::prepare(const TPipelineInstanceParams& local_params, const TDataSink& tsink, @@ -187,6 +199,9 @@ void PipelineTask::_init_profile() { _schedule_counts = ADD_COUNTER(_task_profile, "NumScheduleTimes", TUnit::UNIT); _yield_counts = ADD_COUNTER(_task_profile, "NumYieldTimes", TUnit::UNIT); _core_change_times = ADD_COUNTER(_task_profile, "CoreChangeTimes", TUnit::UNIT); + _memory_reserve_times = ADD_COUNTER(_task_profile, "MemoryReserveTimes", TUnit::UNIT); + _memory_reserve_failed_times = + ADD_COUNTER(_task_profile, "MemoryReserveFailedTimes", TUnit::UNIT); } void PipelineTask::_fresh_profile_counter() { @@ -244,6 +259,25 @@ bool PipelineTask::_is_blocked() { _task_profile->add_info_string("BlockedByDependency", _blocked_dep->name()); } }); + + if (_wake_up_early) { + return false; + } + + for (auto* spill_dependency : _spill_dependencies) { + _blocked_dep = spill_dependency->is_blocked_by(this); + if (_blocked_dep != nullptr) { + _blocked_dep->start_watcher(); + return true; + } + } + + _blocked_dep = _memory_sufficient_dependency->is_blocked_by(this); + if (_blocked_dep != nullptr) { + _blocked_dep->start_watcher(); + return true; + } + // `_dry_run = true` means we do not need data from source operator. if (!_dry_run) { for (int i = _read_dependencies.size() - 1; i >= 0; i--) { @@ -257,16 +291,10 @@ bool PipelineTask::_is_blocked() { } // If all dependencies are ready for this operator, we can execute this task if no datum is needed from upstream operators. if (!_operators[i]->need_more_input_data(_state)) { - if (VLOG_DEBUG_IS_ON) { - VLOG_DEBUG << "query: " << print_id(_state->query_id()) - << ", task id: " << _index << ", operator " << i - << " not need_more_input_data"; - } break; } } } - for (auto* op_dep : _write_dependencies) { _blocked_dep = op_dep->is_blocked_by(this); if (_blocked_dep != nullptr) { @@ -330,6 +358,8 @@ Status PipelineTask::execute(bool* eos) { _task_profile->add_info_string("TaskState", "Runnable"); _task_profile->add_info_string("BlockedByDependency", ""); + const auto query_id = _state->query_id(); + while (!_fragment_context->is_canceled()) { if (_is_blocked()) { return Status::OK(); @@ -349,16 +379,12 @@ Status PipelineTask::execute(bool* eos) { _block->clear_column_data(_root->row_desc().num_materialized_slots()); auto* block = _block.get(); - auto sink_revocable_mem_size = _sink->revocable_mem_size(_state); - if (should_revoke_memory(_state, sink_revocable_mem_size)) { - RETURN_IF_ERROR(_sink->revoke_memory(_state)); - continue; - } DBUG_EXECUTE_IF("fault_inject::PipelineXTask::executing", { Status status = Status::Error("fault_inject pipeline_task executing failed"); return status; }); + // `_sink->is_finished(_state)` means sink operator should be finished if (_sink->is_finished(_state)) { set_wake_up_and_dep_ready(); @@ -366,19 +392,100 @@ Status PipelineTask::execute(bool* eos) { // `_dry_run` means sink operator need no more data *eos = wake_up_early() || _dry_run; - if (!*eos) { + auto workload_group = _state->get_query_ctx()->workload_group(); + if (*eos) { + _pending_block.reset(); + } else if (_pending_block) [[unlikely]] { + LOG(INFO) << "Query: " << print_id(query_id) + << " has pending block, size: " << _pending_block->allocated_bytes(); + _block = std::move(_pending_block); + block = _block.get(); + *eos = _pending_eos; + } else { SCOPED_TIMER(_get_block_timer); + DEFER_RELEASE_RESERVED(); _get_block_counter->update(1); - RETURN_IF_ERROR(_root->get_block_after_projects(_state, block, eos)); - } + const auto reserve_size = _root->get_reserve_mem_size(_state); + _root->reset_reserve_mem_size(_state); + + if (workload_group && _state->enable_reserve_memory() && reserve_size > 0) { + auto st = thread_context()->try_reserve_memory(reserve_size); + + COUNTER_UPDATE(_memory_reserve_times, 1); + if (!st.ok() && !_state->enable_force_spill()) { + COUNTER_UPDATE(_memory_reserve_failed_times, 1); + auto debug_msg = fmt::format( + "Query: {} , try to reserve: {}, operator name: {}, operator id: {}, " + "task id: " + "{}, revocable mem size: {}, failed: {}", + print_id(query_id), PrettyPrinter::print_bytes(reserve_size), + _root->get_name(), _root->node_id(), _state->task_id(), + PrettyPrinter::print_bytes(get_revocable_size()), st.to_string()); + // PROCESS_MEMORY_EXCEEDED error msg alread contains process_mem_log_str + if (!st.is()) { + debug_msg += fmt::format(", debug info: {}", + GlobalMemoryArbitrator::process_mem_log_str()); + } + LOG(INFO) << debug_msg; + + ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query( + _state->get_query_ctx()->shared_from_this(), reserve_size, st); + continue; + } + } - if (*eos) { - RETURN_IF_ERROR(close(Status::OK(), false)); + DCHECK_EQ(_pending_block.get(), nullptr); + RETURN_IF_ERROR(_root->get_block_after_projects(_state, block, eos)); } if (_block->rows() != 0 || *eos) { SCOPED_TIMER(_sink_timer); - Status status = _sink->sink(_state, block, *eos); + Status status = Status::OK(); + DEFER_RELEASE_RESERVED(); + COUNTER_UPDATE(_memory_reserve_times, 1); + const auto sink_reserve_size = _sink->get_reserve_mem_size(_state, *eos); + auto workload_group = _state->get_query_ctx()->workload_group(); + if (_state->enable_reserve_memory() && workload_group && + !(wake_up_early() || _dry_run)) { + status = thread_context()->try_reserve_memory(sink_reserve_size); + + if (status.ok() && _state->enable_force_spill() && _sink->is_spillable() && + _sink->revocable_mem_size(_state) >= + vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + status = Status(ErrorCode::QUERY_MEMORY_EXCEEDED, "Force Spill"); + } + + if (!status.ok()) { + COUNTER_UPDATE(_memory_reserve_failed_times, 1); + auto debug_msg = fmt::format( + "Query: {} try to reserve: {}, sink name: {}, node id: {}, task id: " + "{}, revocable mem size: {}, failed: {}", + print_id(query_id), PrettyPrinter::print_bytes(sink_reserve_size), + _sink->get_name(), _sink->node_id(), _state->task_id(), + PrettyPrinter::print_bytes(get_revocable_size()), status.to_string()); + // PROCESS_MEMORY_EXCEEDED error msg alread contains process_mem_log_str + if (!status.is()) { + debug_msg += fmt::format(", debug info: {}", + GlobalMemoryArbitrator::process_mem_log_str()); + } + VLOG_DEBUG << debug_msg; + + DCHECK_EQ(_pending_block.get(), nullptr); + _pending_block = std::move(_block); + _block = vectorized::Block::create_unique(_pending_block->clone_empty()); + ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query( + _state->get_query_ctx()->shared_from_this(), sink_reserve_size, status); + _pending_eos = *eos; + *eos = false; + continue; + } + } + + if (*eos) { + RETURN_IF_ERROR(close(Status::OK(), false)); + } + + status = _sink->sink(_state, block, *eos); if (status.is()) { set_wake_up_and_dep_ready(); @@ -398,63 +505,6 @@ Status PipelineTask::execute(bool* eos) { return Status::OK(); } -bool PipelineTask::should_revoke_memory(RuntimeState* state, int64_t revocable_mem_bytes) { - auto* query_ctx = state->get_query_ctx(); - auto wg = query_ctx->workload_group(); - if (!wg) { - LOG_ONCE(INFO) << "no workload group for query " << print_id(state->query_id()); - return false; - } - const auto min_revocable_mem_bytes = state->min_revocable_mem(); - - if (UNLIKELY(state->enable_force_spill())) { - if (revocable_mem_bytes >= min_revocable_mem_bytes) { - LOG_ONCE(INFO) << "spill force, query: " << print_id(state->query_id()); - return true; - } - } - - bool is_wg_mem_low_water_mark = false; - bool is_wg_mem_high_water_mark = false; - wg->check_mem_used(&is_wg_mem_low_water_mark, &is_wg_mem_high_water_mark); - if (is_wg_mem_high_water_mark) { - if (revocable_mem_bytes > min_revocable_mem_bytes) { - VLOG_DEBUG << "query " << print_id(state->query_id()) - << " revoke memory, hight water mark"; - return true; - } - return false; - } else if (is_wg_mem_low_water_mark) { - int64_t spill_threshold = query_ctx->spill_threshold(); - int64_t memory_usage = query_ctx->query_mem_tracker->consumption(); - if (spill_threshold == 0 || memory_usage < spill_threshold) { - return false; - } - auto big_memory_operator_num = query_ctx->get_running_big_mem_op_num(); - DCHECK(big_memory_operator_num >= 0); - int64_t mem_limit_of_op; - if (0 == big_memory_operator_num) { - return false; - } else { - mem_limit_of_op = spill_threshold / big_memory_operator_num; - } - - LOG_EVERY_T(INFO, 1) << "query " << print_id(state->query_id()) - << " revoke memory, low water mark, revocable_mem_bytes: " - << PrettyPrinter::print_bytes(revocable_mem_bytes) - << ", mem_limit_of_op: " << PrettyPrinter::print_bytes(mem_limit_of_op) - << ", min_revocable_mem_bytes: " - << PrettyPrinter::print_bytes(min_revocable_mem_bytes) - << ", memory_usage: " << PrettyPrinter::print_bytes(memory_usage) - << ", spill_threshold: " << PrettyPrinter::print_bytes(spill_threshold) - << ", big_memory_operator_num: " << big_memory_operator_num; - return (revocable_mem_bytes > mem_limit_of_op || - revocable_mem_bytes > min_revocable_mem_bytes); - } else { - return false; - } -} - void PipelineTask::finalize() { std::unique_lock lc(_dependency_lock); _finalized = true; @@ -532,6 +582,9 @@ std::string PipelineTask::debug_string() { } } + fmt::format_to(debug_string_buffer, "{}. {}\n", i, + _memory_sufficient_dependency->debug_string(i++)); + fmt::format_to(debug_string_buffer, "Write Dependency Information: \n"); for (size_t j = 0; j < _write_dependencies.size(); j++, i++) { fmt::format_to(debug_string_buffer, "{}. {}\n", i, @@ -552,6 +605,35 @@ std::string PipelineTask::debug_string() { return fmt::to_string(debug_string_buffer); } +size_t PipelineTask::get_revocable_size() const { + if (_finalized || _running || (_eos && !_pending_block)) { + return 0; + } + + return _sink->revocable_mem_size(_state); +} + +Status PipelineTask::revoke_memory(const std::shared_ptr& spill_context) { + if (_finalized) { + if (spill_context) { + spill_context->on_task_finished(); + VLOG_DEBUG << "Query: " << print_id(_state->query_id()) << ", task: " << ((void*)this) + << " finalized"; + } + return Status::OK(); + } + + const auto revocable_size = _sink->revocable_mem_size(_state); + if (revocable_size >= vectorized::SpillStream::MIN_SPILL_WRITE_BATCH_MEM) { + RETURN_IF_ERROR(_sink->revoke_memory(_state, spill_context)); + } else if (spill_context) { + spill_context->on_task_finished(); + LOG(INFO) << "Query: " << print_id(_state->query_id()) << ", task: " << ((void*)this) + << " has not enough data to revoke: " << revocable_size; + } + return Status::OK(); +} + void PipelineTask::wake_up() { // call by dependency static_cast(get_task_queue()->push_back(this)); diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index 1a31e5954f479c..de2ea70055f9cb 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -17,8 +17,7 @@ #pragma once -#include - +#include #include #include #include @@ -143,6 +142,11 @@ class PipelineTask { std::unique_lock lc(_dependency_lock); if (!_finalized) { _execution_dep->set_always_ready(); + _memory_sufficient_dependency->set_always_ready(); + for (auto* dep : _spill_dependencies) { + dep->set_always_ready(); + } + for (auto* dep : _filter_dependencies) { dep->set_always_ready(); } @@ -179,14 +183,6 @@ class PipelineTask { void set_core_id(int core_id) { this->_core_id = core_id; } int get_core_id() const { return this->_core_id; } - /** - * Return true if: - * 1. `enable_force_spill` is true which forces this task to spill data. - * 2. Or memory consumption reaches the high water mark of current workload group (80% of memory limitation by default) and revocable_mem_bytes is bigger than min_revocable_mem_bytes. - * 3. Or memory consumption is higher than the low water mark of current workload group (50% of memory limitation by default) and `query_weighted_consumption >= query_weighted_limit` and revocable memory is big enough. - */ - static bool should_revoke_memory(RuntimeState* state, int64_t revocable_mem_bytes); - void put_in_runnable_queue() { _schedule_time++; _wait_worker_watcher.start(); @@ -195,7 +191,15 @@ class PipelineTask { void pop_out_runnable_queue() { _wait_worker_watcher.stop(); } bool is_running() { return _running.load(); } - void set_running(bool running) { _running = running; } + bool is_revoking() { + for (auto* dep : _spill_dependencies) { + if (dep->is_blocked_by(nullptr) != nullptr) { + return true; + } + } + return false; + } + bool set_running(bool running) { return _running.exchange(running); } bool is_exceed_debug_timeout() { if (_has_exceed_timeout) { @@ -236,9 +240,21 @@ class PipelineTask { } PipelineId pipeline_id() const { return _pipeline->id(); } + [[nodiscard]] size_t get_revocable_size() const; + [[nodiscard]] Status revoke_memory(const std::shared_ptr& spill_context); + + void add_spill_dependency(Dependency* dependency) { + _spill_dependencies.emplace_back(dependency); + } bool wake_up_early() const { return _wake_up_early; } + Dependency* get_memory_sufficient_dependency() const { + return _memory_sufficient_dependency.get(); + } + + void inc_memory_reserve_failed_times() { COUNTER_UPDATE(_memory_reserve_failed_times, 1); } + private: friend class RuntimeFilterDependency; bool _is_blocked(); @@ -256,7 +272,10 @@ class PipelineTask { RuntimeState* _state = nullptr; int _previous_schedule_id = -1; uint32_t _schedule_time = 0; - std::unique_ptr _block; + std::unique_ptr _block; + std::unique_ptr _pending_block; + bool _pending_eos = false; + PipelineFragmentContext* _fragment_context = nullptr; MultiCoreTaskQueue* _task_queue = nullptr; @@ -287,6 +306,8 @@ class PipelineTask { // TODO we should calculate the time between when really runnable and runnable RuntimeProfile::Counter* _yield_counts = nullptr; RuntimeProfile::Counter* _core_change_times = nullptr; + RuntimeProfile::Counter* _memory_reserve_times = nullptr; + RuntimeProfile::Counter* _memory_reserve_failed_times = nullptr; MonotonicStopWatch _pipeline_task_watcher; @@ -297,6 +318,7 @@ class PipelineTask { // `_read_dependencies` is stored as same order as `_operators` std::vector> _read_dependencies; + std::vector _spill_dependencies; std::vector _write_dependencies; std::vector _finish_dependencies; std::vector _filter_dependencies; @@ -314,12 +336,14 @@ class PipelineTask { Dependency* _execution_dep = nullptr; - std::atomic _finalized = false; + std::unique_ptr _memory_sufficient_dependency; + + std::atomic _finalized {false}; std::mutex _dependency_lock; - std::atomic _running = false; - std::atomic _eos = false; - std::atomic _wake_up_early = false; + std::atomic _running {false}; + std::atomic _eos {false}; + std::atomic _wake_up_early {false}; }; } // namespace doris::pipeline diff --git a/be/src/pipeline/task_scheduler.cpp b/be/src/pipeline/task_scheduler.cpp index 45898e764175b2..250638965bbf7b 100644 --- a/be/src/pipeline/task_scheduler.cpp +++ b/be/src/pipeline/task_scheduler.cpp @@ -24,19 +24,25 @@ #include // IWYU pragma: no_include +#include #include // IWYU pragma: keep +#include #include +#include +#include #include #include #include #include #include "common/logging.h" +#include "common/status.h" #include "pipeline/pipeline_task.h" #include "pipeline/task_queue.h" #include "pipeline_fragment_context.h" #include "runtime/exec_env.h" #include "runtime/query_context.h" +#include "runtime/thread_context.h" #include "util/thread.h" #include "util/threadpool.h" #include "util/time.h" @@ -59,8 +65,7 @@ Status TaskScheduler::start() { .set_cgroup_cpu_ctl(_cgroup_cpu_ctl) .build(&_fix_thread_pool)); LOG_INFO("TaskScheduler set cores").tag("size", cores); - _markers.resize(cores, true); - for (int i = 0; i < cores; ++i) { + for (int32_t i = 0; i < cores; ++i) { RETURN_IF_ERROR(_fix_thread_pool->submit_func([this, i] { _do_work(i); })); } return Status::OK(); @@ -98,18 +103,20 @@ void _close_task(PipelineTask* task, Status exec_status) { } void TaskScheduler::_do_work(int index) { - while (_markers[index]) { + while (!_need_to_stop) { auto* task = _task_queue.take(index); if (!task) { continue; } + if (task->is_running()) { static_cast(_task_queue.push_back(task, index)); continue; } - task->log_detail_if_need(); task->set_running(true); task->set_task_queue(&_task_queue); + task->log_detail_if_need(); + auto* fragment_ctx = task->fragment_context(); bool canceled = fragment_ctx->is_canceled(); @@ -191,9 +198,7 @@ void TaskScheduler::stop() { if (!_shutdown) { _task_queue.close(); if (_fix_thread_pool) { - for (size_t i = 0; i < _markers.size(); ++i) { - _markers[i] = false; - } + _need_to_stop = true; _fix_thread_pool->shutdown(); _fix_thread_pool->wait(); } diff --git a/be/src/pipeline/task_scheduler.h b/be/src/pipeline/task_scheduler.h index 3c1b08063dfa61..449f0bfbe5dd20 100644 --- a/be/src/pipeline/task_scheduler.h +++ b/be/src/pipeline/task_scheduler.h @@ -17,10 +17,10 @@ #pragma once -#include - #include +#include #include +#include #include #include #include @@ -30,9 +30,11 @@ #include "common/status.h" #include "gutil/ref_counted.h" #include "pipeline_task.h" +#include "runtime/query_context.h" #include "runtime/workload_group/workload_group.h" #include "task_queue.h" #include "util/thread.h" +#include "util/uid_util.h" namespace doris { class ExecEnv; @@ -44,10 +46,7 @@ namespace doris::pipeline { class TaskScheduler { public: TaskScheduler(int core_num, std::string name, std::shared_ptr cgroup_cpu_ctl) - : _task_queue(core_num), - _shutdown(false), - _name(std::move(name)), - _cgroup_cpu_ctl(cgroup_cpu_ctl) {} + : _task_queue(core_num), _name(std::move(name)), _cgroup_cpu_ctl(cgroup_cpu_ctl) {} ~TaskScheduler(); @@ -61,9 +60,10 @@ class TaskScheduler { private: std::unique_ptr _fix_thread_pool; + MultiCoreTaskQueue _task_queue; - std::vector _markers; - bool _shutdown; + bool _need_to_stop = false; + bool _shutdown = false; std::string _name; std::weak_ptr _cgroup_cpu_ctl; diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 788993f4c2afd8..2dddbbd69b4f81 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -829,11 +829,11 @@ std::string FragmentMgr::dump_pipeline_tasks(TUniqueId& query_id) { Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, QuerySource query_source, const FinishCallback& cb) { - VLOG_ROW << "query: " << print_id(params.query_id) << " exec_plan_fragment params is " + VLOG_ROW << "Query: " << print_id(params.query_id) << " exec_plan_fragment params is " << apache::thrift::ThriftDebugString(params).c_str(); // sometimes TExecPlanFragmentParams debug string is too long and glog // will truncate the log line, so print query options seperately for debuggin purpose - VLOG_ROW << "query: " << print_id(params.query_id) << "query options is " + VLOG_ROW << "Query: " << print_id(params.query_id) << "query options is " << apache::thrift::ThriftDebugString(params.query_options).c_str(); std::shared_ptr query_ctx; diff --git a/be/src/runtime/load_channel.h b/be/src/runtime/load_channel.h index 2889bcf256515b..26368e4a3f1df8 100644 --- a/be/src/runtime/load_channel.h +++ b/be/src/runtime/load_channel.h @@ -69,6 +69,8 @@ class LoadChannel { bool is_high_priority() const { return _is_high_priority; } + WorkloadGroupPtr workload_group() const { return _query_thread_context.wg_wptr.lock(); } + RuntimeProfile::Counter* get_mgr_add_batch_timer() { return _mgr_add_batch_timer; } RuntimeProfile::Counter* get_handle_mem_limit_timer() { return _handle_mem_limit_timer; } diff --git a/be/src/runtime/load_channel_mgr.cpp b/be/src/runtime/load_channel_mgr.cpp index c53cade466be04..55db6564488494 100644 --- a/be/src/runtime/load_channel_mgr.cpp +++ b/be/src/runtime/load_channel_mgr.cpp @@ -150,7 +150,8 @@ Status LoadChannelMgr::add_batch(const PTabletWriterAddBlockRequest& request, // If this is a high priority load task, do not handle this. // because this may block for a while, which may lead to rpc timeout. SCOPED_TIMER(channel->get_handle_mem_limit_timer()); - ExecEnv::GetInstance()->memtable_memory_limiter()->handle_memtable_flush(); + ExecEnv::GetInstance()->memtable_memory_limiter()->handle_workload_group_memtable_flush( + channel->workload_group()); } // 3. add batch to load channel diff --git a/be/src/runtime/memory/global_memory_arbitrator.cpp b/be/src/runtime/memory/global_memory_arbitrator.cpp index 733099acfa5617..6dc9feb43a03ae 100644 --- a/be/src/runtime/memory/global_memory_arbitrator.cpp +++ b/be/src/runtime/memory/global_memory_arbitrator.cpp @@ -36,7 +36,13 @@ std::atomic GlobalMemoryArbitrator::refresh_interval_memory_growth = 0; std::mutex GlobalMemoryArbitrator::cache_adjust_capacity_lock; std::condition_variable GlobalMemoryArbitrator::cache_adjust_capacity_cv; std::atomic GlobalMemoryArbitrator::cache_adjust_capacity_notify {false}; +// This capacity is set by gc thread, it is running periodicity. std::atomic GlobalMemoryArbitrator::last_cache_capacity_adjust_weighted {1}; +// This capacity is set by workload group spill disk thread +std::atomic GlobalMemoryArbitrator::last_wg_trigger_cache_capacity_adjust_weighted {1}; +// The value that take affect +std::atomic GlobalMemoryArbitrator::last_affected_cache_capacity_adjust_weighted {1}; +std::atomic GlobalMemoryArbitrator::any_workload_group_exceed_limit {false}; std::mutex GlobalMemoryArbitrator::memtable_memory_refresh_lock; std::condition_variable GlobalMemoryArbitrator::memtable_memory_refresh_cv; std::atomic GlobalMemoryArbitrator::memtable_memory_refresh_notify {false}; diff --git a/be/src/runtime/memory/global_memory_arbitrator.h b/be/src/runtime/memory/global_memory_arbitrator.h index d050ee2211b7d7..f7c9ae1b6c11d8 100644 --- a/be/src/runtime/memory/global_memory_arbitrator.h +++ b/be/src/runtime/memory/global_memory_arbitrator.h @@ -149,15 +149,16 @@ class GlobalMemoryArbitrator { static std::string process_limit_exceeded_errmsg_str() { return fmt::format( - "{} exceed limit {} or {} less than low water mark {}", process_memory_used_str(), - MemInfo::mem_limit_str(), sys_mem_available_str(), + "{} exceed limit {} or {} less than low water mark {}", + process_memory_used_details_str(), MemInfo::mem_limit_str(), + sys_mem_available_details_str(), PrettyPrinter::print(MemInfo::sys_mem_available_low_water_mark(), TUnit::BYTES)); } static std::string process_soft_limit_exceeded_errmsg_str() { return fmt::format("{} exceed soft limit {} or {} less than warning water mark {}.", - process_memory_used_str(), MemInfo::soft_mem_limit_str(), - sys_mem_available_str(), + process_memory_used_details_str(), MemInfo::soft_mem_limit_str(), + sys_mem_available_details_str(), PrettyPrinter::print(MemInfo::sys_mem_available_warning_water_mark(), TUnit::BYTES)); } @@ -170,6 +171,12 @@ class GlobalMemoryArbitrator { static std::condition_variable cache_adjust_capacity_cv; static std::atomic cache_adjust_capacity_notify; static std::atomic last_cache_capacity_adjust_weighted; + // This capacity is set by workload group spill disk thread + static std::atomic last_wg_trigger_cache_capacity_adjust_weighted; + // The value that take affect + static std::atomic last_affected_cache_capacity_adjust_weighted; + static std::atomic any_workload_group_exceed_limit; + static void notify_cache_adjust_capacity() { cache_adjust_capacity_notify.store(true, std::memory_order_relaxed); cache_adjust_capacity_cv.notify_all(); diff --git a/be/src/runtime/memory/mem_tracker_limiter.cpp b/be/src/runtime/memory/mem_tracker_limiter.cpp index 516ad5e5ab2669..cc4218d7653802 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.cpp +++ b/be/src/runtime/memory/mem_tracker_limiter.cpp @@ -86,6 +86,9 @@ std::shared_ptr MemTrackerLimiter::create_shared(MemTrackerLi const std::string& label, int64_t byte_limit) { auto tracker = std::make_shared(type, label, byte_limit); + // Write tracker is only used to tracker the size, so limit == -1 + auto write_tracker = std::make_shared(type, "Memtable" + label, -1); + tracker->_write_tracker.swap(write_tracker); #ifndef BE_TEST DCHECK(ExecEnv::tracking_memory()); std::lock_guard l( @@ -137,7 +140,7 @@ MemTrackerLimiter::~MemTrackerLimiter() { << ", mem tracker label: " << _label << ", peak consumption: " << peak_consumption() << print_address_sanitizers(); } - DCHECK(reserved_consumption() == 0); + DCHECK_EQ(reserved_consumption(), 0); memory_memtrackerlimiter_cnt << -1; } @@ -379,7 +382,8 @@ std::string MemTrackerLimiter::tracker_limit_exceeded_str() { "{}, peak used {}, current used {}. backend {}, {}.", label(), type_string(_type), MemCounter::print_bytes(limit()), MemCounter::print_bytes(peak_consumption()), MemCounter::print_bytes(consumption()), - BackendOptions::get_localhost(), GlobalMemoryArbitrator::process_memory_used_str()); + BackendOptions::get_localhost(), + GlobalMemoryArbitrator::process_memory_used_details_str()); if (_type == Type::QUERY || _type == Type::LOAD) { err_msg += fmt::format( " exec node:<{}>, can `set exec_mem_limit=8G` to change limit, details see " @@ -563,7 +567,7 @@ int64_t MemTrackerLimiter::free_top_overcommit_query( seek_num++; // 32M small query does not cancel if (tracker->consumption() <= 33554432 || - tracker->consumption() < tracker->limit()) { + tracker->consumption() < tracker->_limit) { small_num++; continue; } @@ -573,7 +577,7 @@ int64_t MemTrackerLimiter::free_top_overcommit_query( continue; } auto overcommit_ratio = int64_t( - (static_cast(tracker->consumption()) / tracker->limit()) * + (static_cast(tracker->consumption()) / tracker->_limit) * 10000); max_pq.emplace(overcommit_ratio, tracker->label()); query_consumption[tracker->label()] = tracker->consumption(); diff --git a/be/src/runtime/memory/mem_tracker_limiter.h b/be/src/runtime/memory/mem_tracker_limiter.h index 79071a849f8198..64a8b97eb32539 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.h +++ b/be/src/runtime/memory/mem_tracker_limiter.h @@ -35,6 +35,7 @@ #include "common/config.h" #include "common/status.h" #include "runtime/memory/mem_counter.h" +#include "runtime/memory/mem_tracker.h" #include "runtime/query_statistics.h" #include "util/string_util.h" #include "util/uid_util.h" @@ -45,6 +46,7 @@ class RuntimeProfile; class MemTrackerLimiter; constexpr size_t MEM_TRACKER_GROUP_NUM = 1000; +constexpr size_t QUERY_MIN_MEMORY = 32 * 1024 * 1024; struct TrackerLimiterGroup { // Note! in order to enable ExecEnv::mem_tracker_limiter_pool support resize, @@ -134,10 +136,10 @@ class MemTrackerLimiter final { int64_t byte_limit = -1); // byte_limit equal to -1 means no consumption limit, only participate in process memory statistics. MemTrackerLimiter(Type type, const std::string& label, int64_t byte_limit); - ~MemTrackerLimiter(); Type type() const { return _type; } + void set_overcommit(bool enable) { _enable_overcommit = enable; } const std::string& label() const { return _label; } std::shared_ptr get_query_statistics() { return _query_statistics; } int64_t group_num() const { return _group_num; } @@ -147,7 +149,7 @@ class MemTrackerLimiter final { Status check_limit(int64_t bytes = 0); // Log the memory usage when memory limit is exceeded. std::string tracker_limit_exceeded_str(); - bool is_overcommit_tracker() const { return type() == Type::QUERY || type() == Type::LOAD; } + void set_limit(int64_t new_mem_limit) { _limit = new_mem_limit; } bool is_query_cancelled() { return _is_query_cancelled; } void set_is_query_cancelled(bool is_cancelled) { _is_query_cancelled.store(is_cancelled); } @@ -178,23 +180,6 @@ class MemTrackerLimiter final { void release(int64_t bytes) { _mem_counter.sub(bytes); } - bool try_consume(int64_t bytes) { - if (UNLIKELY(bytes == 0)) { - return true; - } - bool rt = true; - if (is_overcommit_tracker() && !config::enable_query_memory_overcommit) { - rt = _mem_counter.try_add(bytes, _limit); - } else { - _mem_counter.add(bytes); - } - if (rt && _query_statistics) { - _query_statistics->set_max_peak_memory_bytes(peak_consumption()); - _query_statistics->set_current_used_memory_bytes(consumption()); - } - return rt; - } - void set_consumption(int64_t bytes) { _mem_counter.set(bytes); } // Transfer 'bytes' of consumption from this tracker to 'dst'. @@ -216,8 +201,29 @@ class MemTrackerLimiter final { int64_t reserved_consumption() const { return _reserved_counter.current_value(); } int64_t reserved_peak_consumption() const { return _reserved_counter.peak_value(); } + void reserve(int64_t bytes) { + if (UNLIKELY(bytes == 0)) { + return; + } + _mem_counter.add(bytes); + if (_query_statistics) { + _query_statistics->set_max_peak_memory_bytes(peak_consumption()); + _query_statistics->set_current_used_memory_bytes(consumption()); + } + _reserved_counter.add(bytes); + } + bool try_reserve(int64_t bytes) { - bool rt = try_consume(bytes); + if (UNLIKELY(bytes == 0)) { + return true; + } + // If enable overcommit, then the limit is useless, use a very large value as limit + bool rt = _mem_counter.try_add( + bytes, _enable_overcommit ? std::numeric_limits::max() : _limit.load()); + if (rt && _query_statistics) { + _query_statistics->set_max_peak_memory_bytes(peak_consumption()); + _query_statistics->set_current_used_memory_bytes(consumption()); + } if (rt) { _reserved_counter.add(bytes); } @@ -239,6 +245,10 @@ class MemTrackerLimiter final { static void make_top_consumption_tasks_tracker_profile(RuntimeProfile* profile, int top_num); static void make_all_tasks_tracker_profile(RuntimeProfile* profile); + int64_t write_buffer_size() const { return _write_tracker->consumption(); } + + std::shared_ptr write_tracker() { return _write_tracker; } + void print_log_usage(const std::string& msg); void enable_print_log_usage() { _enable_print_log_usage = true; } @@ -308,6 +318,7 @@ class MemTrackerLimiter final { */ Type _type; + bool _enable_overcommit = true; // label used in the make snapshot, not guaranteed unique. std::string _label; @@ -318,7 +329,7 @@ class MemTrackerLimiter final { MemCounter _reserved_counter; // Limit on memory consumption, in bytes. - int64_t _limit; + std::atomic _limit; // Group number in mem_tracker_limiter_pool and mem_tracker_pool, generated by the timestamp. int64_t _group_num; @@ -335,6 +346,8 @@ class MemTrackerLimiter final { std::shared_ptr _query_statistics = nullptr; + std::shared_ptr _write_tracker; + struct AddressSanitizer { size_t size; std::string stack_trace; @@ -364,14 +377,16 @@ inline void MemTrackerLimiter::cache_consume(int64_t bytes) { } inline Status MemTrackerLimiter::check_limit(int64_t bytes) { - if (bytes <= 0 || (is_overcommit_tracker() && config::enable_query_memory_overcommit)) { + /* + if (bytes <= 0 || _enable_overcommit) { return Status::OK(); } + // check limit should ignore memtable size, because it is treated as a cache if (_limit > 0 && consumption() + bytes > _limit) { return Status::MemoryLimitExceeded(fmt::format("failed alloc size {}, {}", MemCounter::print_bytes(bytes), tracker_limit_exceeded_str())); - } + }*/ return Status::OK(); } diff --git a/be/src/runtime/memory/memory_profile.cpp b/be/src/runtime/memory/memory_profile.cpp index 5d649c526014af..2bf6a175e51516 100644 --- a/be/src/runtime/memory/memory_profile.cpp +++ b/be/src/runtime/memory/memory_profile.cpp @@ -333,6 +333,13 @@ int64_t MemoryProfile::other_current_usage() { return memory_other_trackers_sum_bytes.get_value(); } +std::string MemoryProfile::process_memory_detail_str() const { + return fmt::format("Process Memory Summary: {}\n, {}\n, {}\n, {}", + GlobalMemoryArbitrator::process_mem_log_str(), + print_memory_overview_profile(), print_global_memory_profile(), + print_top_memory_tasks_profile()); +} + void MemoryProfile::print_log_process_usage() { if (_enable_print_log_process_usage) { _enable_print_log_process_usage = false; @@ -342,6 +349,7 @@ void MemoryProfile::print_log_process_usage() { LOG(WARNING) << "\n" << print_metadata_memory_profile(); LOG(WARNING) << "\n" << print_cache_memory_profile(); LOG(WARNING) << "\n" << print_top_memory_tasks_profile(); + LOG(WARNING) << process_memory_detail_str(); } } diff --git a/be/src/runtime/memory/memory_profile.h b/be/src/runtime/memory/memory_profile.h index c6aefb72f22e1a..a50a3aa702833e 100644 --- a/be/src/runtime/memory/memory_profile.h +++ b/be/src/runtime/memory/memory_profile.h @@ -65,6 +65,7 @@ class MemoryProfile { // process memory changes more than 256M, or the GC ends void enable_print_log_process_usage() { _enable_print_log_process_usage = true; } void print_log_process_usage(); + std::string process_memory_detail_str() const; private: std::string return_memory_profile_str(const RuntimeProfile* profile) const { diff --git a/be/src/runtime/memory/memory_reclamation.cpp b/be/src/runtime/memory/memory_reclamation.cpp index 503b2dffe62b72..3a2476f60e960e 100644 --- a/be/src/runtime/memory/memory_reclamation.cpp +++ b/be/src/runtime/memory/memory_reclamation.cpp @@ -235,7 +235,7 @@ int64_t MemoryReclamation::tg_enable_overcommit_group_gc(int64_t request_free_me int64_t total_free_memory = 0; bool gc_all_exceeded = request_free_memory >= total_exceeded_memory; std::string log_prefix = fmt::format( - "work load group that enable overcommit, number of group: {}, request_free_memory:{}, " + "workload group that enable overcommit, number of group: {}, request_free_memory:{}, " "total_exceeded_memory:{}", task_groups.size(), request_free_memory, total_exceeded_memory); if (gc_all_exceeded) { diff --git a/be/src/runtime/memory/thread_mem_tracker_mgr.h b/be/src/runtime/memory/thread_mem_tracker_mgr.h index 9dbf4399492d02..1234134c84c37f 100644 --- a/be/src/runtime/memory/thread_mem_tracker_mgr.h +++ b/be/src/runtime/memory/thread_mem_tracker_mgr.h @@ -83,7 +83,7 @@ class ThreadMemTrackerMgr { void consume(int64_t size, int skip_large_memory_check = 0); void flush_untracked_mem(); - doris::Status try_reserve(int64_t size); + doris::Status try_reserve(int64_t size, bool only_check_process_memory); void release_reserved(); @@ -278,39 +278,55 @@ inline void ThreadMemTrackerMgr::flush_untracked_mem() { _stop_consume = false; } -inline doris::Status ThreadMemTrackerMgr::try_reserve(int64_t size) { +inline doris::Status ThreadMemTrackerMgr::try_reserve(int64_t size, + bool only_check_process_memory) { DCHECK(_limiter_tracker); DCHECK(size >= 0); CHECK(init()); // if _reserved_mem not equal to 0, repeat reserve, // _untracked_mem store bytes that not synchronized to process reserved memory. flush_untracked_mem(); - if (!_limiter_tracker->try_reserve(size)) { - auto err_msg = fmt::format( - "reserve memory failed, size: {}, because memory tracker consumption: {}, limit: " - "{}", - size, _limiter_tracker->consumption(), _limiter_tracker->limit()); - return doris::Status::MemoryLimitExceeded(err_msg); - } auto wg_ptr = _wg_wptr.lock(); - if (wg_ptr) { - if (!wg_ptr->add_wg_refresh_interval_memory_growth(size)) { - auto err_msg = fmt::format("reserve memory failed, size: {}, because {}", size, - wg_ptr->memory_debug_string()); - _limiter_tracker->release(size); // rollback - _limiter_tracker->release_reserved(size); // rollback - return doris::Status::MemoryLimitExceeded(err_msg); + if (only_check_process_memory) { + _limiter_tracker->reserve(size); + if (wg_ptr) { + wg_ptr->add_wg_refresh_interval_memory_growth(size); + } + } else { + if (!_limiter_tracker->try_reserve(size)) { + auto err_msg = fmt::format( + "reserve memory failed, size: {}, because query memory exceeded, memory " + "tracker " + "consumption: {}, limit: {}", + PrettyPrinter::print(size, TUnit::BYTES), + PrettyPrinter::print(_limiter_tracker->consumption(), TUnit::BYTES), + PrettyPrinter::print(_limiter_tracker->limit(), TUnit::BYTES)); + return doris::Status::Error(err_msg); + } + if (wg_ptr) { + if (!wg_ptr->try_add_wg_refresh_interval_memory_growth(size)) { + auto err_msg = fmt::format( + "reserve memory failed, size: {}, because workload group memory exceeded, " + "workload group: {}", + PrettyPrinter::print(size, TUnit::BYTES), wg_ptr->memory_debug_string()); + _limiter_tracker->release(size); // rollback + _limiter_tracker->release_reserved(size); // rollback + return doris::Status::Error(err_msg); + } } } + if (!doris::GlobalMemoryArbitrator::try_reserve_process_memory(size)) { - auto err_msg = fmt::format("reserve memory failed, size: {}, because {}", size, - GlobalMemoryArbitrator::process_mem_log_str()); + auto err_msg = + fmt::format("reserve memory failed, size: {}, because proccess memory exceeded, {}", + PrettyPrinter::print(size, TUnit::BYTES), + GlobalMemoryArbitrator::process_mem_log_str()); _limiter_tracker->release(size); // rollback _limiter_tracker->release_reserved(size); // rollback if (wg_ptr) { wg_ptr->sub_wg_refresh_interval_memory_growth(size); // rollback } - return doris::Status::MemoryLimitExceeded(err_msg); + return doris::Status::Error(err_msg); } _reserved_mem += size; return doris::Status::OK(); diff --git a/be/src/runtime/query_context.cpp b/be/src/runtime/query_context.cpp index c777c8100ef213..241cefb89428a1 100644 --- a/be/src/runtime/query_context.cpp +++ b/be/src/runtime/query_context.cpp @@ -23,12 +23,16 @@ #include #include +#include #include #include #include #include +#include #include "common/logging.h" +#include "common/status.h" +#include "olap/olap_common.h" #include "pipeline/dependency.h" #include "pipeline/pipeline_fragment_context.h" #include "runtime/exec_env.h" @@ -75,7 +79,7 @@ QueryContext::QueryContext(TUniqueId query_id, ExecEnv* exec_env, bool is_nereids, TNetworkAddress current_connect_fe, QuerySource query_source) : _timeout_second(-1), - _query_id(query_id), + _query_id(std::move(query_id)), _exec_env(exec_env), _is_nereids(is_nereids), _query_options(query_options), @@ -85,6 +89,7 @@ QueryContext::QueryContext(TUniqueId query_id, ExecEnv* exec_env, _query_watcher.start(); _shared_hash_table_controller.reset(new vectorized::SharedHashTableController()); _execution_dependency = pipeline::Dependency::create_unique(-1, -1, "ExecutionDependency"); + _runtime_filter_mgr = std::make_unique( TUniqueId(), RuntimeFilterParamsContext::create(this), query_mem_tracker, true); @@ -112,30 +117,38 @@ QueryContext::QueryContext(TUniqueId query_id, ExecEnv* exec_env, void QueryContext::_init_query_mem_tracker() { bool has_query_mem_limit = _query_options.__isset.mem_limit && (_query_options.mem_limit > 0); - int64_t _bytes_limit = has_query_mem_limit ? _query_options.mem_limit : -1; - if (_bytes_limit > MemInfo::mem_limit()) { - VLOG_NOTICE << "Query memory limit " << PrettyPrinter::print(_bytes_limit, TUnit::BYTES) + int64_t bytes_limit = has_query_mem_limit ? _query_options.mem_limit : -1; + if (bytes_limit > MemInfo::mem_limit() || bytes_limit == -1) { + VLOG_NOTICE << "Query memory limit " << PrettyPrinter::print(bytes_limit, TUnit::BYTES) << " exceeds process memory limit of " << PrettyPrinter::print(MemInfo::mem_limit(), TUnit::BYTES) - << ". Using process memory limit instead"; - _bytes_limit = MemInfo::mem_limit(); + << " OR is -1. Using process memory limit instead."; + bytes_limit = MemInfo::mem_limit(); + } + // If the query is a pure load task(streamload, routine load, group commit), then it should not use + // memlimit per query to limit their memory usage. + if (is_pure_load_task()) { + bytes_limit = MemInfo::mem_limit(); } if (_query_options.query_type == TQueryType::SELECT) { query_mem_tracker = MemTrackerLimiter::create_shared( MemTrackerLimiter::Type::QUERY, fmt::format("Query#Id={}", print_id(_query_id)), - _bytes_limit); + bytes_limit); } else if (_query_options.query_type == TQueryType::LOAD) { query_mem_tracker = MemTrackerLimiter::create_shared( MemTrackerLimiter::Type::LOAD, fmt::format("Load#Id={}", print_id(_query_id)), - _bytes_limit); + bytes_limit); } else { // EXTERNAL query_mem_tracker = MemTrackerLimiter::create_shared( MemTrackerLimiter::Type::LOAD, fmt::format("External#Id={}", print_id(_query_id)), - _bytes_limit); + bytes_limit); } if (_query_options.__isset.is_report_success && _query_options.is_report_success) { query_mem_tracker->enable_print_log_usage(); } + query_mem_tracker->set_overcommit(enable_mem_overcommit()); + _user_set_mem_limit = bytes_limit; + _expected_mem_limit = _user_set_mem_limit; } QueryContext::~QueryContext() { @@ -214,6 +227,28 @@ void QueryContext::set_execution_dependency_ready() { _execution_dependency->set_ready(); } +void QueryContext::set_memory_sufficient(bool sufficient) { + if (sufficient) { + { + std::lock_guard l(_paused_mutex); + _paused_reason = Status::OK(); + _paused_timer.stop(); + _paused_period_secs += _paused_timer.elapsed_time() / (1000L * 1000L * 1000L); + } + } else { + _paused_timer.start(); + ++_paused_count; + } + + for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { + auto fragment_ctx = fragment_wptr.lock(); + if (!fragment_ctx) { + continue; + } + fragment_ctx->set_memory_sufficient(sufficient); + } +} + void QueryContext::cancel(Status new_status, int fragment_id) { if (!_exec_status.update(new_status)) { return; @@ -351,7 +386,7 @@ void QueryContext::add_fragment_profile( #endif std::lock_guard l(_profile_mutex); - LOG_INFO("Query X add fragment profile, query {}, fragment {}, pipeline profile count {} ", + LOG_INFO("Add fragment profile, query {}, fragment {}, pipeline profile count {} ", print_id(this->_query_id), fragment_id, pipeline_profiles.size()); _profile_map.insert(std::make_pair(fragment_id, pipeline_profiles)); @@ -378,11 +413,139 @@ void QueryContext::_report_query_profile() { ExecEnv::GetInstance()->runtime_query_statistics_mgr()->trigger_report_profile(); } +void QueryContext::get_revocable_info(size_t* revocable_size, size_t* memory_usage, + bool* has_running_task) const { + *revocable_size = 0; + for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { + auto fragment_ctx = fragment_wptr.lock(); + if (!fragment_ctx) { + continue; + } + + *revocable_size += fragment_ctx->get_revocable_size(has_running_task); + + // Should wait for all tasks are not running before revoking memory. + if (*has_running_task) { + break; + } + } + + *memory_usage = query_mem_tracker->consumption(); +} + +size_t QueryContext::get_revocable_size() const { + size_t revocable_size = 0; + for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { + auto fragment_ctx = fragment_wptr.lock(); + if (!fragment_ctx) { + continue; + } + + bool has_running_task = false; + revocable_size += fragment_ctx->get_revocable_size(&has_running_task); + + // Should wait for all tasks are not running before revoking memory. + if (has_running_task) { + return 0; + } + } + return revocable_size; +} + +Status QueryContext::revoke_memory() { + std::vector> tasks; + std::vector> fragments; + for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { + auto fragment_ctx = fragment_wptr.lock(); + if (!fragment_ctx) { + continue; + } + + auto tasks_of_fragment = fragment_ctx->get_revocable_tasks(); + for (auto* task : tasks_of_fragment) { + tasks.emplace_back(task->get_revocable_size(), task); + } + fragments.emplace_back(std::move(fragment_ctx)); + } + + std::sort(tasks.begin(), tasks.end(), [](auto&& l, auto&& r) { return l.first > r.first; }); + + // Do not use memlimit, use current memory usage. + // For example, if current limit is 1.6G, but current used is 1G, if reserve failed + // should free 200MB memory, not 300MB + const auto target_revoking_size = (int64_t)(query_mem_tracker->consumption() * 0.2); + size_t revoked_size = 0; + size_t total_revokable_size = 0; + + std::vector chosen_tasks; + for (auto&& [revocable_size, task] : tasks) { + // Only revoke the largest task to ensure memory is used as much as possible + // break; + if (revoked_size < target_revoking_size) { + chosen_tasks.emplace_back(task); + revoked_size += revocable_size; + } + total_revokable_size += revocable_size; + } + + std::weak_ptr this_ctx = shared_from_this(); + auto spill_context = std::make_shared( + chosen_tasks.size(), _query_id, [this_ctx](pipeline::SpillContext* context) { + auto query_context = this_ctx.lock(); + if (!query_context) { + return; + } + + LOG(INFO) << query_context->debug_string() << ", context: " << ((void*)context) + << " all spill tasks done, resume it."; + query_context->set_memory_sufficient(true); + }); + + LOG(INFO) << fmt::format( + "{}, spill context: {}, revokable mem: {}/{}, tasks count: {}/{}", this->debug_string(), + ((void*)spill_context.get()), PrettyPrinter::print_bytes(revoked_size), + PrettyPrinter::print_bytes(total_revokable_size), chosen_tasks.size(), tasks.size()); + + for (auto* task : chosen_tasks) { + RETURN_IF_ERROR(task->revoke_memory(spill_context)); + } + return Status::OK(); +} + +void QueryContext::decrease_revoking_tasks_count() { + _revoking_tasks_count.fetch_sub(1); +} + +std::vector QueryContext::get_revocable_tasks() const { + std::vector tasks; + for (auto&& [fragment_id, fragment_wptr] : _fragment_id_to_pipeline_ctx) { + auto fragment_ctx = fragment_wptr.lock(); + if (!fragment_ctx) { + continue; + } + auto tasks_of_fragment = fragment_ctx->get_revocable_tasks(); + tasks.insert(tasks.end(), tasks_of_fragment.cbegin(), tasks_of_fragment.cend()); + } + return tasks; +} + +std::string QueryContext::debug_string() { + std::lock_guard l(_paused_mutex); + return fmt::format( + "QueryId={}, Memory [Used={}, Limit={}, Peak={}], Spill[RunningSpillTaskCnt={}, " + "TotalPausedPeriodSecs={}, LatestPausedReason={}]", + print_id(_query_id), + PrettyPrinter::print(query_mem_tracker->consumption(), TUnit::BYTES), + PrettyPrinter::print(query_mem_tracker->limit(), TUnit::BYTES), + PrettyPrinter::print(query_mem_tracker->peak_consumption(), TUnit::BYTES), + _revoking_tasks_count, _paused_period_secs, _paused_reason.to_string()); +} + std::unordered_map>> QueryContext::_collect_realtime_query_profile() const { std::unordered_map>> res; - for (auto& [fragment_id, fragment_ctx_wptr] : _fragment_id_to_pipeline_ctx) { + for (const auto& [fragment_id, fragment_ctx_wptr] : _fragment_id_to_pipeline_ctx) { if (auto fragment_ctx = fragment_ctx_wptr.lock()) { if (fragment_ctx == nullptr) { std::string msg = diff --git a/be/src/runtime/query_context.h b/be/src/runtime/query_context.h index 621c5ebca90cad..96883acd8c4917 100644 --- a/be/src/runtime/query_context.h +++ b/be/src/runtime/query_context.h @@ -23,6 +23,7 @@ #include #include +#include #include #include #include @@ -31,6 +32,7 @@ #include "common/config.h" #include "common/factory_creator.h" #include "common/object_pool.h" +#include "pipeline/dependency.h" #include "runtime/exec_env.h" #include "runtime/memory/mem_tracker_limiter.h" #include "runtime/query_statistics.h" @@ -46,6 +48,7 @@ namespace doris { namespace pipeline { class PipelineFragmentContext; +class PipelineTask; } // namespace pipeline struct ReportStatusRequest { @@ -75,7 +78,7 @@ const std::string toString(QuerySource query_source); // Some components like DescriptorTbl may be very large // that will slow down each execution of fragments when DeSer them every time. class DescriptorTbl; -class QueryContext { +class QueryContext : public std::enable_shared_from_this { ENABLE_FACTORY_CREATOR(QueryContext); public: @@ -117,6 +120,8 @@ class QueryContext { void set_execution_dependency_ready(); + void set_memory_sufficient(bool sufficient); + void set_ready_to_execute_only(); std::shared_ptr get_shared_hash_table_controller() { @@ -171,6 +176,10 @@ class QueryContext { : false; } + bool enable_force_spill() const { + return _query_options.__isset.enable_force_spill && _query_options.enable_force_spill; + } + // global runtime filter mgr, the runtime filter have remote target or // need local merge should regist here. before publish() or push_to_remote() // the runtime filter should do the local merge work @@ -186,6 +195,10 @@ class QueryContext { pipeline::Dependency* get_execution_dependency() { return _execution_dependency.get(); } + std::vector get_revocable_tasks() const; + + Status revoke_memory(); + void register_query_statistics(std::shared_ptr qs); std::shared_ptr get_query_statistics(); @@ -200,8 +213,6 @@ class QueryContext { ThreadPool* get_memtable_flush_pool(); - int64_t mem_limit() const { return _bytes_limit; } - void set_merge_controller_handler( std::shared_ptr& handler) { _merge_controller_handler = handler; @@ -221,8 +232,39 @@ class QueryContext { return _running_big_mem_op_num.load(std::memory_order_relaxed); } - void set_spill_threshold(int64_t spill_threshold) { _spill_threshold = spill_threshold; } - int64_t spill_threshold() { return _spill_threshold; } + void increase_revoking_tasks_count() { _revoking_tasks_count.fetch_add(1); } + + void decrease_revoking_tasks_count(); + + int get_revoking_tasks_count() const { return _revoking_tasks_count.load(); } + + void get_revocable_info(size_t* revocable_size, size_t* memory_usage, + bool* has_running_task) const; + size_t get_revocable_size() const; + + // This method is called by workload group manager to set query's memlimit using slot + // If user set query limit explicitly, then should use less one + void set_mem_limit(int64_t new_mem_limit) { query_mem_tracker->set_limit(new_mem_limit); } + + int64_t get_mem_limit() const { return query_mem_tracker->limit(); } + + void set_expected_mem_limit(int64_t new_mem_limit) { + _expected_mem_limit = std::min(new_mem_limit, _user_set_mem_limit); + } + + // Expected mem limit is the limit when workload group reached limit. + int64_t expected_mem_limit() { return _expected_mem_limit; } + + std::shared_ptr& get_mem_tracker() { return query_mem_tracker; } + + int32_t get_slot_count() const { + return _query_options.__isset.query_slot_count ? _query_options.query_slot_count : 1; + } + + bool enable_mem_overcommit() const { + return _query_options.__isset.enable_mem_overcommit ? _query_options.enable_mem_overcommit + : false; + } DescriptorTbl* desc_tbl = nullptr; bool set_rsc_info = false; std::string user; @@ -266,15 +308,50 @@ class QueryContext { return _using_brpc_stubs; } + void set_low_memory_mode() { _low_memory_mode = true; } + + bool low_memory_mode() { return _low_memory_mode; } + + void update_paused_reason(const Status& st) { + std::lock_guard l(_paused_mutex); + if (_paused_reason.is()) { + return; + } else if (_paused_reason.is()) { + if (st.is()) { + _paused_reason = st; + return; + } else { + return; + } + } else { + _paused_reason = st; + } + } + + Status paused_reason() { + std::lock_guard l(_paused_mutex); + return _paused_reason; + } + + bool is_pure_load_task() { + return _query_source == QuerySource::STREAM_LOAD || + _query_source == QuerySource::ROUTINE_LOAD || + _query_source == QuerySource::GROUP_COMMIT_LOAD; + } + + std::string debug_string(); + private: int _timeout_second; TUniqueId _query_id; ExecEnv* _exec_env = nullptr; MonotonicStopWatch _query_watcher; - int64_t _bytes_limit = 0; bool _is_nereids = false; std::atomic _running_big_mem_op_num = 0; + std::mutex _revoking_tasks_mutex; + std::atomic _revoking_tasks_count = 0; + // A token used to submit olap scanner to the "_limited_scan_thread_pool", // This thread pool token is created from "_limited_scan_thread_pool" from exec env. // And will be shared by all instances of this query. @@ -301,6 +378,8 @@ class QueryContext { vectorized::SimplifiedScanScheduler* _remote_scan_task_scheduler = nullptr; std::unique_ptr _execution_dependency; + std::vector> _pipeline_tasks; + std::shared_ptr _cpu_statistics = nullptr; // This shared ptr is never used. It is just a reference to hold the object. // There is a weak ptr in runtime filter manager to reference this object. @@ -309,7 +388,14 @@ class QueryContext { std::map> _fragment_id_to_pipeline_ctx; std::mutex _pipeline_map_write_lock; - std::atomic _spill_threshold {0}; + std::mutex _paused_mutex; + Status _paused_reason; + std::atomic _paused_count = 0; + MonotonicStopWatch _paused_timer; + std::atomic _paused_period_secs = 0; + std::atomic _low_memory_mode = false; + int64_t _user_set_mem_limit = 0; + std::atomic _expected_mem_limit = 0; std::mutex _profile_mutex; timespec _query_arrival_timestamp; diff --git a/be/src/runtime/query_statistics.cpp b/be/src/runtime/query_statistics.cpp index 110efef5ab920f..80d9b2cdf2a482 100644 --- a/be/src/runtime/query_statistics.cpp +++ b/be/src/runtime/query_statistics.cpp @@ -44,6 +44,9 @@ void QueryStatistics::merge(const QueryStatistics& other) { if (other_memory_used > 0) { this->current_used_memory_bytes = other_memory_used; } + + _spill_write_bytes_to_local_storage += other._spill_write_bytes_to_local_storage; + _spill_read_bytes_from_local_storage += other._spill_read_bytes_from_local_storage; } void QueryStatistics::to_pb(PQueryStatistics* statistics) { @@ -55,6 +58,8 @@ void QueryStatistics::to_pb(PQueryStatistics* statistics) { statistics->set_max_peak_memory_bytes(max_peak_memory_bytes); statistics->set_scan_bytes_from_remote_storage(_scan_bytes_from_remote_storage); statistics->set_scan_bytes_from_local_storage(_scan_bytes_from_local_storage); + statistics->set_spill_write_bytes_to_local_storage(_spill_write_bytes_to_local_storage); + statistics->set_spill_read_bytes_from_local_storage(_spill_read_bytes_from_local_storage); } void QueryStatistics::to_thrift(TQueryStatistics* statistics) const { @@ -69,6 +74,8 @@ void QueryStatistics::to_thrift(TQueryStatistics* statistics) const { statistics->__set_shuffle_send_rows(shuffle_send_rows); statistics->__set_scan_bytes_from_remote_storage(_scan_bytes_from_remote_storage); statistics->__set_scan_bytes_from_local_storage(_scan_bytes_from_local_storage); + statistics->__set_spill_write_bytes_to_local_storage(_spill_write_bytes_to_local_storage); + statistics->__set_spill_read_bytes_from_local_storage(_spill_read_bytes_from_local_storage); } void QueryStatistics::from_pb(const PQueryStatistics& statistics) { @@ -77,6 +84,8 @@ void QueryStatistics::from_pb(const PQueryStatistics& statistics) { cpu_nanos = statistics.cpu_ms() * NANOS_PER_MILLIS; _scan_bytes_from_local_storage = statistics.scan_bytes_from_local_storage(); _scan_bytes_from_remote_storage = statistics.scan_bytes_from_remote_storage(); + _spill_write_bytes_to_local_storage = statistics.spill_write_bytes_to_local_storage(); + _spill_read_bytes_from_local_storage = statistics.spill_read_bytes_from_local_storage(); } QueryStatistics::~QueryStatistics() {} diff --git a/be/src/runtime/query_statistics.h b/be/src/runtime/query_statistics.h index 0a19dfd46f0a08..0b0174172ff299 100644 --- a/be/src/runtime/query_statistics.h +++ b/be/src/runtime/query_statistics.h @@ -47,7 +47,9 @@ class QueryStatistics { max_peak_memory_bytes(0), current_used_memory_bytes(0), shuffle_send_bytes(0), - shuffle_send_rows(0) {} + shuffle_send_rows(0), + _spill_write_bytes_to_local_storage(0), + _spill_read_bytes_from_local_storage(0) {} virtual ~QueryStatistics(); void merge(const QueryStatistics& other); @@ -80,6 +82,14 @@ class QueryStatistics { current_used_memory_bytes = current_used_memory; } + void add_spill_write_bytes_to_local_storage(int64_t bytes) { + _spill_write_bytes_to_local_storage += bytes; + } + + void add_spill_read_bytes_from_local_storage(int64_t bytes) { + _spill_read_bytes_from_local_storage += bytes; + } + void to_pb(PQueryStatistics* statistics); void to_thrift(TQueryStatistics* statistics) const; void from_pb(const PQueryStatistics& statistics); @@ -106,6 +116,9 @@ class QueryStatistics { std::atomic shuffle_send_bytes; std::atomic shuffle_send_rows; + + std::atomic _spill_write_bytes_to_local_storage; + std::atomic _spill_read_bytes_from_local_storage; }; using QueryStatisticsPtr = std::shared_ptr; // It is used for collecting sub plan query statistics in DataStreamRecvr. diff --git a/be/src/runtime/runtime_query_statistics_mgr.cpp b/be/src/runtime/runtime_query_statistics_mgr.cpp index 93d5256cad7525..25c730e5b276a9 100644 --- a/be/src/runtime/runtime_query_statistics_mgr.cpp +++ b/be/src/runtime/runtime_query_statistics_mgr.cpp @@ -526,23 +526,27 @@ void RuntimeQueryStatisticsMgr::get_active_be_tasks_block(vectorized::Block* blo qs_ctx_ptr->collect_query_statistics(&tqs); SchemaScannerHelper::insert_int64_value(0, be_id, block); SchemaScannerHelper::insert_string_value(1, qs_ctx_ptr->_fe_addr.hostname, block); - SchemaScannerHelper::insert_string_value(2, query_id, block); + SchemaScannerHelper::insert_int64_value(2, qs_ctx_ptr->_wg_id, block); + SchemaScannerHelper::insert_string_value(3, query_id, block); int64_t task_time = qs_ctx_ptr->_is_query_finished ? qs_ctx_ptr->_query_finish_time - qs_ctx_ptr->_query_start_time : MonotonicMillis() - qs_ctx_ptr->_query_start_time; - SchemaScannerHelper::insert_int64_value(3, task_time, block); - SchemaScannerHelper::insert_int64_value(4, tqs.cpu_ms, block); - SchemaScannerHelper::insert_int64_value(5, tqs.scan_rows, block); - SchemaScannerHelper::insert_int64_value(6, tqs.scan_bytes, block); - SchemaScannerHelper::insert_int64_value(7, tqs.max_peak_memory_bytes, block); - SchemaScannerHelper::insert_int64_value(8, tqs.current_used_memory_bytes, block); - SchemaScannerHelper::insert_int64_value(9, tqs.shuffle_send_bytes, block); - SchemaScannerHelper::insert_int64_value(10, tqs.shuffle_send_rows, block); + SchemaScannerHelper::insert_int64_value(4, task_time, block); + SchemaScannerHelper::insert_int64_value(5, tqs.cpu_ms, block); + SchemaScannerHelper::insert_int64_value(6, tqs.scan_rows, block); + SchemaScannerHelper::insert_int64_value(7, tqs.scan_bytes, block); + SchemaScannerHelper::insert_int64_value(8, tqs.max_peak_memory_bytes, block); + SchemaScannerHelper::insert_int64_value(9, tqs.current_used_memory_bytes, block); + SchemaScannerHelper::insert_int64_value(10, tqs.shuffle_send_bytes, block); + SchemaScannerHelper::insert_int64_value(11, tqs.shuffle_send_rows, block); std::stringstream ss; ss << qs_ctx_ptr->_query_type; - SchemaScannerHelper::insert_string_value(11, ss.str(), block); + SchemaScannerHelper::insert_string_value(12, ss.str(), block); + + SchemaScannerHelper::insert_int64_value(13, tqs.spill_write_bytes_to_local_storage, block); + SchemaScannerHelper::insert_int64_value(14, tqs.spill_read_bytes_from_local_storage, block); } } diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index df7c4141691d0b..2cbc45dbd9498f 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -277,6 +277,10 @@ std::shared_ptr RuntimeState::query_mem_tracker() const { return _query_mem_tracker; } +WorkloadGroupPtr RuntimeState::workload_group() { + return _query_ctx->workload_group(); +} + bool RuntimeState::log_error(const std::string& error) { std::lock_guard l(_error_log_lock); diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 3cc3d97b5d22c5..084f6522229ff7 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -26,6 +26,7 @@ #include #include +#include #include #include #include @@ -45,6 +46,7 @@ #include "io/fs/file_system.h" #include "io/fs/s3_file_system.h" #include "runtime/task_execution_context.h" +#include "runtime/workload_group/workload_group.h" #include "util/debug_util.h" #include "util/runtime_profile.h" #include "vec/columns/columns_number.h" @@ -61,6 +63,7 @@ class PipelineXLocalStateBase; class PipelineXSinkLocalStateBase; class PipelineFragmentContext; class PipelineTask; +class Dependency; } // namespace pipeline class DescriptorTbl; @@ -403,20 +406,6 @@ class RuntimeState { bool enable_page_cache() const; - int partitioned_hash_join_rows_threshold() const { - if (!_query_options.__isset.partitioned_hash_join_rows_threshold) { - return 0; - } - return _query_options.partitioned_hash_join_rows_threshold; - } - - int partitioned_hash_agg_rows_threshold() const { - if (!_query_options.__isset.partitioned_hash_agg_rows_threshold) { - return 0; - } - return _query_options.partitioned_hash_agg_rows_threshold; - } - const std::vector& tablet_commit_infos() const { return _tablet_commit_infos; } @@ -445,6 +434,7 @@ class RuntimeState { QueryContext* get_query_ctx() { return _query_ctx; } std::weak_ptr get_query_ctx_weak(); + WorkloadGroupPtr workload_group(); void set_query_mem_tracker(const std::shared_ptr& tracker) { _query_mem_tracker = tracker; @@ -510,13 +500,6 @@ class RuntimeState { : 0; } - int64_t external_sort_bytes_threshold() const { - if (_query_options.__isset.external_sort_bytes_threshold) { - return _query_options.external_sort_bytes_threshold; - } - return 0; - } - void set_be_exec_version(int32_t version) noexcept { _query_options.be_exec_version = version; } inline bool enable_delete_sub_pred_v2() const { @@ -563,23 +546,66 @@ class RuntimeState { std::shared_ptr* producer_filter); bool is_nereids() const; - bool enable_join_spill() const { + bool enable_spill() const { return (_query_options.__isset.enable_force_spill && _query_options.enable_force_spill) || - (_query_options.__isset.enable_join_spill && _query_options.enable_join_spill); + (_query_options.__isset.enable_spill && _query_options.enable_spill); } - bool enable_sort_spill() const { - return (_query_options.__isset.enable_force_spill && _query_options.enable_force_spill) || - (_query_options.__isset.enable_sort_spill && _query_options.enable_sort_spill); + bool enable_force_spill() const { + return _query_options.__isset.enable_force_spill && _query_options.enable_force_spill; } - bool enable_agg_spill() const { - return (_query_options.__isset.enable_force_spill && _query_options.enable_force_spill) || - (_query_options.__isset.enable_agg_spill && _query_options.enable_agg_spill); + bool enable_reserve_memory() const { + return _query_options.__isset.enable_reserve_memory && _query_options.enable_reserve_memory; } - bool enable_force_spill() const { - return _query_options.__isset.enable_force_spill && _query_options.enable_force_spill; + 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 1; + } + + 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); + } + return 134217728; + } + + 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); + } + return 8388608; + } + + 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); + } + return 32; + } + + 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); + } + return 32; + } + + int64_t low_memory_mode_buffer_limit() const { + if (_query_options.__isset.low_memory_mode_buffer_limit) { + return std::max(_query_options.low_memory_mode_buffer_limit, (int64_t)1); + } + return 32L * 1024 * 1024; + } + + int spill_revocable_memory_high_watermark_percent() const { + if (_query_options.__isset.revocable_memory_high_watermark_percent) { + return _query_options.revocable_memory_high_watermark_percent; + } + return -1; } bool enable_local_merge_sort() const { @@ -601,7 +627,16 @@ class RuntimeState { if (_query_options.__isset.min_revocable_mem) { return std::max(_query_options.min_revocable_mem, (int64_t)1); } - return 1; + return 32L * 1024 * 1024; + } + + size_t minimum_operator_memory_required_bytes() const { + if (_query_options.__isset.minimum_operator_memory_required_kb) { + return _query_options.minimum_operator_memory_required_kb * 1024; + } else { + // refer other database + return 100 * 1024; + } } void set_max_operator_id(int max_operator_id) { _max_operator_id = max_operator_id; } diff --git a/be/src/runtime/thread_context.cpp b/be/src/runtime/thread_context.cpp index c89f532e5927a6..55c5273ef71ddd 100644 --- a/be/src/runtime/thread_context.cpp +++ b/be/src/runtime/thread_context.cpp @@ -46,10 +46,18 @@ AttachTask::AttachTask(const std::shared_ptr& mem_tracker) { AttachTask::AttachTask(RuntimeState* runtime_state) { signal::set_signal_is_nereids(runtime_state->is_nereids()); - QueryThreadContext query_thread_context = {runtime_state->query_id(), - runtime_state->query_mem_tracker(), - runtime_state->get_query_ctx()->workload_group()}; - init(query_thread_context); + // RuntimeState not always has query ctx. + // For example during push handler or schema change + if (runtime_state->get_query_ctx() == nullptr) { + QueryThreadContext query_thread_context = {runtime_state->query_id(), + runtime_state->query_mem_tracker()}; + init(query_thread_context); + } else { + QueryThreadContext query_thread_context = { + runtime_state->query_id(), runtime_state->query_mem_tracker(), + runtime_state->get_query_ctx()->workload_group()}; + init(query_thread_context); + } } AttachTask::AttachTask(const QueryThreadContext& query_thread_context) { diff --git a/be/src/runtime/thread_context.h b/be/src/runtime/thread_context.h index 9ba7949ec5afad..a51b03a8b96ecc 100644 --- a/be/src/runtime/thread_context.h +++ b/be/src/runtime/thread_context.h @@ -100,7 +100,8 @@ #define SCOPED_CONSUME_MEM_TRACKER_BY_HOOK(mem_tracker) \ auto VARNAME_LINENUM(add_mem_consumer) = doris::AddThreadMemTrackerConsumerByHook(mem_tracker) #else -#define SCOPED_PEAK_MEM() auto VARNAME_LINENUM(scoped_tls_pm) = doris::ScopedInitThreadContext() +#define SCOPED_PEAK_MEM(peak_mem) \ + auto VARNAME_LINENUM(scoped_tls_pm) = doris::ScopedInitThreadContext() #define SCOPED_CONSUME_MEM_TRACKER_BY_HOOK(mem_tracker) \ auto VARNAME_LINENUM(scoped_tls_cmtbh) = doris::ScopedInitThreadContext() #endif @@ -250,13 +251,22 @@ class ThreadContext { thread_mem_tracker_mgr->consume(size, skip_large_memory_check); } + doris::Status try_reserve_process_memory(const int64_t size) const { +#ifdef USE_MEM_TRACKER + DCHECK(doris::k_doris_exit || !doris::config::enable_memory_orphan_check || + thread_mem_tracker()->label() != "Orphan") + << doris::memory_orphan_check_msg; +#endif + return thread_mem_tracker_mgr->try_reserve(size, true); + } + doris::Status try_reserve_memory(const int64_t size) const { #ifdef USE_MEM_TRACKER DCHECK(doris::k_doris_exit || !doris::config::enable_memory_orphan_check || thread_mem_tracker()->label() != "Orphan") << doris::memory_orphan_check_msg; #endif - return thread_mem_tracker_mgr->try_reserve(size); + return thread_mem_tracker_mgr->try_reserve(size, false); } void release_reserved_memory() const { diff --git a/be/src/runtime/workload_group/workload_group.cpp b/be/src/runtime/workload_group/workload_group.cpp index d68a69cc98135d..48d1a3c15b7d19 100644 --- a/be/src/runtime/workload_group/workload_group.cpp +++ b/be/src/runtime/workload_group/workload_group.cpp @@ -48,8 +48,12 @@ namespace doris { const static std::string MEMORY_LIMIT_DEFAULT_VALUE = "0%"; const static bool ENABLE_MEMORY_OVERCOMMIT_DEFAULT_VALUE = true; const static int CPU_HARD_LIMIT_DEFAULT_VALUE = -1; -const static int MEMORY_LOW_WATERMARK_DEFAULT_VALUE = 50; -const static int MEMORY_HIGH_WATERMARK_DEFAULT_VALUE = 80; + +const static int MEMORY_LOW_WATERMARK_DEFAULT_VALUE = 80; +const static int MEMORY_HIGH_WATERMARK_DEFAULT_VALUE = 95; +// This is a invalid value, and should ignore this value during usage +const static int TOTAL_QUERY_SLOT_COUNT_DEFAULT_VALUE = 0; +const static int LOAD_BUFFER_RATIO_DEFAULT_VALUE = 20; WorkloadGroup::WorkloadGroup(const WorkloadGroupInfo& wg_info) : WorkloadGroup(wg_info, true) {} @@ -58,6 +62,7 @@ WorkloadGroup::WorkloadGroup(const WorkloadGroupInfo& tg_info, bool need_create_ _name(tg_info.name), _version(tg_info.version), _memory_limit(tg_info.memory_limit), + _load_buffer_ratio(tg_info.write_buffer_ratio), _enable_memory_overcommit(tg_info.enable_memory_overcommit), _cpu_share(tg_info.cpu_share), _mem_tracker_limiter_pool(MEM_TRACKER_GROUP_NUM), @@ -69,6 +74,8 @@ WorkloadGroup::WorkloadGroup(const WorkloadGroupInfo& tg_info, bool need_create_ _memory_high_watermark(tg_info.memory_high_watermark), _scan_bytes_per_second(tg_info.read_bytes_per_second), _remote_scan_bytes_per_second(tg_info.remote_read_bytes_per_second), + _total_query_slot_count(tg_info.total_query_slot_count), + _slot_mem_policy(tg_info.slot_mem_policy), _need_create_query_thread_pool(need_create_query_thread_pool) { std::vector& data_dir_list = io::BeConfDataDirReader::be_config_data_dir_list; for (const auto& data_dir : data_dir_list) { @@ -81,31 +88,62 @@ WorkloadGroup::WorkloadGroup(const WorkloadGroupInfo& tg_info, bool need_create_ std::string WorkloadGroup::debug_string() const { std::shared_lock rl {_mutex}; + auto realtime_total_mem_used = _total_mem_used + _wg_refresh_interval_memory_growth.load(); + auto mem_used_ratio = realtime_total_mem_used / ((double)_memory_limit + 1); return fmt::format( - "TG[id = {}, name = {}, cpu_share = {}, memory_limit = {}, enable_memory_overcommit = " - "{}, version = {}, cpu_hard_limit = {}, scan_thread_num = " + "WorkloadGroup[id = {}, name = {}, version = {}, cpu_share = {}, " + "total_query_slot_count = {}, " + "memory_limit = {}, slot_memory_policy = {}, write_buffer_ratio= {}%, " + "enable_memory_overcommit = {}, total_mem_used = {} (write_buffer_size={})," + "wg_refresh_interval_memory_growth = {}, mem_used_ratio = {}, cpu_hard_limit = {}, " + "scan_thread_num = " "{}, max_remote_scan_thread_num = {}, min_remote_scan_thread_num = {}, " "memory_low_watermark={}, memory_high_watermark={}, is_shutdown={}, query_num={}, " "read_bytes_per_second={}, remote_read_bytes_per_second={}]", - _id, _name, cpu_share(), PrettyPrinter::print(_memory_limit, TUnit::BYTES), - _enable_memory_overcommit ? "true" : "false", _version, cpu_hard_limit(), - _scan_thread_num, _max_remote_scan_thread_num, _min_remote_scan_thread_num, - _memory_low_watermark, _memory_high_watermark, _is_shutdown, _query_ctxs.size(), - _scan_bytes_per_second, _remote_scan_bytes_per_second); + _id, _name, _version, cpu_share(), _total_query_slot_count, + PrettyPrinter::print(_memory_limit, TUnit::BYTES), to_string(_slot_mem_policy), + _load_buffer_ratio, _enable_memory_overcommit ? "true" : "false", + PrettyPrinter::print(_total_mem_used.load(), TUnit::BYTES), + PrettyPrinter::print(_write_buffer_size.load(), TUnit::BYTES), + PrettyPrinter::print(_wg_refresh_interval_memory_growth.load(), TUnit::BYTES), + mem_used_ratio, cpu_hard_limit(), _scan_thread_num, _max_remote_scan_thread_num, + _min_remote_scan_thread_num, _memory_low_watermark, _memory_high_watermark, + _is_shutdown, _query_ctxs.size(), _scan_bytes_per_second, + _remote_scan_bytes_per_second); +} + +bool WorkloadGroup::try_add_wg_refresh_interval_memory_growth(int64_t size) { + auto realtime_total_mem_used = + _total_mem_used + _wg_refresh_interval_memory_growth.load() + size; + if ((realtime_total_mem_used > + ((double)_memory_limit * _memory_high_watermark.load(std::memory_order_relaxed) / 100))) { + // If a group is enable memory overcommit, then not need check the limit + // It is always true, and it will only fail when process memory is not + // enough. + return _enable_memory_overcommit; + } else { + _wg_refresh_interval_memory_growth.fetch_add(size); + return true; + } } std::string WorkloadGroup::memory_debug_string() const { + auto realtime_total_mem_used = _total_mem_used + _wg_refresh_interval_memory_growth.load(); + auto mem_used_ratio = realtime_total_mem_used / ((double)_memory_limit + 1); return fmt::format( - "TG[id = {}, name = {}, memory_limit = {}, enable_memory_overcommit = " - "{}, weighted_memory_limit = {}, total_mem_used = {}, " - "wg_refresh_interval_memory_growth = {}, memory_low_watermark = {}, " - "memory_high_watermark = {}, version = {}, is_shutdown = {}, query_num = {}]", - _id, _name, PrettyPrinter::print(_memory_limit, TUnit::BYTES), - _enable_memory_overcommit ? "true" : "false", - PrettyPrinter::print(_weighted_memory_limit, TUnit::BYTES), - PrettyPrinter::print(_total_mem_used, TUnit::BYTES), - PrettyPrinter::print(_wg_refresh_interval_memory_growth, TUnit::BYTES), - _memory_low_watermark, _memory_high_watermark, _version, _is_shutdown, + "WorkloadGroup[id = {}, name = {}, version = {}," + "total_query_slot_count = {}, " + "memory_limit = {}, slot_memory_policy = {}, write_buffer_ratio= {}%, " + "enable_memory_overcommit = {}, total_mem_used = {} (write_buffer_size={})," + "wg_refresh_interval_memory_growth = {}, mem_used_ratio = {}, " + "memory_low_watermark={}, memory_high_watermark={}, is_shutdown={}, query_num={}]", + _id, _name, _version, _total_query_slot_count, + PrettyPrinter::print(_memory_limit, TUnit::BYTES), to_string(_slot_mem_policy), + _load_buffer_ratio, _enable_memory_overcommit ? "true" : "false", + PrettyPrinter::print(_total_mem_used.load(), TUnit::BYTES), + PrettyPrinter::print(_write_buffer_size.load(), TUnit::BYTES), + PrettyPrinter::print(_wg_refresh_interval_memory_growth.load(), TUnit::BYTES), + mem_used_ratio, _memory_low_watermark, _memory_high_watermark, _is_shutdown, _query_ctxs.size()); } @@ -135,6 +173,9 @@ void WorkloadGroup::check_and_update(const WorkloadGroupInfo& tg_info) { _memory_high_watermark = tg_info.memory_high_watermark; _scan_bytes_per_second = tg_info.read_bytes_per_second; _remote_scan_bytes_per_second = tg_info.remote_read_bytes_per_second; + _total_query_slot_count = tg_info.total_query_slot_count; + _load_buffer_ratio = tg_info.write_buffer_ratio; + _slot_mem_policy = tg_info.slot_mem_policy; } else { return; } @@ -142,9 +183,9 @@ void WorkloadGroup::check_and_update(const WorkloadGroupInfo& tg_info) { } // MemtrackerLimiter is not removed during query context release, so that should remove it here. -int64_t WorkloadGroup::make_memory_tracker_snapshots( - std::list>* tracker_snapshots) { +int64_t WorkloadGroup::refresh_memory_usage() { int64_t used_memory = 0; + int64_t write_buffer_size = 0; for (auto& mem_tracker_group : _mem_tracker_limiter_pool) { std::lock_guard l(mem_tracker_group.group_lock); for (auto trackerWptr = mem_tracker_group.trackers.begin(); @@ -153,17 +194,17 @@ int64_t WorkloadGroup::make_memory_tracker_snapshots( if (tracker == nullptr) { trackerWptr = mem_tracker_group.trackers.erase(trackerWptr); } else { - if (tracker_snapshots != nullptr) { - tracker_snapshots->insert(tracker_snapshots->end(), tracker); - } used_memory += tracker->consumption(); + write_buffer_size += tracker->write_buffer_size(); ++trackerWptr; } } } // refresh total memory used. - _total_mem_used = used_memory; + + _total_mem_used = used_memory + write_buffer_size; _wg_metrics->update_memory_used_bytes(used_memory); + _write_buffer_size = write_buffer_size; // reserve memory is recorded in the query mem tracker // and _total_mem_used already contains all the current reserve memory. // so after refreshing _total_mem_used, reset _wg_refresh_interval_memory_growth. @@ -172,7 +213,7 @@ int64_t WorkloadGroup::make_memory_tracker_snapshots( } int64_t WorkloadGroup::memory_used() { - return make_memory_tracker_snapshots(nullptr); + return refresh_memory_usage(); } void WorkloadGroup::do_sweep() { @@ -209,6 +250,91 @@ void WorkloadGroup::add_mem_tracker_limiter(std::shared_ptr m _mem_tracker_limiter_pool[group_num].trackers.end(), mem_tracker_ptr); } +int64_t WorkloadGroup::free_overcommited_memory(int64_t need_free_mem, RuntimeProfile* profile) { + if (need_free_mem <= 0) { + return 0; + } + int64_t used_memory = memory_used(); + // Change need free mem to exceed limit + need_free_mem = std::min(used_memory - _memory_limit, need_free_mem); + if (need_free_mem <= 0) { + return 0; + } + + int64_t freed_mem = 0; + + std::string cancel_str = + fmt::format("Kill overcommit query, wg id:{}, name:{}, used:{}, limit:{}, backend:{}.", + _id, _name, MemCounter::print_bytes(used_memory), + MemCounter::print_bytes(_memory_limit), BackendOptions::get_localhost()); + + auto cancel_top_overcommit_str = [cancel_str](int64_t mem_consumption, + const std::string& label) { + return fmt::format( + "{} cancel top memory overcommit tracker <{}> consumption {}. details:{}, " + "Execute again after enough memory, details see be.INFO.", + cancel_str, label, MemCounter::print_bytes(mem_consumption), + GlobalMemoryArbitrator::process_limit_exceeded_errmsg_str()); + }; + + LOG(INFO) << fmt::format( + "Workload group start gc, id:{} name:{}, memory limit: {}, used: {}, " + "need_free_mem: {}.", + _id, _name, _memory_limit, used_memory, need_free_mem); + Defer defer {[&]() { + LOG(INFO) << fmt::format( + "Workload group finished gc, id:{} name:{}, memory limit: {}, used: " + "{}, need_free_mem: {}, freed memory: {}.", + _id, _name, _memory_limit, used_memory, need_free_mem, freed_mem); + }}; + + // 1. free top overcommit query + RuntimeProfile* tmq_profile = profile->create_child( + fmt::format("FreeGroupTopOvercommitQuery:Name {}", _name), true, true); + freed_mem += MemTrackerLimiter::free_top_overcommit_query( + need_free_mem - freed_mem, MemTrackerLimiter::Type::QUERY, _mem_tracker_limiter_pool, + cancel_top_overcommit_str, tmq_profile, MemTrackerLimiter::GCType::WORK_LOAD_GROUP); + // To be compatible with the non-group's gc logic, minorGC just gc overcommit query + if (freed_mem >= need_free_mem) { + return freed_mem; + } + auto cancel_top_usage_str = [cancel_str](int64_t mem_consumption, const std::string& label) { + return fmt::format( + "{} cancel top memory used tracker <{}> consumption {}. details:{}, Execute " + "again " + "after enough memory, details see be.INFO.", + cancel_str, label, MemCounter::print_bytes(mem_consumption), + GlobalMemoryArbitrator::process_soft_limit_exceeded_errmsg_str()); + }; + // 2. free top usage query + tmq_profile = + profile->create_child(fmt::format("FreeGroupTopUsageQuery:Name {}", _name), true, true); + freed_mem += MemTrackerLimiter::free_top_memory_query( + need_free_mem - freed_mem, MemTrackerLimiter::Type::QUERY, _mem_tracker_limiter_pool, + cancel_top_usage_str, tmq_profile, MemTrackerLimiter::GCType::WORK_LOAD_GROUP); + if (freed_mem >= need_free_mem) { + return freed_mem; + } + + // 3. free top overcommit load + tmq_profile = profile->create_child(fmt::format("FreeGroupTopOvercommitLoad:Name {}", _name), + true, true); + freed_mem += MemTrackerLimiter::free_top_overcommit_query( + need_free_mem - freed_mem, MemTrackerLimiter::Type::LOAD, _mem_tracker_limiter_pool, + cancel_top_overcommit_str, tmq_profile, MemTrackerLimiter::GCType::WORK_LOAD_GROUP); + if (freed_mem >= need_free_mem) { + return freed_mem; + } + + // 4. free top usage load + tmq_profile = + profile->create_child(fmt::format("FreeGroupTopUsageLoad:Name {}", _name), true, true); + freed_mem += MemTrackerLimiter::free_top_memory_query( + need_free_mem - freed_mem, MemTrackerLimiter::Type::LOAD, _mem_tracker_limiter_pool, + cancel_top_usage_str, tmq_profile, MemTrackerLimiter::GCType::WORK_LOAD_GROUP); + return freed_mem; +} + int64_t WorkloadGroup::gc_memory(int64_t need_free_mem, RuntimeProfile* profile, bool is_minor_gc) { if (need_free_mem <= 0) { return 0; @@ -416,6 +542,24 @@ WorkloadGroupInfo WorkloadGroupInfo::parse_topic_info( remote_read_bytes_per_second = tworkload_group_info.remote_read_bytes_per_second; } + // 16 total slots + int total_query_slot_count = TOTAL_QUERY_SLOT_COUNT_DEFAULT_VALUE; + if (tworkload_group_info.__isset.total_query_slot_count) { + total_query_slot_count = tworkload_group_info.total_query_slot_count; + } + + // 17 load buffer memory limit + int write_buffer_ratio = LOAD_BUFFER_RATIO_DEFAULT_VALUE; + if (tworkload_group_info.__isset.write_buffer_ratio) { + write_buffer_ratio = tworkload_group_info.write_buffer_ratio; + } + + // 18 slot memory policy + TWgSlotMemoryPolicy::type slot_mem_policy = TWgSlotMemoryPolicy::NONE; + if (tworkload_group_info.__isset.slot_memory_policy) { + slot_mem_policy = tworkload_group_info.slot_memory_policy; + } + return {.id = tg_id, .name = name, .cpu_share = cpu_share, @@ -430,7 +574,10 @@ WorkloadGroupInfo WorkloadGroupInfo::parse_topic_info( .memory_low_watermark = memory_low_watermark, .memory_high_watermark = memory_high_watermark, .read_bytes_per_second = read_bytes_per_second, - .remote_read_bytes_per_second = remote_read_bytes_per_second}; + .remote_read_bytes_per_second = remote_read_bytes_per_second, + .total_query_slot_count = total_query_slot_count, + .slot_mem_policy = slot_mem_policy, + .write_buffer_ratio = write_buffer_ratio}; } std::weak_ptr WorkloadGroup::get_cgroup_cpu_ctl_wptr() { diff --git a/be/src/runtime/workload_group/workload_group.h b/be/src/runtime/workload_group/workload_group.h index 35a8802e4c449a..94ac64adb12ee5 100644 --- a/be/src/runtime/workload_group/workload_group.h +++ b/be/src/runtime/workload_group/workload_group.h @@ -28,6 +28,7 @@ #include #include +#include "common/factory_creator.h" #include "common/status.h" #include "service/backend_options.h" #include "util/hash_util.hpp" @@ -56,6 +57,8 @@ struct TrackerLimiterGroup; class WorkloadGroupMetrics; class WorkloadGroup : public std::enable_shared_from_this { + ENABLE_FACTORY_CREATOR(WorkloadGroup); + public: explicit WorkloadGroup(const WorkloadGroupInfo& tg_info); @@ -79,18 +82,20 @@ class WorkloadGroup : public std::enable_shared_from_this { int64_t memory_limit() const { std::shared_lock r_lock(_mutex); return _memory_limit; - }; + } - int64_t weighted_memory_limit() const { return _weighted_memory_limit; }; + int64_t total_mem_used() const { return _total_mem_used; } - void set_weighted_memory_limit(int64_t weighted_memory_limit) { - _weighted_memory_limit = weighted_memory_limit; - } + int64_t write_buffer_size() const { return _write_buffer_size; } + + void enable_write_buffer_limit(bool enable_limit) { _enable_write_buffer_limit = enable_limit; } + + bool enable_write_buffer_limit() const { return _enable_write_buffer_limit; } + + bool exceed_write_buffer_limit() const { return _write_buffer_size > write_buffer_limit(); } // make memory snapshots and refresh total memory used at the same time. - int64_t make_memory_tracker_snapshots( - std::list>* tracker_snapshots); - // call make_memory_tracker_snapshots, so also refresh total memory used. + int64_t refresh_memory_usage(); int64_t memory_used(); void do_sweep(); @@ -98,35 +103,35 @@ class WorkloadGroup : public std::enable_shared_from_this { int memory_low_watermark() const { return _memory_low_watermark.load(std::memory_order_relaxed); } + int memory_high_watermark() const { return _memory_high_watermark.load(std::memory_order_relaxed); } void set_weighted_memory_ratio(double ratio); - bool add_wg_refresh_interval_memory_growth(int64_t size) { - auto realtime_total_mem_used = - _total_mem_used + _wg_refresh_interval_memory_growth.load() + size; - if ((realtime_total_mem_used > - ((double)_weighted_memory_limit * - _memory_high_watermark.load(std::memory_order_relaxed) / 100))) { - return false; - } else { - _wg_refresh_interval_memory_growth.fetch_add(size); - return true; - } + + int total_query_slot_count() const { + return _total_query_slot_count.load(std::memory_order_relaxed); } + + void add_wg_refresh_interval_memory_growth(int64_t size) { + _wg_refresh_interval_memory_growth.fetch_add(size); + } + + bool try_add_wg_refresh_interval_memory_growth(int64_t size); + void sub_wg_refresh_interval_memory_growth(int64_t size) { _wg_refresh_interval_memory_growth.fetch_sub(size); } - void check_mem_used(bool* is_low_wartermark, bool* is_high_wartermark) const { + void check_mem_used(bool* is_low_watermark, bool* is_high_watermark) const { auto realtime_total_mem_used = _total_mem_used + _wg_refresh_interval_memory_growth.load(); - *is_low_wartermark = (realtime_total_mem_used > - ((double)_weighted_memory_limit * - _memory_low_watermark.load(std::memory_order_relaxed) / 100)); - *is_high_wartermark = (realtime_total_mem_used > - ((double)_weighted_memory_limit * - _memory_high_watermark.load(std::memory_order_relaxed) / 100)); + *is_low_watermark = (realtime_total_mem_used > + ((double)_memory_limit * + _memory_low_watermark.load(std::memory_order_relaxed) / 100)); + *is_high_watermark = (realtime_total_mem_used > + ((double)_memory_limit * + _memory_high_watermark.load(std::memory_order_relaxed) / 100)); } std::string debug_string() const; @@ -143,6 +148,13 @@ class WorkloadGroup : public std::enable_shared_from_this { return _memory_limit > 0; } + TWgSlotMemoryPolicy::type slot_memory_policy() const { return _slot_mem_policy; } + + bool exceed_limit() { + std::shared_lock r_lock(_mutex); + return _memory_limit > 0 ? _total_mem_used > _memory_limit : false; + } + Status add_query(TUniqueId query_id, std::shared_ptr query_ctx) { std::unique_lock wlock(_mutex); if (_is_shutdown) { @@ -211,6 +223,10 @@ class WorkloadGroup : public std::enable_shared_from_this { friend class WorkloadGroupMetrics; + int64_t write_buffer_limit() const { return _memory_limit * _load_buffer_ratio / 100; } + + int64_t free_overcommited_memory(int64_t need_free_mem, RuntimeProfile* profile); + private: void create_cgroup_cpu_ctl_no_lock(); void upsert_cgroup_cpu_ctl_no_lock(WorkloadGroupInfo* wg_info); @@ -222,11 +238,14 @@ class WorkloadGroup : public std::enable_shared_from_this { std::string _name; int64_t _version; int64_t _memory_limit; // bytes - // `weighted_memory_limit` less than or equal to _memory_limit, calculate after exclude public memory. - // more detailed description in `refresh_wg_weighted_memory_limit`. - std::atomic _weighted_memory_limit {0}; // - // last value of make_memory_tracker_snapshots, refresh every time make_memory_tracker_snapshots is called. + // For example, load memtable, write to parquet. + // If the wg's memory reached high water mark, then the load buffer + // will be restricted to this limit. + int64_t _load_buffer_ratio = 0; + std::atomic _enable_write_buffer_limit = false; + std::atomic_int64_t _total_mem_used = 0; // bytes + std::atomic_int64_t _write_buffer_size = 0; std::atomic_int64_t _wg_refresh_interval_memory_growth; bool _enable_memory_overcommit; std::atomic _cpu_share; @@ -239,6 +258,8 @@ class WorkloadGroup : public std::enable_shared_from_this { std::atomic _memory_high_watermark; std::atomic _scan_bytes_per_second {-1}; std::atomic _remote_scan_bytes_per_second {-1}; + std::atomic _total_query_slot_count = 0; + std::atomic _slot_mem_policy {TWgSlotMemoryPolicy::NONE}; // means workload group is mark dropped // new query can not submit @@ -283,6 +304,9 @@ struct WorkloadGroupInfo { const int memory_high_watermark = 0; const int read_bytes_per_second = -1; const int remote_read_bytes_per_second = -1; + const int total_query_slot_count = 0; + const TWgSlotMemoryPolicy::type slot_mem_policy = TWgSlotMemoryPolicy::NONE; + const int write_buffer_ratio = 0; // log cgroup cpu info uint64_t cgroup_cpu_shares = 0; int cgroup_cpu_hard_limit = 0; diff --git a/be/src/runtime/workload_group/workload_group_manager.cpp b/be/src/runtime/workload_group/workload_group_manager.cpp index 1e01a7ce1bafb1..fae389d6341212 100644 --- a/be/src/runtime/workload_group/workload_group_manager.cpp +++ b/be/src/runtime/workload_group/workload_group_manager.cpp @@ -17,17 +17,22 @@ #include "workload_group_manager.h" +#include + #include #include #include #include +#include "common/status.h" #include "exec/schema_scanner/schema_scanner_helper.h" #include "pipeline/task_scheduler.h" +#include "runtime/memory/global_memory_arbitrator.h" #include "runtime/memory/mem_tracker_limiter.h" #include "runtime/workload_group/workload_group.h" #include "runtime/workload_group/workload_group_metrics.h" #include "util/mem_info.h" +#include "util/pretty_printer.h" #include "util/threadpool.h" #include "util/time.h" #include "vec/core/block.h" @@ -54,6 +59,16 @@ void WorkloadGroupMgr::init_internal_workload_group() { } } +PausedQuery::PausedQuery(std::shared_ptr query_ctx, double cache_ratio, + bool any_wg_exceed_limit, int64_t reserve_size) + : query_ctx_(query_ctx), + cache_ratio_(cache_ratio), + any_wg_exceed_limit_(any_wg_exceed_limit), + reserve_size_(reserve_size), + query_id_(print_id(query_ctx->query_id())) { + enqueue_at = std::chrono::system_clock::now(); +} + WorkloadGroupPtr WorkloadGroupMgr::get_or_create_workload_group( const WorkloadGroupInfo& workload_group_info) { { @@ -179,12 +194,14 @@ void WorkloadGroupMgr::refresh_wg_weighted_memory_limit() { // 1. make all workload groups memory snapshots(refresh workload groups total memory used at the same time) // and calculate total memory used of all queries. int64_t all_workload_groups_mem_usage = 0; - std::unordered_map wgs_mem_info; + bool has_wg_exceed_limit = false; for (auto& [wg_id, wg] : _workload_groups) { - wgs_mem_info[wg_id].total_mem_used = - wg->make_memory_tracker_snapshots(&wgs_mem_info[wg_id].tracker_snapshots); - all_workload_groups_mem_usage += wgs_mem_info[wg_id].total_mem_used; + all_workload_groups_mem_usage += wg->refresh_memory_usage(); + if (wg->exceed_limit()) { + has_wg_exceed_limit = true; + } } + doris::GlobalMemoryArbitrator::any_workload_group_exceed_limit = has_wg_exceed_limit; if (all_workload_groups_mem_usage <= 0) { return; } @@ -212,6 +229,8 @@ void WorkloadGroupMgr::refresh_wg_weighted_memory_limit() { if (all_workload_groups_mem_usage < process_memory_usage) { int64_t public_memory = process_memory_usage - all_workload_groups_mem_usage; weighted_memory_limit_ratio = 1 - (double)public_memory / (double)process_memory_limit; + // Round the value from 1% to 100%. + weighted_memory_limit_ratio = std::floor(weighted_memory_limit_ratio * 100) / 100; } std::string debug_msg = fmt::format( @@ -221,59 +240,9 @@ void WorkloadGroupMgr::refresh_wg_weighted_memory_limit() { doris::GlobalMemoryArbitrator::sys_mem_available_details_str(), PrettyPrinter::print(all_workload_groups_mem_usage, TUnit::BYTES), weighted_memory_limit_ratio); - LOG_EVERY_T(INFO, 10) << debug_msg; - + LOG_EVERY_T(INFO, 60) << debug_msg; for (auto& wg : _workload_groups) { - // 3.1 calculate query spill threshold of task group - auto wg_weighted_mem_limit = - int64_t(wg.second->memory_limit() * weighted_memory_limit_ratio); - wg.second->set_weighted_memory_limit(wg_weighted_mem_limit); - - // 3.2 set workload groups weighted memory limit and all query spill threshold. - auto wg_query_count = wgs_mem_info[wg.first].tracker_snapshots.size(); - int64_t query_spill_threshold = - wg_query_count ? (wg_weighted_mem_limit + wg_query_count) / wg_query_count - : wg_weighted_mem_limit; - for (const auto& query : wg.second->queries()) { - auto query_ctx = query.second.lock(); - if (!query_ctx) { - continue; - } - query_ctx->set_spill_threshold(query_spill_threshold); - } - - // 3.3 only print debug logs, if workload groups is_high_wartermark or is_low_wartermark. - bool is_low_wartermark = false; - bool is_high_wartermark = false; - wg.second->check_mem_used(&is_low_wartermark, &is_high_wartermark); - std::string debug_msg; - if (is_high_wartermark || is_low_wartermark) { - debug_msg = fmt::format( - "\nWorkload Group {}: mem limit: {}, mem used: {}, weighted mem limit: {}, " - "used " - "ratio: {}, query count: {}, query spill threshold: {}", - wg.second->name(), - PrettyPrinter::print(wg.second->memory_limit(), TUnit::BYTES), - PrettyPrinter::print(wgs_mem_info[wg.first].total_mem_used, TUnit::BYTES), - PrettyPrinter::print(wg_weighted_mem_limit, TUnit::BYTES), - (double)wgs_mem_info[wg.first].total_mem_used / wg_weighted_mem_limit, - wg_query_count, PrettyPrinter::print(query_spill_threshold, TUnit::BYTES)); - - debug_msg += "\n Query Memory Summary:"; - // check whether queries need to revoke memory for task group - for (const auto& query_mem_tracker : wgs_mem_info[wg.first].tracker_snapshots) { - debug_msg += fmt::format( - "\n MemTracker Label={}, Used={}, SpillThreshold={}, " - "Peak={}", - query_mem_tracker->label(), - PrettyPrinter::print(query_mem_tracker->consumption(), TUnit::BYTES), - PrettyPrinter::print(query_spill_threshold, TUnit::BYTES), - PrettyPrinter::print(query_mem_tracker->peak_consumption(), TUnit::BYTES)); - } - LOG_EVERY_T(INFO, 1) << debug_msg; - } else { - continue; - } + update_queries_limit_(wg.second, false); } } @@ -295,7 +264,6 @@ void WorkloadGroupMgr::get_wg_resource_usage(vectorized::Block* block) { cpu_usage_p = std::round(cpu_usage_p * 100.0) / 100.0; SchemaScannerHelper::insert_double_value(3, cpu_usage_p, block); - SchemaScannerHelper::insert_int64_value( 4, wg->get_metrics()->get_local_scan_bytes_per_second(), block); SchemaScannerHelper::insert_int64_value( @@ -310,6 +278,602 @@ void WorkloadGroupMgr::refresh_workload_group_metrics() { } } +void WorkloadGroupMgr::add_paused_query(const std::shared_ptr& query_ctx, + int64_t reserve_size, const Status& status) { + DCHECK(query_ctx != nullptr); + query_ctx->update_paused_reason(status); + query_ctx->set_low_memory_mode(); + query_ctx->set_memory_sufficient(false); + std::lock_guard lock(_paused_queries_lock); + auto wg = query_ctx->workload_group(); + auto&& [it, inserted] = _paused_queries_list[wg].emplace( + query_ctx, doris::GlobalMemoryArbitrator::last_affected_cache_capacity_adjust_weighted, + doris::GlobalMemoryArbitrator::any_workload_group_exceed_limit, reserve_size); + // Check if this is an invalid reserve, for example, if the reserve size is too large, larger than the query limit + // if hard limit is enabled, then not need enable other queries hard limit. + if (inserted) { + LOG(INFO) << "Insert one new paused query: " << query_ctx->debug_string() + << ", workload group: " << wg->debug_string(); + } +} + +/** + * Strategy 1: A revocable query should not have any running task(PipelineTask). + * strategy 2: If the workload group has any task exceed workload group memlimit, then set all queryctx's memlimit + * strategy 3: If any query exceed process memlimit, then should clear all caches. + * strategy 4: If any query exceed query's memlimit, then do spill disk or cancel it. + * strategy 5: If any query exceed process's memlimit and cache is zero, then do following: + */ +void WorkloadGroupMgr::handle_paused_queries() { + { + std::shared_lock r_lock(_group_mutex); + for (auto& [wg_id, wg] : _workload_groups) { + std::unique_lock lock(_paused_queries_lock); + if (_paused_queries_list[wg].empty()) { + // Add an empty set to wg that not contains paused queries. + } + } + } + + std::unique_lock lock(_paused_queries_lock); + bool has_revoked_from_other_group = false; + for (auto it = _paused_queries_list.begin(); it != _paused_queries_list.end();) { + auto& queries_list = it->second; + const auto& wg = it->first; + + LOG_EVERY_T(INFO, 120) << "Paused queries count: " << queries_list.size(); + + bool is_low_watermark = false; + bool is_high_watermark = false; + wg->check_mem_used(&is_low_watermark, &is_high_watermark); + + bool has_changed_hard_limit = false; + int64_t flushed_memtable_bytes = 0; + // If the query is paused because its limit exceed the query itself's memlimit, then just spill disk. + // The query's memlimit is set using slot mechanism and its value is set using the user settings, not + // by weighted value. So if reserve failed, then it is actually exceed limit. + for (auto query_it = queries_list.begin(); query_it != queries_list.end();) { + auto query_ctx = query_it->query_ctx_.lock(); + // The query is finished during in paused list. + if (query_ctx == nullptr) { + LOG(INFO) << "Query: " << query_it->query_id() << " is nullptr, erase it."; + query_it = queries_list.erase(query_it); + continue; + } + if (query_ctx->is_cancelled()) { + LOG(INFO) << "Query: " << print_id(query_ctx->query_id()) + << " was canceled, remove from paused list"; + query_it = queries_list.erase(query_it); + continue; + } + + if (query_ctx->paused_reason().is()) { + // Streamload, kafka load, group commit will never have query memory exceeded error because + // their query limit is very large. + bool spill_res = + handle_single_query_(query_ctx, query_it->reserve_size_, + query_it->elapsed_time(), query_ctx->paused_reason()); + if (!spill_res) { + ++query_it; + continue; + } else { + query_it = queries_list.erase(query_it); + continue; + } + } else if (query_ctx->paused_reason().is()) { + // Only deal with non overcommit workload group. + if (wg->enable_memory_overcommit()) { + // Soft limit wg will only reserve failed when process limit exceed. But in some corner case, + // when reserve, the wg is hard limit, the query reserve failed, but when this loop run + // the wg is converted to soft limit. + // So that should resume the query. + LOG(WARNING) + << "Query: " << print_id(query_ctx->query_id()) + << " reserve memory failed because exceed workload group memlimit, it " + "should not happen, resume it again. paused reason: " + << query_ctx->paused_reason(); + query_ctx->set_memory_sufficient(true); + query_it = queries_list.erase(query_it); + continue; + } + // check if the reserve is too large, if it is too large, + // should set the query's limit only. + // Check the query's reserve with expected limit. + if (query_ctx->expected_mem_limit() < + query_ctx->get_mem_tracker()->consumption() + query_it->reserve_size_) { + query_ctx->set_mem_limit(query_ctx->expected_mem_limit()); + query_ctx->set_memory_sufficient(true); + LOG(INFO) << "Workload group memory reserve failed because " + << query_ctx->debug_string() << " reserve size " + << PrettyPrinter::print_bytes(query_it->reserve_size_) + << " is too large, set hard limit to " + << PrettyPrinter::print_bytes(query_ctx->expected_mem_limit()) + << " and resume running."; + query_it = queries_list.erase(query_it); + continue; + } + if (flushed_memtable_bytes <= 0) { + flushed_memtable_bytes = + flush_memtable_from_current_group_(wg, query_it->reserve_size_); + } + if (flushed_memtable_bytes > 0) { + // Flushed some memtable, just wait flush finished and not do anything more. + wg->enable_write_buffer_limit(true); + ++query_it; + continue; + } + if (!has_changed_hard_limit) { + update_queries_limit_(wg, true); + has_changed_hard_limit = true; + LOG(INFO) << "Query: " << print_id(query_ctx->query_id()) << " reserve memory(" + << PrettyPrinter::print_bytes(query_it->reserve_size_) + << ") failed due to workload group memory exceed, " + "should set the workload group work in memory insufficent mode, " + "so that other query will reduce their memory." + << " Query mem limit: " + << PrettyPrinter::print_bytes(query_ctx->get_mem_limit()) + << " mem usage: " + << PrettyPrinter::print_bytes( + query_ctx->get_mem_tracker()->consumption()) + << ", wg: " << wg->debug_string(); + } + if (wg->slot_memory_policy() == TWgSlotMemoryPolicy::NONE) { + // If not enable slot memory policy, then should spill directly + // Maybe there are another query that use too much memory, but we + // not encourage not enable slot memory. + // TODO should kill the query that exceed limit. + bool spill_res = handle_single_query_(query_ctx, query_it->reserve_size_, + query_it->elapsed_time(), + query_ctx->paused_reason()); + if (!spill_res) { + ++query_it; + continue; + } else { + query_it = queries_list.erase(query_it); + continue; + } + } else { + // Should not put the query back to task scheduler immediately, because when wg's memory not sufficient, + // and then set wg's flag, other query may not free memory very quickly. + if (query_it->elapsed_time() > config::spill_in_paused_queue_timeout_ms) { + // set wg's memory to insufficent, then add it back to task scheduler to run. + LOG(INFO) << "Query: " << print_id(query_ctx->query_id()) + << " will be resume."; + query_ctx->set_memory_sufficient(true); + query_it = queries_list.erase(query_it); + continue; + } else { + ++query_it; + continue; + } + } + } else { + // If wg's memlimit not exceed, but process memory exceed, it means cache or other metadata + // used too much memory. Should clean all cache here. + // 1. Check cache used, if cache is larger than > 0, then just return and wait for it to 0 to release some memory. + if (doris::GlobalMemoryArbitrator::last_affected_cache_capacity_adjust_weighted > + 0.001 && + doris::GlobalMemoryArbitrator::last_wg_trigger_cache_capacity_adjust_weighted > + 0.001) { + doris::GlobalMemoryArbitrator::last_wg_trigger_cache_capacity_adjust_weighted = + 0; + doris::GlobalMemoryArbitrator::notify_cache_adjust_capacity(); + LOG(INFO) << "There are some queries need process memory, so that set cache " + "capacity " + "to 0 now"; + } + if (query_it->cache_ratio_ < 0.001) { + // 1. Check if could revoke some memory from memtable + if (flushed_memtable_bytes <= 0) { + flushed_memtable_bytes = + flush_memtable_from_current_group_(wg, query_it->reserve_size_); + } + if (flushed_memtable_bytes > 0) { + // Flushed some memtable, just wait flush finished and not do anything more. + ++query_it; + continue; + } + // TODO should wait here to check if the process has release revoked_size memory and then continue. + if (!has_revoked_from_other_group) { + int64_t revoked_size = revoke_memory_from_other_group_( + query_ctx, wg->enable_memory_overcommit(), query_it->reserve_size_); + if (revoked_size > 0) { + has_revoked_from_other_group = true; + query_ctx->set_memory_sufficient(true); + query_it = queries_list.erase(query_it); + // Do not care if the revoked_size > reserve size, and try to run again. + continue; + } else { + bool spill_res = handle_single_query_( + query_ctx, query_it->reserve_size_, query_it->elapsed_time(), + query_ctx->paused_reason()); + if (spill_res) { + query_it = queries_list.erase(query_it); + continue; + } else { + ++query_it; + continue; + } + } + } else { + // If any query is cancelled during process limit stage, should resume other query and + // do not do any check now. + query_ctx->set_memory_sufficient(true); + query_it = queries_list.erase(query_it); + continue; + } + } + if (doris::GlobalMemoryArbitrator::last_affected_cache_capacity_adjust_weighted < + 0.001 && + query_it->cache_ratio_ > 0.001) { + LOG(INFO) << "Query: " << print_id(query_ctx->query_id()) + << " will be resume after cache adjust."; + query_ctx->set_memory_sufficient(true); + query_it = queries_list.erase(query_it); + continue; + } + ++query_it; + } + } + // Not need waiting flush memtable and below low watermark disable load buffer limit + if (flushed_memtable_bytes <= 0 && !is_low_watermark) { + wg->enable_write_buffer_limit(false); + } + + if (queries_list.empty()) { + it = _paused_queries_list.erase(it); + continue; + } else { + // Finished deal with one workload group, and should deal with next one. + ++it; + } + } +} + +// Return the expected free bytes if memtable could flush +int64_t WorkloadGroupMgr::flush_memtable_from_current_group_(WorkloadGroupPtr wg, + int64_t need_free_mem) { + // If there are a lot of memtable memory, then wait them flush finished. + MemTableMemoryLimiter* memtable_limiter = + doris::ExecEnv::GetInstance()->memtable_memory_limiter(); + int64_t memtable_active_bytes = 0; + int64_t memtable_queue_bytes = 0; + int64_t memtable_flush_bytes = 0; + memtable_limiter->get_workload_group_memtable_usage( + wg->id(), &memtable_active_bytes, &memtable_queue_bytes, &memtable_flush_bytes); + // TODO: should add a signal in memtable limiter to prevent new batch + // For example, streamload, it will not reserve many memory, but it will occupy many memtable memory. + // TODO: 0.2 should be a workload group properties. For example, the group is optimized for load,then the value + // should be larged, if the group is optimized for query, then the value should be smaller. + int64_t max_wg_memtable_bytes = wg->write_buffer_limit(); + if (memtable_active_bytes + memtable_queue_bytes + memtable_flush_bytes > + max_wg_memtable_bytes) { + // There are many table in flush queue, just waiting them flush finished. + if (memtable_active_bytes < (int64_t)(max_wg_memtable_bytes * 0.6)) { + LOG_EVERY_T(INFO, 60) << wg->name() + << " load memtable size is: " << memtable_active_bytes << ", " + << memtable_queue_bytes << ", " << memtable_flush_bytes + << ", load buffer limit is: " << max_wg_memtable_bytes + << " wait for flush finished to release more memory"; + return memtable_queue_bytes + memtable_flush_bytes; + } else { + // Flush some memtables(currently written) to flush queue. + memtable_limiter->flush_workload_group_memtables( + wg->id(), memtable_active_bytes - (int64_t)(max_wg_memtable_bytes * 0.6)); + LOG_EVERY_T(INFO, 60) << wg->name() + << " load memtable size is: " << memtable_active_bytes << ", " + << memtable_queue_bytes << ", " << memtable_flush_bytes + << ", flush some active memtable to revoke memory"; + return memtable_queue_bytes + memtable_flush_bytes + memtable_active_bytes - + (int64_t)(max_wg_memtable_bytes * 0.6); + } + } + return 0; +} + +int64_t WorkloadGroupMgr::revoke_memory_from_other_group_(std::shared_ptr requestor, + bool hard_limit, int64_t need_free_mem) { + int64_t total_freed_mem = 0; + std::unique_ptr profile = std::make_unique("RevokeMemory"); + // 1. memtable like memory + // 2. query exceed workload group limit + int64_t freed_mem = revoke_overcommited_memory_(requestor, need_free_mem, profile.get()); + total_freed_mem += freed_mem; + // The revoke process may kill current requestor, so should return now. + if (need_free_mem - total_freed_mem < 0 || requestor->is_cancelled()) { + return total_freed_mem; + } + if (hard_limit) { + freed_mem = cancel_top_query_in_overcommit_group_(need_free_mem - total_freed_mem, + doris::QUERY_MIN_MEMORY, profile.get()); + } else { + freed_mem = cancel_top_query_in_overcommit_group_( + need_free_mem - total_freed_mem, requestor->get_mem_tracker()->consumption(), + profile.get()); + } + total_freed_mem += freed_mem; + // The revoke process may kill current requestor, so should return now. + if (need_free_mem - total_freed_mem < 0 || requestor->is_cancelled()) { + return total_freed_mem; + } + return total_freed_mem; +} + +// Revoke memory from workload group that exceed it's limit. For example, if the wg's limit is 10g, but used 12g +// then should revoke 2g from the group. +int64_t WorkloadGroupMgr::revoke_overcommited_memory_(std::shared_ptr requestor, + int64_t need_free_mem, + RuntimeProfile* profile) { + int64_t total_freed_mem = 0; + // 1. check memtable usage, and try to free them. + int64_t freed_mem = revoke_memtable_from_overcommited_groups_(need_free_mem, profile); + total_freed_mem += freed_mem; + // The revoke process may kill current requestor, so should return now. + if (need_free_mem - total_freed_mem < 0 || requestor->is_cancelled()) { + return total_freed_mem; + } + // 2. Cancel top usage query, one by one + using WorkloadGroupMem = std::pair; + auto cmp = [](WorkloadGroupMem left, WorkloadGroupMem right) { + return left.second < right.second; + }; + std::priority_queue, decltype(cmp)> heap(cmp); + { + std::shared_lock r_lock(_group_mutex); + for (auto iter = _workload_groups.begin(); iter != _workload_groups.end(); iter++) { + if (requestor->workload_group() != nullptr && + iter->second->id() == requestor->workload_group()->id()) { + continue; + } + heap.emplace(iter->second, iter->second->memory_used()); + } + } + while (!heap.empty() && need_free_mem - total_freed_mem > 0 && !requestor->is_cancelled()) { + auto [wg, sort_mem] = heap.top(); + heap.pop(); + freed_mem = wg->free_overcommited_memory(need_free_mem - total_freed_mem, profile); + total_freed_mem += freed_mem; + } + return total_freed_mem; +} + +// If the memtable is too large, then flush them and wait for finished. +int64_t WorkloadGroupMgr::revoke_memtable_from_overcommited_groups_(int64_t need_free_mem, + RuntimeProfile* profile) { + return 0; +} + +// 1. Sort all memory limiter in all overcommit wg, and cancel the top usage task that with most memory. +// 2. Maybe not valid because it's memory not exceed limit. +int64_t WorkloadGroupMgr::cancel_top_query_in_overcommit_group_(int64_t need_free_mem, + int64_t lower_bound, + RuntimeProfile* profile) { + return 0; +} + +// streamload, kafka routine load, group commit +// insert into select +// select + +// If the query could release some memory, for example, spill disk, then the return value is true. +// If the query could not release memory, then cancel the query, the return value is true. +// If the query is not ready to do these tasks, it means just wait, then return value is false. +bool WorkloadGroupMgr::handle_single_query_(const std::shared_ptr& query_ctx, + size_t size_to_reserve, int64_t time_in_queue, + Status paused_reason) { + size_t revocable_size = 0; + size_t memory_usage = 0; + bool has_running_task = false; + const auto query_id = print_id(query_ctx->query_id()); + query_ctx->get_revocable_info(&revocable_size, &memory_usage, &has_running_task); + if (has_running_task) { + LOG(INFO) << "Query: " << print_id(query_ctx->query_id()) + << " is paused, but still has running task, skip it."; + return false; + } + + const auto wg = query_ctx->workload_group(); + auto revocable_tasks = query_ctx->get_revocable_tasks(); + if (revocable_tasks.empty()) { + const auto limit = query_ctx->get_mem_limit(); + const auto reserved_size = query_ctx->query_mem_tracker->reserved_consumption(); + if (paused_reason.is()) { + // During waiting time, another operator in the query may finished and release + // many memory and we could run. + if ((memory_usage + size_to_reserve) < limit) { + LOG(INFO) << "Query: " << query_id << ", usage(" + << PrettyPrinter::print_bytes(memory_usage) << " + " << size_to_reserve + << ") less than limit(" << PrettyPrinter::print_bytes(limit) + << "), resume it."; + query_ctx->set_memory_sufficient(true); + return true; + } else if (time_in_queue >= config::spill_in_paused_queue_timeout_ms) { + // Use MEM_LIMIT_EXCEEDED so that FE could parse the error code and do try logic + auto msg1 = fmt::format( + "Query {} failed beause query limit is exceeded, but could " + "not find memory that could release or spill to disk. Query memory usage: " + "{}, limit: {}, reserved " + "size: {}, try to reserve: {}, wg info: {}.", + query_id, PrettyPrinter::print_bytes(memory_usage), + PrettyPrinter::print_bytes(limit), + PrettyPrinter::print_bytes(reserved_size), + PrettyPrinter::print_bytes(size_to_reserve), wg->memory_debug_string()); + LOG(INFO) << fmt::format("{}.\n{}", msg1, + doris::ProcessProfile::instance() + ->memory_profile() + ->process_memory_detail_str()); + query_ctx->cancel(doris::Status::Error(msg1)); + } else { + return false; + } + } else if (paused_reason.is()) { + if (!wg->exceed_limit()) { + LOG(INFO) << "Query: " << query_id + << " paused caused by WORKLOAD_GROUP_MEMORY_EXCEEDED, now resume it."; + query_ctx->set_memory_sufficient(true); + return true; + } else if (time_in_queue > config::spill_in_paused_queue_timeout_ms) { + auto msg1 = fmt::format( + "Query {} failed because workload group memory is exceeded" + ", and there is no cache now. And could not find task to spill. " + "Query memory usage: {}, limit: {}, reserved " + "size: {}, try to reserve: {}, wg info: {}." + " Maybe you should set the workload group's limit to a lower value.", + query_id, PrettyPrinter::print_bytes(memory_usage), + PrettyPrinter::print_bytes(limit), + PrettyPrinter::print_bytes(reserved_size), + PrettyPrinter::print_bytes(size_to_reserve), wg->memory_debug_string()); + LOG(INFO) << fmt::format("{}.\n{}", msg1, + doris::ProcessProfile::instance() + ->memory_profile() + ->process_memory_detail_str()); + query_ctx->cancel(doris::Status::Error(msg1)); + } else { + return false; + } + } else { + // Should not consider about process memory. For example, the query's limit is 100g, workload + // group's memlimit is 10g, process memory is 20g. The query reserve will always failed in wg + // limit, and process is always have memory, so that it will resume and failed reserve again. + const size_t test_memory_size = std::max(size_to_reserve, 32L * 1024 * 1024); + if (!GlobalMemoryArbitrator::is_exceed_soft_mem_limit(test_memory_size)) { + LOG(INFO) << "Query: " << query_id + << ", process limit not exceeded now, resume this query" + << ", process memory info: " + << GlobalMemoryArbitrator::process_memory_used_details_str() + << ", wg info: " << wg->debug_string(); + query_ctx->set_memory_sufficient(true); + return true; + } else if (time_in_queue > config::spill_in_paused_queue_timeout_ms) { + auto msg1 = fmt::format( + "Query {} failed because process memory is exceeded" + ", and there is no cache now. And could not find task to spill. " + "Query memory usage: {}, limit: {}, reserved " + "size: {}, try to reserve: {}, wg info: {}." + " Maybe you should set the workload group's limit to a lower value.", + query_id, PrettyPrinter::print_bytes(memory_usage), + PrettyPrinter::print_bytes(limit), + PrettyPrinter::print_bytes(reserved_size), + PrettyPrinter::print_bytes(size_to_reserve), wg->memory_debug_string()); + LOG(INFO) << fmt::format("{}.\n{}", msg1, + doris::ProcessProfile::instance() + ->memory_profile() + ->process_memory_detail_str()); + query_ctx->cancel(doris::Status::Error(msg1)); + } else { + return false; + } + } + } else { + SCOPED_ATTACH_TASK(query_ctx.get()); + auto status = query_ctx->revoke_memory(); + if (!status.ok()) { + ExecEnv::GetInstance()->fragment_mgr()->cancel_query(query_ctx->query_id(), status); + } + } + return true; +} + +void WorkloadGroupMgr::update_queries_limit_(WorkloadGroupPtr wg, bool enable_hard_limit) { + auto wg_mem_limit = wg->memory_limit(); + auto all_query_ctxs = wg->queries(); + bool is_low_watermark = false; + bool is_high_watermark = false; + wg->check_mem_used(&is_low_watermark, &is_high_watermark); + int64_t wg_high_water_mark_limit = + (int64_t)(wg_mem_limit * wg->memory_high_watermark() * 1.0 / 100); + int64_t memtable_usage = wg->write_buffer_size(); + int64_t wg_high_water_mark_except_load = wg_high_water_mark_limit; + if (memtable_usage > wg->write_buffer_limit()) { + wg_high_water_mark_except_load = wg_high_water_mark_limit - wg->write_buffer_limit(); + } else { + wg_high_water_mark_except_load = + wg_high_water_mark_limit - memtable_usage - 10 * 1024 * 1024; + } + std::string debug_msg; + if (is_high_watermark || is_low_watermark) { + debug_msg = fmt::format( + "\nWorkload Group {}: mem limit: {}, mem used: {}, " + "high water mark mem limit: {}, load memtable usage: {}, used ratio: {}", + wg->name(), PrettyPrinter::print(wg->memory_limit(), TUnit::BYTES), + PrettyPrinter::print(wg->total_mem_used(), TUnit::BYTES), + PrettyPrinter::print(wg_high_water_mark_limit, TUnit::BYTES), + PrettyPrinter::print(memtable_usage, TUnit::BYTES), + (double)(wg->total_mem_used()) / wg_mem_limit); + } + + // If reached low watermark and wg is not enable memory overcommit, then enable load buffer limit + if (is_low_watermark && !wg->enable_memory_overcommit()) { + wg->enable_write_buffer_limit(true); + } + // Both enable overcommit and not enable overcommit, if user set slot memory policy + // then we will replace the memtracker's memlimit with + if (wg->slot_memory_policy() == TWgSlotMemoryPolicy::NONE) { + return; + } + int32_t total_used_slot_count = 0; + int32_t total_slot_count = wg->total_query_slot_count(); + // calculate total used slot count + for (const auto& query : all_query_ctxs) { + auto query_ctx = query.second.lock(); + if (!query_ctx) { + continue; + } + // Streamload kafka load group commit, not modify slot + if (!query_ctx->is_pure_load_task()) { + total_used_slot_count += query_ctx->get_slot_count(); + } + } + // calculate per query weighted memory limit + debug_msg = "Query Memory Summary: \n"; + for (const auto& query : all_query_ctxs) { + auto query_ctx = query.second.lock(); + if (!query_ctx) { + continue; + } + int64_t query_weighted_mem_limit = 0; + int64_t expected_query_weighted_mem_limit = 0; + // If the query enable hard limit, then it should not use the soft limit + if (wg->slot_memory_policy() == TWgSlotMemoryPolicy::FIXED) { + if (total_slot_count < 1) { + LOG(WARNING) + << "Query " << print_id(query_ctx->query_id()) + << " enabled hard limit, but the slot count < 1, could not take affect"; + } else { + // If the query enable hard limit, then not use weighted info any more, just use the settings limit. + query_weighted_mem_limit = (int64_t)((wg_high_water_mark_except_load * + query_ctx->get_slot_count() * 1.0) / + total_slot_count); + expected_query_weighted_mem_limit = query_weighted_mem_limit; + } + } else { + // If low water mark is not reached, then use process memory limit as query memory limit. + // It means it will not take effect. + // If there are some query in paused list, then limit should take effect. + expected_query_weighted_mem_limit = + total_used_slot_count > 0 + ? (int64_t)((wg_high_water_mark_except_load + total_used_slot_count) * + query_ctx->get_slot_count() * 1.0 / total_used_slot_count) + : wg_high_water_mark_except_load; + if (!is_low_watermark && !enable_hard_limit) { + query_weighted_mem_limit = wg_high_water_mark_except_load; + } else { + query_weighted_mem_limit = expected_query_weighted_mem_limit; + } + } + debug_msg += query_ctx->debug_string() + "\n"; + // If the query is a pure load task, then should not modify its limit. Or it will reserve + // memory failed and we did not hanle it. + if (!query_ctx->is_pure_load_task()) { + // If slot memory policy is enabled, then overcommit is disabled. + query_ctx->get_mem_tracker()->set_overcommit(false); + query_ctx->set_mem_limit(query_weighted_mem_limit); + query_ctx->set_expected_mem_limit(expected_query_weighted_mem_limit); + } + } + LOG_EVERY_T(INFO, 60) << debug_msg; +} + void WorkloadGroupMgr::stop() { for (auto iter = _workload_groups.begin(); iter != _workload_groups.end(); iter++) { iter->second->try_stop_schedulers(); diff --git a/be/src/runtime/workload_group/workload_group_manager.h b/be/src/runtime/workload_group/workload_group_manager.h index 5d75a4558ef4f8..990150923d6a9f 100644 --- a/be/src/runtime/workload_group/workload_group_manager.h +++ b/be/src/runtime/workload_group/workload_group_manager.h @@ -29,6 +29,7 @@ class CgroupCpuCtl; namespace vectorized { class Block; +class QueryContext; } // namespace vectorized namespace pipeline { @@ -41,6 +42,34 @@ const static uint64_t INTERNAL_WORKLOAD_GROUP_ID = static_cast(TWorkloadType::type::INTERNAL); const static std::string INTERNAL_WORKLOAD_GROUP_NAME = "_internal"; +class PausedQuery { +public: + // Use weak ptr to save query ctx, to make sure if the query is cancelled + // the resource will be released + std::weak_ptr query_ctx_; + std::chrono::system_clock::time_point enqueue_at; + size_t last_mem_usage {0}; + double cache_ratio_ {0.0}; + bool any_wg_exceed_limit_ {false}; + int64_t reserve_size_ {0}; + + PausedQuery(std::shared_ptr query_ctx, double cache_ratio, + bool any_wg_exceed_limit, int64_t reserve_size); + + int64_t elapsed_time() const { + auto now = std::chrono::system_clock::now(); + return std::chrono::duration_cast(now - enqueue_at).count(); + } + + std::string query_id() const { return query_id_; } + + bool operator<(const PausedQuery& other) const { return query_id_ < other.query_id_; } + + bool operator==(const PausedQuery& other) const { return query_id_ == other.query_id_; } + +private: + std::string query_id_; +}; class WorkloadGroupMgr { public: WorkloadGroupMgr() = default; @@ -78,11 +107,34 @@ class WorkloadGroupMgr { void refresh_workload_group_metrics(); + void add_paused_query(const std::shared_ptr& query_ctx, int64_t reserve_size, + const Status& status); + + void handle_paused_queries(); + private: + int64_t cancel_top_query_in_overcommit_group_(int64_t need_free_mem, int64_t lower_bound, + RuntimeProfile* profile); + int64_t flush_memtable_from_current_group_(WorkloadGroupPtr wg, int64_t need_free_mem); + bool handle_single_query_(const std::shared_ptr& query_ctx, + size_t size_to_reserve, int64_t time_in_queue, Status paused_reason); + int64_t revoke_memory_from_other_group_(std::shared_ptr requestor, + bool hard_limit, int64_t need_free_mem); + int64_t revoke_overcommited_memory_(std::shared_ptr requestor, + int64_t need_free_mem, RuntimeProfile* profile); + int64_t revoke_memtable_from_overcommited_groups_(int64_t need_free_mem, + RuntimeProfile* profile); + void update_queries_limit_(WorkloadGroupPtr wg, bool enable_hard_limit); + std::shared_mutex _group_mutex; std::unordered_map _workload_groups; std::shared_mutex _clear_cgroup_lock; + + // Save per group paused query list, it should be a global structure, not per + // workload group, because we need do some coordinate work globally. + std::mutex _paused_queries_lock; + std::map> _paused_queries_list; }; } // namespace doris diff --git a/be/src/runtime/workload_management/workload_action.cpp b/be/src/runtime/workload_management/workload_action.cpp index 77042b074fd624..dc435876d1d090 100644 --- a/be/src/runtime/workload_management/workload_action.cpp +++ b/be/src/runtime/workload_management/workload_action.cpp @@ -23,7 +23,7 @@ namespace doris { void WorkloadActionCancelQuery::exec(WorkloadQueryInfo* query_info) { std::stringstream msg; - msg << "query " << query_info->query_id + msg << "Query " << query_info->query_id << " cancelled by workload policy: " << query_info->policy_name << ", id:" << query_info->policy_id << ", " << query_info->cond_eval_msg; std::string msg_str = msg.str(); diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index 729e5470c97ad5..9e6c44e9baf1b8 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -37,6 +37,7 @@ #include "olap/olap_common.h" #include "runtime/define_primitive_type.h" #include "vec/common/cow.h" +#include "vec/common/custom_allocator.h" #include "vec/common/pod_array_fwd.h" #include "vec/common/string_ref.h" #include "vec/common/typeid_cast.h" @@ -322,14 +323,23 @@ class IColumn : public COW { return 0; } - virtual void serialize_vec(std::vector& keys, size_t num_rows, - size_t max_row_byte_size) const { + void serialize_vec(std::vector& keys, size_t num_rows, + size_t max_row_byte_size) const { + serialize_vec(keys.data(), num_rows, max_row_byte_size); + } + + void serialize_vec_with_null_map(std::vector& keys, size_t num_rows, + const uint8_t* null_map) const { + serialize_vec_with_null_map(keys.data(), num_rows, null_map); + } + + virtual void serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Method serialize_vec is not supported for " + get_name()); __builtin_unreachable(); } - virtual void serialize_vec_with_null_map(std::vector& keys, size_t num_rows, + virtual void serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const uint8_t* null_map) const { throw doris::Exception( ErrorCode::NOT_IMPLEMENTED_ERROR, @@ -337,15 +347,24 @@ class IColumn : public COW { __builtin_unreachable(); } + void deserialize_vec(std::vector& keys, const size_t num_rows) { + deserialize_vec(keys.data(), num_rows); + } + + void deserialize_vec_with_null_map(std::vector& keys, const size_t num_rows, + const uint8_t* null_map) { + deserialize_vec_with_null_map(keys.data(), num_rows, null_map); + } + // This function deserializes group-by keys into column in the vectorized way. - virtual void deserialize_vec(std::vector& keys, const size_t num_rows) { + virtual void deserialize_vec(StringRef* keys, const size_t num_rows) { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "Method deserialize_vec is not supported for " + get_name()); __builtin_unreachable(); } // Used in ColumnNullable::deserialize_vec - virtual void deserialize_vec_with_null_map(std::vector& keys, const size_t num_rows, + virtual void deserialize_vec_with_null_map(StringRef* keys, const size_t num_rows, const uint8_t* null_map) { throw doris::Exception( ErrorCode::NOT_IMPLEMENTED_ERROR, diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index 80d50e26e16ead..f9281891700640 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -178,8 +178,7 @@ class ColumnConst final : public COWHelper { size_t get_max_row_byte_size() const override { return data->get_max_row_byte_size(); } - void serialize_vec(std::vector& keys, size_t num_rows, - size_t max_row_byte_size) const override { + void serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const override { data->serialize_vec(keys, num_rows, max_row_byte_size); } @@ -198,7 +197,7 @@ class ColumnConst final : public COWHelper { get_data_column_ptr()->update_crc_with_value(start, end, hash, nullptr); } - void serialize_vec_with_null_map(std::vector& keys, size_t num_rows, + void serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const uint8_t* null_map) const override { data->serialize_vec_with_null_map(keys, num_rows, null_map); } diff --git a/be/src/vec/columns/column_decimal.cpp b/be/src/vec/columns/column_decimal.cpp index cf0193b29e187e..3168be747c87f2 100644 --- a/be/src/vec/columns/column_decimal.cpp +++ b/be/src/vec/columns/column_decimal.cpp @@ -74,7 +74,7 @@ size_t ColumnDecimal::get_max_row_byte_size() const { } template -void ColumnDecimal::serialize_vec(std::vector& keys, size_t num_rows, +void ColumnDecimal::serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const { for (size_t i = 0; i < num_rows; ++i) { memcpy_fixed(const_cast(keys[i].data + keys[i].size), (char*)&data[i]); @@ -83,7 +83,7 @@ void ColumnDecimal::serialize_vec(std::vector& keys, size_t num_ro } template -void ColumnDecimal::serialize_vec_with_null_map(std::vector& keys, size_t num_rows, +void ColumnDecimal::serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const UInt8* null_map) const { DCHECK(null_map != nullptr); const bool has_null = simd::contain_byte(null_map, num_rows, 1); @@ -111,7 +111,7 @@ void ColumnDecimal::serialize_vec_with_null_map(std::vector& keys, } template -void ColumnDecimal::deserialize_vec(std::vector& keys, const size_t num_rows) { +void ColumnDecimal::deserialize_vec(StringRef* keys, const size_t num_rows) { for (size_t i = 0; i < num_rows; ++i) { keys[i].data = deserialize_and_insert_from_arena(keys[i].data); keys[i].size -= sizeof(T); @@ -119,8 +119,7 @@ void ColumnDecimal::deserialize_vec(std::vector& keys, const size_ } template -void ColumnDecimal::deserialize_vec_with_null_map(std::vector& keys, - const size_t num_rows, +void ColumnDecimal::deserialize_vec_with_null_map(StringRef* keys, const size_t num_rows, const uint8_t* null_map) { for (size_t i = 0; i < num_rows; ++i) { if (null_map[i] == 0) { diff --git a/be/src/vec/columns/column_decimal.h b/be/src/vec/columns/column_decimal.h index dde7a1c6237481..f4c6691b9c610c 100644 --- a/be/src/vec/columns/column_decimal.h +++ b/be/src/vec/columns/column_decimal.h @@ -163,15 +163,14 @@ class ColumnDecimal final : public COWHelper> { size_t get_max_row_byte_size() const override; - void serialize_vec(std::vector& keys, size_t num_rows, - size_t max_row_byte_size) const override; + void serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const override; - void serialize_vec_with_null_map(std::vector& keys, size_t num_rows, + void serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const uint8_t* null_map) const override; - void deserialize_vec(std::vector& keys, const size_t num_rows) override; + void deserialize_vec(StringRef* keys, const size_t num_rows) override; - void deserialize_vec_with_null_map(std::vector& keys, const size_t num_rows, + void deserialize_vec_with_null_map(StringRef* keys, const size_t num_rows, const uint8_t* null_map) override; void update_hash_with_value(size_t n, SipHash& hash) const override; diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp index c58c78f5611d02..2420bbc309b541 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -249,13 +249,13 @@ size_t ColumnNullable::get_max_row_byte_size() const { return flag_size + get_nested_column().get_max_row_byte_size(); } -void ColumnNullable::serialize_vec(std::vector& keys, size_t num_rows, +void ColumnNullable::serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const { const auto& arr = get_null_map_data(); get_nested_column().serialize_vec_with_null_map(keys, num_rows, arr.data()); } -void ColumnNullable::deserialize_vec(std::vector& keys, const size_t num_rows) { +void ColumnNullable::deserialize_vec(StringRef* keys, const size_t num_rows) { auto& arr = get_null_map_data(); const size_t old_size = arr.size(); arr.resize(old_size + num_rows); diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 801af78ca2b4c2..d1450c9060a62d 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -180,10 +180,10 @@ class ColumnNullable final : public COWHelper, public N StringRef serialize_value_into_arena(size_t n, Arena& arena, char const*& begin) const override; const char* deserialize_and_insert_from_arena(const char* pos) override; size_t get_max_row_byte_size() const override; - void serialize_vec(std::vector& keys, size_t num_rows, - size_t max_row_byte_size) const override; - void deserialize_vec(std::vector& keys, size_t num_rows) override; + void serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const override; + + void deserialize_vec(StringRef* keys, size_t num_rows) override; void insert_range_from(const IColumn& src, size_t start, size_t length) override; diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h index 037656508e2a7f..b87f7fd2c87561 100644 --- a/be/src/vec/columns/column_object.h +++ b/be/src/vec/columns/column_object.h @@ -492,22 +492,23 @@ class ColumnObject final : public COWHelper { "get_max_row_byte_size" + get_name()); } - void serialize_vec(std::vector& keys, size_t num_rows, - size_t max_row_byte_size) const override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "serialize_vec" + get_name()); + void serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const override { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "serialize_vec" + std::string(get_name())); } - void serialize_vec_with_null_map(std::vector& keys, size_t num_rows, + void serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const uint8_t* null_map) const override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "serialize_vec_with_null_map" + get_name()); } - void deserialize_vec(std::vector& keys, const size_t num_rows) override { - throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "deserialize_vec" + get_name()); + void deserialize_vec(StringRef* keys, const size_t num_rows) override { + throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, + "deserialize_vec" + std::string(get_name())); } - void deserialize_vec_with_null_map(std::vector& keys, const size_t num_rows, + void deserialize_vec_with_null_map(StringRef* keys, const size_t num_rows, const uint8_t* null_map) override { throw doris::Exception(ErrorCode::NOT_IMPLEMENTED_ERROR, "deserialize_vec_with_null_map" + get_name()); diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp index db0088e67c27b6..1173509cb9032d 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -439,8 +439,7 @@ size_t ColumnStr::get_max_row_byte_size() const { } template -void ColumnStr::serialize_vec(std::vector& keys, size_t num_rows, - size_t max_row_byte_size) const { +void ColumnStr::serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const { for (size_t i = 0; i < num_rows; ++i) { // Use uint32 instead of size_t to reduce agg key's length. auto offset(static_cast(offset_at(i))); @@ -454,7 +453,7 @@ void ColumnStr::serialize_vec(std::vector& keys, size_t num_rows, } template -void ColumnStr::serialize_vec_with_null_map(std::vector& keys, size_t num_rows, +void ColumnStr::serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const UInt8* null_map) const { DCHECK(null_map != nullptr); @@ -495,7 +494,7 @@ void ColumnStr::serialize_vec_with_null_map(std::vector& keys, siz } template -void ColumnStr::deserialize_vec(std::vector& keys, const size_t num_rows) { +void ColumnStr::deserialize_vec(StringRef* keys, const size_t num_rows) { for (size_t i = 0; i != num_rows; ++i) { auto original_ptr = keys[i].data; keys[i].data = deserialize_and_insert_from_arena(original_ptr); @@ -504,8 +503,8 @@ void ColumnStr::deserialize_vec(std::vector& keys, const size_t nu } template -void ColumnStr::deserialize_vec_with_null_map(std::vector& keys, - const size_t num_rows, const uint8_t* null_map) { +void ColumnStr::deserialize_vec_with_null_map(StringRef* keys, const size_t num_rows, + const uint8_t* null_map) { for (size_t i = 0; i != num_rows; ++i) { if (null_map[i] == 0) { auto original_ptr = keys[i].data; diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index 1674fd90933dbe..9a30b2030ad34a 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -373,17 +373,16 @@ class ColumnStr final : public COWHelper> { const char* deserialize_and_insert_from_arena(const char* pos) override; - void deserialize_vec(std::vector& keys, const size_t num_rows) override; + void deserialize_vec(StringRef* keys, const size_t num_rows) override; size_t get_max_row_byte_size() const override; - void serialize_vec(std::vector& keys, size_t num_rows, - size_t max_row_byte_size) const override; + void serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const override; - void serialize_vec_with_null_map(std::vector& keys, size_t num_rows, + void serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const uint8_t* null_map) const override; - void deserialize_vec_with_null_map(std::vector& keys, const size_t num_rows, + void deserialize_vec_with_null_map(StringRef* keys, const size_t num_rows, const uint8_t* null_map) override; void update_xxHash_with_value(size_t start, size_t end, uint64_t& hash, diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp index 7a23156063d4e1..6a9d33cc7e6816 100644 --- a/be/src/vec/columns/column_vector.cpp +++ b/be/src/vec/columns/column_vector.cpp @@ -64,7 +64,7 @@ size_t ColumnVector::get_max_row_byte_size() const { } template -void ColumnVector::serialize_vec(std::vector& keys, size_t num_rows, +void ColumnVector::serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const { for (size_t i = 0; i < num_rows; ++i) { memcpy_fixed(const_cast(keys[i].data + keys[i].size), (char*)&data[i]); @@ -73,7 +73,7 @@ void ColumnVector::serialize_vec(std::vector& keys, size_t num_row } template -void ColumnVector::serialize_vec_with_null_map(std::vector& keys, size_t num_rows, +void ColumnVector::serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const UInt8* null_map) const { DCHECK(null_map != nullptr); @@ -103,7 +103,7 @@ void ColumnVector::serialize_vec_with_null_map(std::vector& keys, } template -void ColumnVector::deserialize_vec(std::vector& keys, const size_t num_rows) { +void ColumnVector::deserialize_vec(StringRef* keys, const size_t num_rows) { for (size_t i = 0; i != num_rows; ++i) { keys[i].data = deserialize_and_insert_from_arena(keys[i].data); keys[i].size -= sizeof(T); @@ -111,8 +111,7 @@ void ColumnVector::deserialize_vec(std::vector& keys, const size_t } template -void ColumnVector::deserialize_vec_with_null_map(std::vector& keys, - const size_t num_rows, +void ColumnVector::deserialize_vec_with_null_map(StringRef* keys, const size_t num_rows, const uint8_t* null_map) { for (size_t i = 0; i < num_rows; ++i) { if (null_map[i] == 0) { diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index c14361c6572245..e8d895f5682914 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -247,17 +247,16 @@ class ColumnVector final : public COWHelper> { const char* deserialize_and_insert_from_arena(const char* pos) override; - void deserialize_vec(std::vector& keys, const size_t num_rows) override; + void deserialize_vec(StringRef* keys, const size_t num_rows) override; - void deserialize_vec_with_null_map(std::vector& keys, const size_t num_rows, + void deserialize_vec_with_null_map(StringRef* keys, const size_t num_rows, const uint8_t* null_map) override; size_t get_max_row_byte_size() const override; - void serialize_vec(std::vector& keys, size_t num_rows, - size_t max_row_byte_size) const override; + void serialize_vec(StringRef* keys, size_t num_rows, size_t max_row_byte_size) const override; - void serialize_vec_with_null_map(std::vector& keys, size_t num_rows, + void serialize_vec_with_null_map(StringRef* keys, size_t num_rows, const uint8_t* null_map) const override; void update_xxHash_with_value(size_t start, size_t end, uint64_t& hash, diff --git a/be/src/vec/common/allocator.cpp b/be/src/vec/common/allocator.cpp index e407dbbaab4afa..42570591e5a0c7 100644 --- a/be/src/vec/common/allocator.cpp +++ b/be/src/vec/common/allocator.cpp @@ -36,6 +36,7 @@ #include "runtime/process_profile.h" #include "runtime/thread_context.h" #include "util/mem_info.h" +#include "util/pretty_printer.h" #include "util/stack_util.h" #include "util/uid_util.h" @@ -87,8 +88,10 @@ void Allocator::sys_mem "Allocator sys memory check failed: Cannot alloc:{}, consuming " "tracker:<{}>, peak used {}, current used {}, exec node:<{}>, {}.", size, doris::thread_context()->thread_mem_tracker()->label(), - doris::thread_context()->thread_mem_tracker()->peak_consumption(), - doris::thread_context()->thread_mem_tracker()->consumption(), + doris::PrettyPrinter::print_bytes( + doris::thread_context()->thread_mem_tracker()->peak_consumption()), + doris::PrettyPrinter::print_bytes( + doris::thread_context()->thread_mem_tracker()->consumption()), doris::thread_context()->thread_mem_tracker_mgr->last_consumer_tracker_label(), doris::GlobalMemoryArbitrator::process_limit_exceeded_errmsg_str()); diff --git a/be/src/vec/common/columns_hashing.h b/be/src/vec/common/columns_hashing.h index f0a365cfc09f37..60c2370bc6086a 100644 --- a/be/src/vec/common/columns_hashing.h +++ b/be/src/vec/common/columns_hashing.h @@ -152,8 +152,12 @@ struct HashMethodSingleLowNullableColumn : public SingleColumnMethod { template ALWAYS_INLINE FindResult find_key_with_hash(Data& data, size_t i, Key key, size_t hash_value) { - if (key_column->is_null_at(i) && data.has_null_key_data()) { - return FindResult {&data.template get_null_key_data(), true}; + if (key_column->is_null_at(i)) { + if (data.has_null_key_data()) { + return FindResult {&data.template get_null_key_data(), true}; + } else { + return FindResult {nullptr, false}; + } } return Base::find_key_impl(key, hash_value, data); } diff --git a/be/src/vec/common/custom_allocator.h b/be/src/vec/common/custom_allocator.h index eee800a059d329..6361a60689c0b3 100644 --- a/be/src/vec/common/custom_allocator.h +++ b/be/src/vec/common/custom_allocator.h @@ -20,7 +20,7 @@ #include "vec/common/allocator.h" #include "vec/common/allocator_fwd.h" -template > +template > class CustomStdAllocator; template diff --git a/be/src/vec/common/hash_table/hash_map_context.h b/be/src/vec/common/hash_table/hash_map_context.h index 875c035b425dab..7f25b2511988ff 100644 --- a/be/src/vec/common/hash_table/hash_map_context.h +++ b/be/src/vec/common/hash_table/hash_map_context.h @@ -51,10 +51,10 @@ struct MethodBaseInner { bool inited_iterator = false; Key* keys = nullptr; Arena arena; - std::vector hash_values; + DorisVector hash_values; // use in join case - std::vector bucket_nums; + DorisVector bucket_nums; MethodBaseInner() { hash_table.reset(new HashMap()); } virtual ~MethodBaseInner() = default; @@ -68,6 +68,10 @@ struct MethodBaseInner { const uint8_t* null_map = nullptr, bool is_join = false, bool is_build = false, uint32_t bucket_size = 0) = 0; + [[nodiscard]] virtual size_t estimated_size(const ColumnRawPtrs& key_columns, size_t num_rows, + bool is_join = false, bool is_build = false, + uint32_t bucket_size = 0) = 0; + virtual size_t serialized_keys_size(bool is_build) const { return 0; } void init_join_bucket_num(uint32_t num_rows, uint32_t bucket_size, const uint8_t* null_map) { @@ -193,10 +197,10 @@ struct MethodSerialized : public MethodBase { using State = ColumnsHashing::HashMethodSerialized; using Base::try_presis_key; // need keep until the hash probe end. - std::vector build_stored_keys; + DorisVector build_stored_keys; Arena build_arena; // refresh each time probe - std::vector stored_keys; + DorisVector stored_keys; StringRef serialize_keys_to_pool_contiguous(size_t i, size_t keys_size, const ColumnRawPtrs& key_columns, Arena& pool) { @@ -210,8 +214,24 @@ struct MethodSerialized : public MethodBase { return {begin, sum_size}; } + size_t estimated_size(const ColumnRawPtrs& key_columns, size_t num_rows, bool is_join, + bool is_build, uint32_t bucket_size) override { + size_t size = 0; + for (const auto& column : key_columns) { + size += column->byte_size(); + } + + size += sizeof(StringRef) * num_rows; // stored_keys + if (is_join) { + size += sizeof(uint32_t) * num_rows; // bucket_nums + } else { + size += sizeof(size_t) * num_rows; // hash_values + } + return size; + } + void init_serialized_keys_impl(const ColumnRawPtrs& key_columns, size_t num_rows, - std::vector& input_keys, Arena& input_arena) { + DorisVector& input_keys, Arena& input_arena) { input_arena.clear(); input_keys.resize(num_rows); @@ -238,14 +258,18 @@ struct MethodSerialized : public MethodBase { } for (const auto& column : key_columns) { - column->serialize_vec(input_keys, num_rows, max_one_row_byte_size); + column->serialize_vec(input_keys.data(), num_rows, max_one_row_byte_size); } } Base::keys = input_keys.data(); } size_t serialized_keys_size(bool is_build) const override { - return is_build ? build_arena.size() : Base::arena.size(); + if (is_build) { + return build_stored_keys.size() * sizeof(StringRef) + build_arena.size(); + } else { + return stored_keys.size() * sizeof(StringRef) + Base::arena.size(); + } } void init_serialized_keys(const ColumnRawPtrs& key_columns, size_t num_rows, @@ -281,23 +305,35 @@ struct MethodStringNoCache : public MethodBase { ColumnsHashing::HashMethodString; // need keep until the hash probe end. - std::vector _build_stored_keys; + DorisVector _build_stored_keys; // refresh each time probe - std::vector _stored_keys; + DorisVector _stored_keys; size_t serialized_keys_size(bool is_build) const override { return is_build ? (_build_stored_keys.size() * sizeof(StringRef)) : (_stored_keys.size() * sizeof(StringRef)); } + size_t estimated_size(const ColumnRawPtrs& key_columns, size_t num_rows, bool is_join, + bool is_build, uint32_t bucket_size) override { + size_t size = 0; + size += sizeof(StringRef) * num_rows; // stored_keys + if (is_join) { + size += sizeof(uint32_t) * num_rows; // bucket_nums + } else { + size += sizeof(size_t) * num_rows; // hash_values + } + return size; + } + void init_serialized_keys_impl(const ColumnRawPtrs& key_columns, size_t num_rows, - std::vector& stored_keys) { + DorisVector& stored_keys) { const IColumn& column = *key_columns[0]; const auto& nested_column = column.is_nullable() ? assert_cast(column).get_nested_column() : column; - auto serialized_str = [](const auto& column_string, std::vector& stored_keys) { + auto serialized_str = [](const auto& column_string, DorisVector& stored_keys) { const auto& offsets = column_string.get_offsets(); const auto* chars = column_string.get_chars().data(); stored_keys.resize(column_string.size()); @@ -345,6 +381,17 @@ struct MethodOneNumber : public MethodBase { using State = ColumnsHashing::HashMethodOneNumber; + size_t estimated_size(const ColumnRawPtrs& key_columns, size_t num_rows, bool is_join, + bool is_build, uint32_t bucket_size) override { + size_t size = 0; + if (is_join) { + size += sizeof(uint32_t) * num_rows; // bucket_nums + } else { + size += sizeof(size_t) * num_rows; // hash_values + } + return size; + } + void init_serialized_keys(const ColumnRawPtrs& key_columns, size_t num_rows, const uint8_t* null_map = nullptr, bool is_join = false, bool is_build = false, uint32_t bucket_size = 0) override { @@ -382,16 +429,16 @@ struct MethodKeysFixed : public MethodBase { using State = ColumnsHashing::HashMethodKeysFixed; // need keep until the hash probe end. use only in join - std::vector build_stored_keys; + DorisVector build_stored_keys; // refresh each time probe hash table - std::vector stored_keys; + DorisVector stored_keys; Sizes key_sizes; MethodKeysFixed(Sizes key_sizes_) : key_sizes(std::move(key_sizes_)) {} template void pack_fixeds(size_t row_numbers, const ColumnRawPtrs& key_columns, - const ColumnRawPtrs& nullmap_columns, std::vector& result) { + const ColumnRawPtrs& nullmap_columns, DorisVector& result) { size_t bitmap_size = get_bitmap_size(nullmap_columns.size()); // set size to 0 at first, then use resize to call default constructor on index included from [0, row_numbers) to reset all memory result.clear(); @@ -458,6 +505,19 @@ struct MethodKeysFixed : public MethodBase { return (is_build ? build_stored_keys.size() : stored_keys.size()) * sizeof(typename Base::Key); } + + size_t estimated_size(const ColumnRawPtrs& key_columns, size_t num_rows, bool is_join, + bool is_build, uint32_t bucket_size) override { + size_t size = 0; + size += sizeof(StringRef) * num_rows; // stored_keys + if (is_join) { + size += sizeof(uint32_t) * num_rows; // bucket_nums + } else { + size += sizeof(size_t) * num_rows; // hash_values + } + return size; + } + void init_serialized_keys(const ColumnRawPtrs& key_columns, size_t num_rows, const uint8_t* null_map = nullptr, bool is_join = false, bool is_build = false, uint32_t bucket_size = 0) override { diff --git a/be/src/vec/common/hash_table/hash_table.h b/be/src/vec/common/hash_table/hash_table.h index 809868e2beea86..ec3d5ce9768ddc 100644 --- a/be/src/vec/common/hash_table/hash_table.h +++ b/be/src/vec/common/hash_table/hash_table.h @@ -785,6 +785,18 @@ class HashTable : private boost::noncopyable, } } + size_t estimate_memory(size_t num_elem) const { + if (!add_elem_size_overflow(num_elem)) { + return 0; + } + + auto new_size = num_elem + grower.buf_size(); + Grower new_grower = grower; + new_grower.set(new_size); + + return new_grower.buf_size() * sizeof(Cell); + } + /// Insert a value. In the case of any more complex values, it is better to use the `emplace` function. std::pair ALWAYS_INLINE insert(const value_type& x) { std::pair res; diff --git a/be/src/vec/common/hash_table/join_hash_table.h b/be/src/vec/common/hash_table/join_hash_table.h index ab501d67698dbe..faccb4136d36be 100644 --- a/be/src/vec/common/hash_table/join_hash_table.h +++ b/be/src/vec/common/hash_table/join_hash_table.h @@ -24,6 +24,7 @@ #include "common/exception.h" #include "common/status.h" #include "vec/columns/column_filter_helper.h" +#include "vec/common/custom_allocator.h" #include "vec/common/hash_table/hash.h" #include "vec/common/hash_table/hash_table_allocator.h" @@ -70,7 +71,7 @@ class JoinHashTable { size_t size() const { return next.size(); } - std::vector& get_visited() { return visited; } + DorisVector& get_visited() { return visited; } bool empty_build_side() const { return _empty_build_side; } @@ -209,7 +210,7 @@ class JoinHashTable { bool keep_null_key() { return _keep_null_key; } - void pre_build_idxs(std::vector& buckets) const { + void pre_build_idxs(DorisVector& buckets) const { for (unsigned int& bucket : buckets) { bucket = first[bucket]; } @@ -469,13 +470,13 @@ class JoinHashTable { } const Key* __restrict build_keys; - std::vector visited; + DorisVector visited; uint32_t bucket_size = 1; int max_batch_size = 4064; - std::vector first = {0}; - std::vector next = {0}; + DorisVector first = {0}; + DorisVector next = {0}; // use in iter hash map mutable uint32_t iter_idx = 1; diff --git a/be/src/vec/common/hash_table/ph_hash_map.h b/be/src/vec/common/hash_table/ph_hash_map.h index 414624c6e1a1f8..4404cd784de86e 100644 --- a/be/src/vec/common/hash_table/ph_hash_map.h +++ b/be/src/vec/common/hash_table/ph_hash_map.h @@ -195,6 +195,15 @@ class PHHashMap : private boost::noncopyable { return (_hash_map.size() + row) > (capacity * 7 / 8); } + size_t estimate_memory(size_t num_elem) const { + if (!add_elem_size_overflow(num_elem)) { + return 0; + } + + auto new_size = _hash_map.capacity() * 2 + 1; + return new_size * sizeof(typename HashMapImpl::slot_type); + } + size_t size() const { return _hash_map.size(); } template char* get_null_key_data() { diff --git a/be/src/vec/common/hash_table/string_hash_table.h b/be/src/vec/common/hash_table/string_hash_table.h index 892598a83263b9..4bc3d0d0e46375 100644 --- a/be/src/vec/common/hash_table/string_hash_table.h +++ b/be/src/vec/common/hash_table/string_hash_table.h @@ -678,4 +678,30 @@ class StringHashTable : private boost::noncopyable { m3.add_elem_size_overflow(add_size) || m4.add_elem_size_overflow(add_size) || ms.add_elem_size_overflow(add_size); } + + size_t estimate_memory(size_t num_elem) const { + size_t estimate_size = 0; + + if (m1.add_elem_size_overflow(num_elem)) { + estimate_size = m1.estimate_memory(num_elem); + } + + if (m2.add_elem_size_overflow(num_elem)) { + estimate_size = std::max(estimate_size, m2.estimate_memory(num_elem)); + } + + if (m3.add_elem_size_overflow(num_elem)) { + estimate_size = std::max(estimate_size, m3.estimate_memory(num_elem)); + } + + if (m4.add_elem_size_overflow(num_elem)) { + estimate_size = std::max(estimate_size, m4.estimate_memory(num_elem)); + } + + if (ms.add_elem_size_overflow(num_elem)) { + estimate_size = std::max(estimate_size, ms.estimate_memory(num_elem)); + } + + return estimate_size; + } }; diff --git a/be/src/vec/common/sort/sorter.cpp b/be/src/vec/common/sort/sorter.cpp index 4f7de1d379aea9..08ee23e1bbf9bb 100644 --- a/be/src/vec/common/sort/sorter.cpp +++ b/be/src/vec/common/sort/sorter.cpp @@ -204,6 +204,38 @@ FullSorter::FullSorter(VSortExecExprs& vsort_exec_exprs, int limit, int64_t offs : Sorter(vsort_exec_exprs, limit, offset, pool, is_asc_order, nulls_first), _state(MergeSorterState::create_unique(row_desc, offset, limit, state, profile)) {} +size_t FullSorter::get_reserve_mem_size(RuntimeState* state, bool eos) const { + size_t size_to_reserve = 0; + const auto rows = _state->unsorted_block_->rows(); + if (rows != 0) { + const auto bytes = _state->unsorted_block_->bytes(); + const auto allocated_bytes = _state->unsorted_block_->allocated_bytes(); + const auto bytes_per_row = bytes / rows; + const auto estimated_size_of_next_block = bytes_per_row * state->batch_size(); + auto new_block_bytes = estimated_size_of_next_block + bytes; + auto new_rows = rows + state->batch_size(); + // If the new size is greater than 85% of allocalted bytes, it maybe need to realloc. + if ((new_block_bytes * 100 / allocated_bytes) >= 85) { + size_to_reserve += (size_t)(allocated_bytes * 1.15); + } + auto sort = new_rows > buffered_block_size_ || new_block_bytes > buffered_block_bytes_; + if (sort) { + // new column is created when doing sort, reserve average size of one column + // for estimation + size_to_reserve += new_block_bytes / _state->unsorted_block_->columns(); + + // helping data structures used during sorting + size_to_reserve += new_rows * sizeof(IColumn::Permutation::value_type); + + auto sort_columns_count = _vsort_exec_exprs.lhs_ordering_expr_ctxs().size(); + if (1 != sort_columns_count) { + size_to_reserve += new_rows * sizeof(EqualRangeIterator); + } + } + } + return size_to_reserve; +} + Status FullSorter::append_block(Block* block) { DCHECK(block->rows() > 0); diff --git a/be/src/vec/common/sort/sorter.h b/be/src/vec/common/sort/sorter.h index 36c535c9101db9..89dddd1fc5faa9 100644 --- a/be/src/vec/common/sort/sorter.h +++ b/be/src/vec/common/sort/sorter.h @@ -121,6 +121,8 @@ class Sorter { virtual size_t data_size() const = 0; + virtual size_t get_reserve_mem_size(RuntimeState* state, bool eos) const { return 0; } + // for topn runtime predicate const SortDescription& get_sort_description() const { return _sort_description; } virtual Field get_top_value() { return Field {Field::Types::Null}; } @@ -171,6 +173,8 @@ class FullSorter final : public Sorter { size_t data_size() const override; + size_t get_reserve_mem_size(RuntimeState* state, bool eos) const override; + Status merge_sort_read_for_spill(RuntimeState* state, doris::vectorized::Block* block, int batch_size, bool* eos) override; void reset() override; @@ -186,7 +190,11 @@ class FullSorter final : public Sorter { static constexpr size_t INITIAL_BUFFERED_BLOCK_BYTES = 64 * 1024 * 1024; - size_t buffered_block_bytes_ = INITIAL_BUFFERED_BLOCK_BYTES; + static constexpr size_t SPILL_BUFFERED_BLOCK_SIZE = 4 * 1024 * 1024; + static constexpr size_t SPILL_BUFFERED_BLOCK_BYTES = 256 << 20; + + size_t buffered_block_size_ = SPILL_BUFFERED_BLOCK_SIZE; + size_t buffered_block_bytes_ = SPILL_BUFFERED_BLOCK_BYTES; }; } // namespace doris::vectorized diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index 951c2661faf172..29442c8ecac90a 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -734,7 +734,9 @@ void Block::clear_column_data(int64_t column_size) noexcept { } for (auto& d : data) { if (d.column) { - DCHECK_EQ(d.column->use_count(), 1) << " " << print_use_count(); + // Temporarily disable reference count check because a column might be referenced multiple times within a block. + // Queries like this: `select c, c from t1;` + // DCHECK_EQ(d.column->use_count(), 1) << " " << print_use_count(); (*std::move(d.column)).assume_mutable()->clear(); } } diff --git a/be/src/vec/core/block.h b/be/src/vec/core/block.h index 2242db3f9058c2..1941d1d2f37e79 100644 --- a/be/src/vec/core/block.h +++ b/be/src/vec/core/block.h @@ -550,9 +550,11 @@ class MutableBlock { [[nodiscard]] Status merge_impl_ignore_overflow(T&& block) { if (_columns.size() != block.columns()) { return Status::Error( - "Merge block not match, self:[columns: {}, types: {}], input:[columns: {}, " + "Merge block not match, self column count: {}, [columns: {}, types: {}], " + "input column count: {}, [columns: {}, " "types: {}], ", - dump_names(), dump_types(), block.dump_names(), block.dump_types()); + _columns.size(), dump_names(), dump_types(), block.columns(), + block.dump_names(), block.dump_types()); } for (int i = 0; i < _columns.size(); ++i) { DCHECK(_data_types[i]->equals(*block.get_by_position(i).type)) @@ -585,9 +587,11 @@ class MutableBlock { } else { if (_columns.size() != block.columns()) { return Status::Error( - "Merge block not match, self:[columns: {}, types: {}], input:[columns: {}, " + "Merge block not match, self column count: {}, [columns: {}, types: {}], " + "input column count: {}, [columns: {}, " "types: {}], ", - dump_names(), dump_types(), block.dump_names(), block.dump_types()); + _columns.size(), dump_names(), dump_types(), block.columns(), + block.dump_names(), block.dump_types()); } for (int i = 0; i < _columns.size(); ++i) { if (!_data_types[i]->equals(*block.get_by_position(i).type)) { diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp index 688204bdff8ca7..5f0aeaae67b51e 100644 --- a/be/src/vec/exec/scan/scanner_context.cpp +++ b/be/src/vec/exec/scan/scanner_context.cpp @@ -248,7 +248,9 @@ vectorized::BlockUPtr ScannerContext::get_free_block(bool force) { } void ScannerContext::return_free_block(vectorized::BlockUPtr block) { - if (block->mem_reuse() && _block_memory_usage < _max_bytes_in_queue) { + // If under low memory mode, should not return the freeblock, it will occupy too much memory. + if (!_local_state->low_memory_mode() && block->mem_reuse() && + _block_memory_usage < _max_bytes_in_queue) { size_t block_size_to_reuse = block->allocated_bytes(); _block_memory_usage += block_size_to_reuse; _scanner_memory_used_counter->set(_block_memory_usage); @@ -259,6 +261,14 @@ void ScannerContext::return_free_block(vectorized::BlockUPtr block) { } } +void ScannerContext::clear_free_blocks() { + vectorized::BlockUPtr block; + while (_free_blocks.try_dequeue(block)) { + // do nothing + } + block.reset(); +} + Status ScannerContext::submit_scan_task(std::shared_ptr scan_task) { _scanner_sched_counter->update(1); _num_scheduled_scanners++; @@ -337,37 +347,54 @@ Status ScannerContext::get_block_from_queue(RuntimeState* state, vectorized::Blo _num_finished_scanners++; std::weak_ptr next_scanner; // submit one of the remaining scanners - if (_scanners.try_dequeue(next_scanner)) { - auto submit_status = submit_scan_task(std::make_shared(next_scanner)); - if (!submit_status.ok()) { - _process_status = submit_status; - _set_scanner_done(); - return _process_status; - } - } else { - // no more scanner to be scheduled - // `_free_blocks` serve all running scanners, maybe it's too large for the remaining scanners - int free_blocks_for_each = _free_blocks.size_approx() / _num_running_scanners; + // If under low memory mode, then there should be at most 4 scanner running + if (_num_running_scanners > low_memory_mode_scanners() && + _local_state->low_memory_mode()) { _num_running_scanners--; - for (int i = 0; i < free_blocks_for_each; ++i) { - vectorized::BlockUPtr removed_block; - if (_free_blocks.try_dequeue(removed_block)) { - _block_memory_usage -= block->allocated_bytes(); + } else { + if (_scanners.try_dequeue(next_scanner)) { + auto submit_status = + submit_scan_task(std::make_shared(next_scanner)); + if (!submit_status.ok()) { + _process_status = submit_status; + _set_scanner_done(); + return _process_status; + } + } else { + // no more scanner to be scheduled + // `_free_blocks` serve all running scanners, maybe it's too large for the remaining scanners + int free_blocks_for_each = + _free_blocks.size_approx() / _num_running_scanners; + _num_running_scanners--; + for (int i = 0; i < free_blocks_for_each; ++i) { + vectorized::BlockUPtr removed_block; + if (_free_blocks.try_dequeue(removed_block)) { + _block_memory_usage -= block->allocated_bytes(); + } } } } } else { - // resubmit current running scanner to read the next block - Status submit_status = submit_scan_task(scan_task); - if (!submit_status.ok()) { - _process_status = submit_status; - _set_scanner_done(); - return _process_status; + if (_local_state->low_memory_mode() && + _num_running_scanners > low_memory_mode_scanners()) { + _num_running_scanners--; + // push the scanner to the stack since it is not eos + _scanners.enqueue(scan_task->scanner); + } else { + // resubmit current running scanner to read the next block + Status submit_status = submit_scan_task(scan_task); + if (!submit_status.ok()) { + _process_status = submit_status; + _set_scanner_done(); + return _process_status; + } } } } - // scale up - RETURN_IF_ERROR(_try_to_scale_up()); + if (!_local_state->low_memory_mode()) { + // scale up + RETURN_IF_ERROR(_try_to_scale_up()); + } } if (_num_finished_scanners == _all_scanners.size() && _tasks_queue.empty()) { @@ -541,4 +568,8 @@ void ScannerContext::update_peak_running_scanner(int num) { _local_state->_peak_running_scanner->add(num); } +bool ScannerContext::low_memory_mode() const { + return _local_state->low_memory_mode(); +} + } // namespace doris::vectorized diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index d1cf06d56686ac..ed5224289bdadc 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -123,8 +123,11 @@ class ScannerContext : public std::enable_shared_from_this, vectorized::BlockUPtr get_free_block(bool force); void return_free_block(vectorized::BlockUPtr block); + void clear_free_blocks(); inline void inc_block_usage(size_t usage) { _block_memory_usage += usage; } + int64_t block_memory_usage() { return _block_memory_usage; } + // Caller should make sure the pipeline task is still running when calling this function void update_peak_running_scanner(int num); @@ -156,6 +159,19 @@ class ScannerContext : public std::enable_shared_from_this, int batch_size() const { return _batch_size; } + // During low memory mode, there will be at most 4 scanners running and every scanner will + // cache at most 1MB data. So that every instance will keep 8MB buffer. + bool low_memory_mode() const; + + // TODO(yiguolei) add this as session variable + int32_t low_memory_mode_scan_bytes_per_scanner() const { + return 1 * 1024 * 1024; // 1MB + } + + int32_t low_memory_mode_scanners() const { return 4; } + + pipeline::ScanLocalStateBase* local_state() const { return _local_state; } + // the unique id of this context std::string ctx_id; TUniqueId _query_id; @@ -200,7 +216,7 @@ class ScannerContext : public std::enable_shared_from_this, moodycamel::ConcurrentQueue> _scanners; int32_t _num_scheduled_scanners = 0; int32_t _num_finished_scanners = 0; - int32_t _num_running_scanners = 0; + std::atomic_int _num_running_scanners = 0; // weak pointer for _scanners, used in stop function std::vector> _all_scanners; std::shared_ptr _scanner_profile; diff --git a/be/src/vec/exec/scan/scanner_scheduler.cpp b/be/src/vec/exec/scan/scanner_scheduler.cpp index 1b14d172790671..cd7f6cfb0edc8d 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.cpp +++ b/be/src/vec/exec/scan/scanner_scheduler.cpp @@ -33,10 +33,12 @@ #include "common/logging.h" #include "common/status.h" #include "olap/tablet.h" +#include "pipeline/pipeline_task.h" #include "runtime/exec_env.h" #include "runtime/memory/mem_tracker.h" #include "runtime/runtime_state.h" #include "runtime/thread_context.h" +#include "runtime/workload_group/workload_group_manager.h" #include "util/async_io.h" // IWYU pragma: keep #include "util/blocking_queue.hpp" #include "util/cpu_info.h" @@ -183,6 +185,32 @@ std::unique_ptr ScannerScheduler::new_limited_scan_pool_token( return _limited_scan_thread_pool->new_token(mode, max_concurrency); } +void handle_reserve_memory_failure(RuntimeState* state, std::shared_ptr ctx, + const Status& st, size_t reserve_size) { + ctx->clear_free_blocks(); + auto* pipeline_task = state->get_task(); + auto* local_state = ctx->local_state(); + + pipeline_task->inc_memory_reserve_failed_times(); + auto debug_msg = fmt::format( + "Query: {} , scanner try to reserve: {}, operator name {}, " + "operator " + "id: {}, " + "task id: " + "{}, revocable mem size: {}, failed: {}", + print_id(state->query_id()), PrettyPrinter::print_bytes(reserve_size), + local_state->get_name(), local_state->parent()->node_id(), state->task_id(), + PrettyPrinter::print_bytes(pipeline_task->get_revocable_size()), st.to_string()); + // PROCESS_MEMORY_EXCEEDED error msg alread contains process_mem_log_str + if (!st.is()) { + debug_msg += fmt::format(", debug info: {}", GlobalMemoryArbitrator::process_mem_log_str()); + } + LOG(INFO) << debug_msg; + + ExecEnv::GetInstance()->workload_group_mgr()->add_paused_query( + state->get_query_ctx()->shared_from_this(), reserve_size, st); +} + void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, std::shared_ptr scan_task) { auto task_lock = ctx->task_exec_ctx(); @@ -220,6 +248,10 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, bool eos = false; ASSIGN_STATUS_IF_CATCH_EXCEPTION( RuntimeState* state = ctx->state(); DCHECK(nullptr != state); + // scanner->open may alloc plenty amount of memory(read blocks of data), + // so better to also check low memory and clear free blocks here. + if (ctx->low_memory_mode()) { ctx->clear_free_blocks(); } + if (!scanner->is_init()) { status = scanner->init(); if (!status.ok()) { @@ -242,8 +274,23 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, } size_t raw_bytes_threshold = config::doris_scanner_row_bytes; - size_t raw_bytes_read = 0; bool first_read = true; int64_t limit = scanner->limit(); - while (!eos && raw_bytes_read < raw_bytes_threshold) { + if (ctx->low_memory_mode()) { + ctx->clear_free_blocks(); + if (raw_bytes_threshold > ctx->low_memory_mode_scan_bytes_per_scanner()) { + raw_bytes_threshold = ctx->low_memory_mode_scan_bytes_per_scanner(); + } + } + + size_t raw_bytes_read = 0; + bool first_read = true; int64_t limit = scanner->limit(); + // If the first block is full, then it is true. Or the first block + second block > batch_size + bool has_first_full_block = false; + + // During low memory mode, every scan task will return at most 2 block to reduce memory usage. + while (!eos && raw_bytes_read < raw_bytes_threshold && + !(ctx->low_memory_mode() && has_first_full_block) && + !(has_first_full_block && + doris::thread_context()->thread_mem_tracker()->limit_exceeded())) { if (UNLIKELY(ctx->done())) { eos = true; break; @@ -252,7 +299,21 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, config::doris_scanner_max_run_time_ms * 1e6) { break; } - BlockUPtr free_block = ctx->get_free_block(first_read); + DEFER_RELEASE_RESERVED(); + BlockUPtr free_block; + if (first_read) { + free_block = ctx->get_free_block(first_read); + } else { + if (state->enable_reserve_memory()) { + size_t block_avg_bytes = scanner->get_block_avg_bytes(); + auto st = thread_context()->try_reserve_memory(block_avg_bytes); + if (!st.ok()) { + handle_reserve_memory_failure(state, ctx, st, block_avg_bytes); + break; + } + } + free_block = ctx->get_free_block(first_read); + } if (free_block == nullptr) { break; } @@ -287,9 +348,13 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, ctx->inc_block_usage(scan_task->cached_blocks.back().first->allocated_bytes() - block_size); } else { + if (!scan_task->cached_blocks.empty()) { + has_first_full_block = true; + } ctx->inc_block_usage(free_block->allocated_bytes()); scan_task->cached_blocks.emplace_back(std::move(free_block), free_block_bytes); } + if (limit > 0 && limit < ctx->batch_size()) { // If this scanner has limit, and less than batch size, // return immediately and no need to wait raw_bytes_threshold. @@ -301,6 +366,20 @@ void ScannerScheduler::_scanner_scan(std::shared_ptr ctx, // to avoid user specify a large limit and causing too much small blocks. break; } + + if (scan_task->cached_blocks.back().first->rows() > 0) { + auto block_avg_bytes = (scan_task->cached_blocks.back().first->bytes() + + scan_task->cached_blocks.back().first->rows() - 1) / + scan_task->cached_blocks.back().first->rows() * + ctx->batch_size(); + scanner->update_block_avg_bytes(block_avg_bytes); + } + if (ctx->low_memory_mode()) { + ctx->clear_free_blocks(); + if (raw_bytes_threshold > ctx->low_memory_mode_scan_bytes_per_scanner()) { + raw_bytes_threshold = ctx->low_memory_mode_scan_bytes_per_scanner(); + } + } } // end for while if (UNLIKELY(!status.ok())) { diff --git a/be/src/vec/exec/scan/vscanner.h b/be/src/vec/exec/scan/vscanner.h index bb68055e1f07a3..384850eb220b71 100644 --- a/be/src/vec/exec/scan/vscanner.h +++ b/be/src/vec/exec/scan/vscanner.h @@ -72,7 +72,10 @@ class VScanner { virtual Status init() { return Status::OK(); } // Not virtual, all child will call this method explictly virtual Status prepare(RuntimeState* state, const VExprContextSPtrs& conjuncts); - virtual Status open(RuntimeState* state) { return Status::OK(); } + virtual Status open(RuntimeState* state) { + _block_avg_bytes = state->batch_size() * 8; + return Status::OK(); + } Status get_block(RuntimeState* state, Block* block, bool* eos); Status get_block_after_projects(RuntimeState* state, vectorized::Block* block, bool* eos); @@ -158,6 +161,10 @@ class VScanner { int64_t limit() const { return _limit; } + auto get_block_avg_bytes() const { return _block_avg_bytes; } + + void update_block_avg_bytes(size_t block_avg_bytes) { _block_avg_bytes = block_avg_bytes; } + protected: void _discard_conjuncts() { for (auto& conjunct : _conjuncts) { @@ -213,6 +220,8 @@ class VScanner { // num of rows return from scanner, after filter block int64_t _num_rows_return = 0; + size_t _block_avg_bytes = 0; + // Set true after counter is updated finally bool _has_updated_counter = false; diff --git a/be/src/vec/exprs/vectorized_fn_call.cpp b/be/src/vec/exprs/vectorized_fn_call.cpp index 43aa24090393a1..ffe54d9b030aa8 100644 --- a/be/src/vec/exprs/vectorized_fn_call.cpp +++ b/be/src/vec/exprs/vectorized_fn_call.cpp @@ -190,6 +190,24 @@ Status VectorizedFnCall::_do_execute(doris::vectorized::VExprContext* context, return Status::OK(); } +size_t VectorizedFnCall::estimate_memory(const size_t rows) { + if (is_const_and_have_executed()) { // const have execute in open function + return 0; + } + + size_t estimate_size = 0; + for (auto& child : _children) { + estimate_size += child->estimate_memory(rows); + } + + if (_data_type->have_maximum_size_of_value()) { + estimate_size += rows * _data_type->get_size_of_value_in_memory(); + } else { + estimate_size += rows * 512; /// FIXME: estimated value... + } + return estimate_size; +} + Status VectorizedFnCall::execute_runtime_fitler(doris::vectorized::VExprContext* context, doris::vectorized::Block* block, int* result_column_id, ColumnNumbers& args) { diff --git a/be/src/vec/exprs/vectorized_fn_call.h b/be/src/vec/exprs/vectorized_fn_call.h index cea328f413b620..6f3b7d186f53f9 100644 --- a/be/src/vec/exprs/vectorized_fn_call.h +++ b/be/src/vec/exprs/vectorized_fn_call.h @@ -70,6 +70,8 @@ class VectorizedFnCall : public VExpr { bool can_push_down_to_index() const override; bool equals(const VExpr& other) override; + size_t estimate_memory(const size_t rows) override; + protected: FunctionBasePtr _function; std::string _expr_name; diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp index 7cfb96e77118fb..3047df8b2e602d 100644 --- a/be/src/vec/exprs/vexpr.cpp +++ b/be/src/vec/exprs/vexpr.cpp @@ -739,6 +739,24 @@ Status VExpr::_evaluate_inverted_index(VExprContext* context, const FunctionBase return Status::OK(); } +size_t VExpr::estimate_memory(const size_t rows) { + if (is_const_and_have_executed()) { + return 0; + } + + size_t estimate_size = 0; + for (auto& child : _children) { + estimate_size += child->estimate_memory(rows); + } + + if (_data_type->have_maximum_size_of_value()) { + estimate_size += rows * _data_type->get_size_of_value_in_memory(); + } else { + estimate_size += rows * 64; /// TODO: need a more reasonable value + } + return estimate_size; +} + bool VExpr::fast_execute(doris::vectorized::VExprContext* context, doris::vectorized::Block* block, int* result_column_id) { if (context->get_inverted_index_context() && diff --git a/be/src/vec/exprs/vexpr.h b/be/src/vec/exprs/vexpr.h index 91786337244013..ea4575b7e61f49 100644 --- a/be/src/vec/exprs/vexpr.h +++ b/be/src/vec/exprs/vexpr.h @@ -129,6 +129,8 @@ class VExpr { Status _evaluate_inverted_index(VExprContext* context, const FunctionBasePtr& function, uint32_t segment_num_rows); + virtual size_t estimate_memory(const size_t rows); + // Only the 4th parameter is used in the runtime filter. In and MinMax need overwrite the // interface virtual Status execute_runtime_fitler(VExprContext* context, Block* block, diff --git a/be/src/vec/exprs/vexpr_context.cpp b/be/src/vec/exprs/vexpr_context.cpp index b01ce2fade3660..7540ff66bbc1e7 100644 --- a/be/src/vec/exprs/vexpr_context.cpp +++ b/be/src/vec/exprs/vexpr_context.cpp @@ -17,6 +17,7 @@ #include "vec/exprs/vexpr_context.h" +#include #include #include #include @@ -141,7 +142,9 @@ Status VExprContext::filter_block(VExprContext* vexpr_ctx, Block* block, size_t return Status::OK(); } int result_column_id = -1; + size_t origin_size = block->allocated_bytes(); RETURN_IF_ERROR(vexpr_ctx->execute(block, &result_column_id)); + vexpr_ctx->_memory_usage = (block->allocated_bytes() - origin_size); return Block::filter_block(block, result_column_id, column_to_keep); } @@ -322,8 +325,16 @@ Status VExprContext::execute_conjuncts_and_filter_block(const VExprContextSPtrs& int column_to_keep) { IColumn::Filter result_filter(block->rows(), 1); bool can_filter_all; + + _reset_memory_usage(ctxs); + RETURN_IF_ERROR( execute_conjuncts(ctxs, nullptr, false, block, &result_filter, &can_filter_all)); + + // Accumulate the usage of `result_filter` into the first context. + if (!ctxs.empty()) { + ctxs[0]->_memory_usage += result_filter.allocated_bytes(); + } if (can_filter_all) { for (auto& col : columns_to_filter) { // NOLINTNEXTLINE(performance-move-const-arg) @@ -354,9 +365,15 @@ Status VExprContext::execute_conjuncts_and_filter_block(const VExprContextSPtrs& std::vector& columns_to_filter, int column_to_keep, IColumn::Filter& filter) { + _reset_memory_usage(ctxs); filter.resize_fill(block->rows(), 1); bool can_filter_all; RETURN_IF_ERROR(execute_conjuncts(ctxs, nullptr, false, block, &filter, &can_filter_all)); + + // Accumulate the usage of `result_filter` into the first context. + if (!ctxs.empty()) { + ctxs[0]->_memory_usage += filter.allocated_bytes(); + } if (can_filter_all) { for (auto& col : columns_to_filter) { // NOLINTNEXTLINE(performance-move-const-arg) @@ -381,13 +398,20 @@ Status VExprContext::get_output_block_after_execute_exprs( auto rows = input_block.rows(); vectorized::Block tmp_block(input_block.get_columns_with_type_and_name()); vectorized::ColumnsWithTypeAndName result_columns; + _reset_memory_usage(output_vexpr_ctxs); + for (const auto& vexpr_ctx : output_vexpr_ctxs) { int result_column_id = -1; + int origin_columns = tmp_block.columns(); + size_t origin_usage = tmp_block.allocated_bytes(); RETURN_IF_ERROR(vexpr_ctx->execute(&tmp_block, &result_column_id)); DCHECK(result_column_id != -1); + + vexpr_ctx->_memory_usage = tmp_block.allocated_bytes() - origin_usage; const auto& col = tmp_block.get_by_position(result_column_id); - if (do_projection) { + if (do_projection && origin_columns <= result_column_id) { result_columns.emplace_back(col.column->clone_resized(rows), col.type, col.name); + vexpr_ctx->_memory_usage += result_columns.back().column->allocated_bytes(); } else { result_columns.emplace_back(tmp_block.get_by_position(result_column_id)); } @@ -396,5 +420,11 @@ Status VExprContext::get_output_block_after_execute_exprs( return Status::OK(); } +void VExprContext::_reset_memory_usage(const VExprContextSPtrs& contexts) { + std::for_each(contexts.begin(), contexts.end(), + [](auto&& context) { context->_memory_usage = 0; }); +} + #include "common/compile_check_end.h" + } // namespace doris::vectorized diff --git a/be/src/vec/exprs/vexpr_context.h b/be/src/vec/exprs/vexpr_context.h index d3b159fc400ecc..3fb6a1a3b74407 100644 --- a/be/src/vec/exprs/vexpr_context.h +++ b/be/src/vec/exprs/vexpr_context.h @@ -19,6 +19,8 @@ #include +#include +#include #include #include #include @@ -271,10 +273,21 @@ class VExprContext { return *this; } + [[nodiscard]] static size_t get_memory_usage(const VExprContextSPtrs& contexts) { + size_t usage = 0; + std::for_each(contexts.cbegin(), contexts.cend(), + [&usage](auto&& context) { usage += context->_memory_usage; }); + return usage; + } + + [[nodiscard]] size_t get_memory_usage() const { return _memory_usage; } + private: // Close method is called in vexpr context dector, not need call expicility void close(); + static void _reset_memory_usage(const VExprContextSPtrs& contexts); + friend class VExpr; /// The expr tree this context is for. @@ -301,5 +314,6 @@ class VExprContext { bool _force_materialize_slot = false; std::shared_ptr _inverted_index_context; + size_t _memory_usage = 0; }; } // namespace doris::vectorized diff --git a/be/src/vec/exprs/vin_predicate.cpp b/be/src/vec/exprs/vin_predicate.cpp index 4b1622589be608..244f4c06e605e9 100644 --- a/be/src/vec/exprs/vin_predicate.cpp +++ b/be/src/vec/exprs/vin_predicate.cpp @@ -146,6 +146,26 @@ Status VInPredicate::execute(VExprContext* context, Block* block, int* result_co return Status::OK(); } +size_t VInPredicate::estimate_memory(const size_t rows) { + if (is_const_and_have_executed()) { + return 0; + } + + size_t estimate_size = 0; + + for (int i = 0; i < _children.size(); ++i) { + estimate_size += _children[i]->estimate_memory(rows); + } + + if (_data_type->is_nullable()) { + estimate_size += rows * sizeof(uint8_t); + } + + estimate_size += rows * sizeof(uint8_t); + + return estimate_size; +} + const std::string& VInPredicate::expr_name() const { return _expr_name; } diff --git a/be/src/vec/exprs/vin_predicate.h b/be/src/vec/exprs/vin_predicate.h index d9bc6e3cab9978..71072807813906 100644 --- a/be/src/vec/exprs/vin_predicate.h +++ b/be/src/vec/exprs/vin_predicate.h @@ -43,6 +43,7 @@ class VInPredicate final : public VExpr { VInPredicate(const TExprNode& node); ~VInPredicate() override = default; Status execute(VExprContext* context, Block* block, int* result_column_id) override; + size_t estimate_memory(const size_t rows) override; Status prepare(RuntimeState* state, const RowDescriptor& desc, VExprContext* context) override; Status open(RuntimeState* state, VExprContext* context, FunctionContext::FunctionStateScope scope) override; diff --git a/be/src/vec/exprs/vslot_ref.h b/be/src/vec/exprs/vslot_ref.h index 2bea6ea5c06664..f9165ee1e35536 100644 --- a/be/src/vec/exprs/vslot_ref.h +++ b/be/src/vec/exprs/vslot_ref.h @@ -54,6 +54,8 @@ class VSlotRef final : public VExpr { bool equals(const VExpr& other) override; + size_t estimate_memory(const size_t rows) override { return 0; } + private: int _slot_id; int _column_id; diff --git a/be/src/vec/runtime/partitioner.cpp b/be/src/vec/runtime/partitioner.cpp index 2654026ed918f1..5dc470a8d27345 100644 --- a/be/src/vec/runtime/partitioner.cpp +++ b/be/src/vec/runtime/partitioner.cpp @@ -68,6 +68,7 @@ template Status Crc32HashPartitioner::clone(RuntimeState* state, std::unique_ptr& partitioner) { auto* new_partitioner = new Crc32HashPartitioner(cast_set(_partition_count)); + partitioner.reset(new_partitioner); new_partitioner->_partition_expr_ctxs.resize(_partition_expr_ctxs.size()); for (size_t i = 0; i < _partition_expr_ctxs.size(); i++) { diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp index c3277b0917e84c..1eca1333a97aa9 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.cpp +++ b/be/src/vec/runtime/vdata_stream_recvr.cpp @@ -333,7 +333,8 @@ VDataStreamRecvr::VDataStreamRecvr(VDataStreamMgr* stream_mgr, pipeline::Exchang _sender_to_local_channel_dependency.resize(num_queues); for (size_t i = 0; i < num_queues; i++) { _sender_to_local_channel_dependency[i] = pipeline::Dependency::create_shared( - _dest_node_id, _dest_node_id, "LocalExchangeChannelDependency", true); + _dest_node_id, _dest_node_id, fmt::format("LocalExchangeChannelDependency_{}", i), + true); } _sender_queues.reserve(num_queues); int num_sender_per_queue = is_merging ? 1 : num_senders; @@ -389,6 +390,9 @@ Status VDataStreamRecvr::add_block(std::unique_ptr pblock, int sender_id int64_t packet_seq, ::google::protobuf::Closure** done, const int64_t wait_for_worker, const uint64_t time_to_find_recvr) { + if (_parent->state()->get_query_ctx()->low_memory_mode()) { + set_low_memory_mode(); + } SCOPED_ATTACH_TASK(_query_thread_context); int use_sender_id = _is_merging ? sender_id : 0; return _sender_queues[use_sender_id]->add_block(std::move(pblock), be_number, packet_seq, done, @@ -396,6 +400,9 @@ Status VDataStreamRecvr::add_block(std::unique_ptr pblock, int sender_id } void VDataStreamRecvr::add_block(Block* block, int sender_id, bool use_move) { + if (_parent->state()->get_query_ctx()->low_memory_mode()) { + set_low_memory_mode(); + } int use_sender_id = _is_merging ? sender_id : 0; _sender_queues[use_sender_id]->add_block(block, use_move); } diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h index 1639366c8b83d6..0489b99e22fa9a 100644 --- a/be/src/vec/runtime/vdata_stream_recvr.h +++ b/be/src/vec/runtime/vdata_stream_recvr.h @@ -117,6 +117,8 @@ class VDataStreamRecvr : public HasTaskExecutionCtx { std::shared_ptr get_local_channel_dependency(int sender_id); + void set_low_memory_mode() { _sender_queue_mem_limit = 1012 * 1024; } + private: friend struct BlockSupplierSortCursorImpl; @@ -142,7 +144,7 @@ class VDataStreamRecvr : public HasTaskExecutionCtx { std::unique_ptr _mem_tracker; // Managed by object pool std::vector _sender_queues; - size_t _sender_queue_mem_limit; + std::atomic _sender_queue_mem_limit; std::unique_ptr _merger; diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index f46602e8d67158..5c83d3eb2eaeae 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -108,9 +108,7 @@ Status Channel::open(RuntimeState* state) { } } _be_number = state->be_number(); - _brpc_timeout_ms = get_execution_rpc_timeout_ms(state->execution_timeout()); - _serializer.set_is_local(_is_local); // In bucket shuffle join will set fragment_instance_id (-1, -1) diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.cpp b/be/src/vec/sink/writer/vtablet_writer_v2.cpp index f3ad6b1d5e3f0c..0526d8f377e9c3 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.cpp +++ b/be/src/vec/sink/writer/vtablet_writer_v2.cpp @@ -510,7 +510,8 @@ Status VTabletWriterV2::_write_memtable(std::shared_ptr block } { SCOPED_TIMER(_wait_mem_limit_timer); - ExecEnv::GetInstance()->memtable_memory_limiter()->handle_memtable_flush(); + ExecEnv::GetInstance()->memtable_memory_limiter()->handle_workload_group_memtable_flush( + _state->workload_group()); } SCOPED_TIMER(_write_memtable_timer); st = delta_writer->write(block.get(), rows.row_idxes); diff --git a/be/src/vec/spill/spill_reader.cpp b/be/src/vec/spill/spill_reader.cpp index 3c2f00273a4b5d..40323f824a815f 100644 --- a/be/src/vec/spill/spill_reader.cpp +++ b/be/src/vec/spill/spill_reader.cpp @@ -17,6 +17,8 @@ #include "vec/spill/spill_reader.h" +#include + #include #include "common/cast_set.h" @@ -27,6 +29,7 @@ #include "runtime/exec_env.h" #include "util/slice.h" #include "vec/core/block.h" +#include "vec/spill/spill_stream_manager.h" namespace doris { #include "common/compile_check_begin.h" namespace io { @@ -39,7 +42,9 @@ Status SpillReader::open() { return Status::OK(); } - SCOPED_TIMER(read_timer_); + SCOPED_TIMER(_read_file_timer); + + COUNTER_UPDATE(_read_file_count, 1); RETURN_IF_ERROR(io::global_local_filesystem()->open_file(file_path_, &file_reader_)); @@ -48,16 +53,25 @@ Status SpillReader::open() { 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 (_query_statistics) { + _query_statistics->add_spill_read_bytes_from_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 (_query_statistics) { + _query_statistics->add_spill_read_bytes_from_local_storage(bytes_read); + } size_t buff_size = std::max(block_count_ * sizeof(size_t), max_sub_block_size_); try { @@ -75,6 +89,12 @@ Status SpillReader::open() { 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 (_query_statistics) { + _query_statistics->add_spill_read_bytes_from_local_storage(bytes_read); + } block_start_offsets_.resize(block_count_ + 1); for (size_t i = 0; i < block_count_; ++i) { @@ -85,6 +105,11 @@ Status SpillReader::open() { 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(); @@ -105,21 +130,28 @@ Status SpillReader::read(Block* block, bool* eos) { Slice result(read_buff_.get(), bytes_to_read); size_t bytes_read = 0; { - SCOPED_TIMER(read_timer_); + 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); - COUNTER_UPDATE(read_bytes_, bytes_read); if (bytes_read > 0) { + COUNTER_UPDATE(_read_file_size, bytes_read); + ExecEnv::GetInstance()->spill_stream_mgr()->update_spill_read_bytes(bytes_read); + if (_query_statistics) { + _query_statistics->add_spill_read_bytes_from_local_storage(bytes_read); + } + COUNTER_UPDATE(_read_block_count, 1); { - SCOPED_TIMER(deserialize_timer_); + SCOPED_TIMER(_deserialize_timer); if (!pb_block_.ParseFromArray(result.data, cast_set(result.size))) { return Status::InternalError("Failed to read spilled block"); } RETURN_IF_ERROR(block->deserialize(pb_block_)); } + COUNTER_UPDATE(_read_block_data_size, block->bytes()); + COUNTER_UPDATE(_read_rows_count, block->rows()); } else { block->clear_column_data(); } diff --git a/be/src/vec/spill/spill_reader.h b/be/src/vec/spill/spill_reader.h index 6dea8376206df9..da975bdb605dd1 100644 --- a/be/src/vec/spill/spill_reader.h +++ b/be/src/vec/spill/spill_reader.h @@ -25,6 +25,7 @@ #include "common/status.h" #include "io/fs/file_reader_writer_fwd.h" +#include "runtime/query_statistics.h" #include "util/runtime_profile.h" namespace doris::vectorized { @@ -32,8 +33,11 @@ namespace doris::vectorized { class Block; class SpillReader { public: - SpillReader(int64_t stream_id, std::string file_path) - : stream_id_(stream_id), file_path_(std::move(file_path)) {} + SpillReader(std::shared_ptr query_statistics, int64_t stream_id, + std::string file_path) + : stream_id_(stream_id), + file_path_(std::move(file_path)), + _query_statistics(std::move(query_statistics)) {} ~SpillReader() { (void)close(); } @@ -51,12 +55,14 @@ class SpillReader { size_t block_count() const { return block_count_; } - void set_counters(RuntimeProfile::Counter* read_timer, - RuntimeProfile::Counter* deserialize_timer, - RuntimeProfile::Counter* read_bytes) { - read_timer_ = read_timer; - deserialize_timer_ = deserialize_timer; - read_bytes_ = read_bytes; + void set_counters(RuntimeProfile* profile) { + _read_file_timer = profile->get_counter("SpillReadFileTime"); + _deserialize_timer = profile->get_counter("SpillReadDerializeBlockTime"); + _read_block_count = profile->get_counter("SpillReadBlockCount"); + _read_block_data_size = profile->get_counter("SpillReadBlockBytes"); + _read_file_size = profile->get_counter("SpillReadFileBytes"); + _read_rows_count = profile->get_counter("SpillReadRows"); + _read_file_count = profile->get_counter("SpillReadFileCount"); } private: @@ -72,9 +78,15 @@ class SpillReader { PBlock pb_block_; - RuntimeProfile::Counter* read_timer_; - RuntimeProfile::Counter* deserialize_timer_; - RuntimeProfile::Counter* read_bytes_; + 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 _query_statistics = nullptr; }; using SpillReaderUPtr = std::unique_ptr; diff --git a/be/src/vec/spill/spill_stream.cpp b/be/src/vec/spill/spill_stream.cpp index d83cbabb1893d9..fafebb4e62ae41 100644 --- a/be/src/vec/spill/spill_stream.cpp +++ b/be/src/vec/spill/spill_stream.cpp @@ -25,6 +25,7 @@ #include "io/fs/local_file_system.h" #include "runtime/exec_env.h" +#include "runtime/query_context.h" #include "runtime/runtime_state.h" #include "runtime/thread_context.h" #include "util/debug_points.h" @@ -45,16 +46,31 @@ SpillStream::SpillStream(RuntimeState* state, int64_t stream_id, SpillDataDir* d batch_rows_(batch_rows), batch_bytes_(batch_bytes), query_id_(state->query_id()), - profile_(profile) {} + profile_(profile) { + _total_file_count = profile_->get_counter("SpillWriteFileTotalCount"); + _current_file_count = profile_->get_counter("SpillWriteFileCurrentCount"); + _current_file_size = 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", { @@ -80,10 +96,29 @@ void SpillStream::gc() { } Status SpillStream::prepare() { - writer_ = std::make_unique(stream_id_, batch_rows_, data_dir_, spill_dir_); + writer_ = std::make_unique( + state_->get_query_ctx()->get_mem_tracker()->get_query_statistics(), profile_, + stream_id_, batch_rows_, data_dir_, spill_dir_); + _set_write_counters(profile_); - reader_ = std::make_unique(stream_id_, writer_->get_file_path()); - return Status::OK(); + reader_ = std::make_unique( + state_->get_query_ctx()->get_mem_tracker()->get_query_statistics(), 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()->get_mem_tracker()->get_query_statistics(), stream_id_, + writer_->get_file_path()); } const TUniqueId& SpillStream::query_id() const { @@ -93,12 +128,6 @@ const TUniqueId& SpillStream::query_id() const { const std::string& SpillStream::get_spill_root_dir() const { return data_dir_->path(); } -Status SpillStream::prepare_spill() { - DBUG_EXECUTE_IF("fault_inject::spill_stream::prepare_spill", { - return Status::Error("fault_inject spill_stream prepare_spill failed"); - }); - return writer_->open(); -} Status SpillStream::spill_block(RuntimeState* state, const Block& block, bool eof) { size_t written_bytes = 0; @@ -107,9 +136,7 @@ Status SpillStream::spill_block(RuntimeState* state, const Block& block, bool eo }); RETURN_IF_ERROR(writer_->write(state, block, written_bytes)); if (eof) { - RETURN_IF_ERROR(writer_->close()); - total_written_bytes_ = writer_->get_written_bytes(); - writer_.reset(); + RETURN_IF_ERROR(spill_eof()); } else { total_written_bytes_ = writer_->get_written_bytes(); } @@ -123,6 +150,10 @@ Status SpillStream::spill_eof() { auto status = writer_->close(); total_written_bytes_ = writer_->get_written_bytes(); writer_.reset(); + + if (status.ok()) { + _ready_for_reading = true; + } return status; } diff --git a/be/src/vec/spill/spill_stream.h b/be/src/vec/spill/spill_stream.h index a95a4760322703..525abbb7855d26 100644 --- a/be/src/vec/spill/spill_stream.h +++ b/be/src/vec/spill/spill_stream.h @@ -36,7 +36,8 @@ class SpillDataDir; class SpillStream { public: // to avoid too many small file writes - static constexpr int MIN_SPILL_WRITE_BATCH_MEM = 32 * 1024; + 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); @@ -56,47 +57,40 @@ class SpillStream { int64_t get_written_bytes() const { return total_written_bytes_; } - Status prepare_spill(); - Status spill_block(RuntimeState* state, const Block& block, bool eof); Status spill_eof(); Status read_next_block_sync(Block* block, bool* eos); - void set_write_counters(RuntimeProfile::Counter* serialize_timer, - RuntimeProfile::Counter* write_block_counter, - RuntimeProfile::Counter* write_bytes_counter, - RuntimeProfile::Counter* write_timer, - RuntimeProfile::Counter* wait_io_timer) { - writer_->set_counters(serialize_timer, write_block_counter, write_bytes_counter, - write_timer); - write_wait_io_timer_ = wait_io_timer; - } - - void set_read_counters(RuntimeProfile::Counter* read_timer, - RuntimeProfile::Counter* deserialize_timer, - RuntimeProfile::Counter* read_bytes, - RuntimeProfile::Counter* wait_io_timer) { - reader_->set_counters(read_timer, deserialize_timer, read_bytes); - read_wait_io_timer_ = wait_io_timer; - } + void set_read_counters(RuntimeProfile* profile) { reader_->set_counters(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_; @@ -105,8 +99,9 @@ class SpillStream { TUniqueId query_id_; RuntimeProfile* profile_ = nullptr; - RuntimeProfile::Counter* write_wait_io_timer_ = nullptr; - RuntimeProfile::Counter* read_wait_io_timer_ = 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 vectorized diff --git a/be/src/vec/spill/spill_stream_manager.cpp b/be/src/vec/spill/spill_stream_manager.cpp index 89bd09b7fadd06..833c5471fca5c0 100644 --- a/be/src/vec/spill/spill_stream_manager.cpp +++ b/be/src/vec/spill/spill_stream_manager.cpp @@ -43,6 +43,9 @@ namespace doris::vectorized { #include "common/compile_check_begin.h" +SpillStreamManager::~SpillStreamManager() { + DorisMetrics::instance()->metric_registry()->deregister_entity(_entity); +} SpillStreamManager::SpillStreamManager( std::unordered_map>&& spill_store_map) @@ -84,9 +87,27 @@ Status SpillStreamManager::init() { "Spill", "spill_gc_thread", [this]() { this->_spill_gc_thread_callback(); }, &_spill_gc_thread)); LOG(INFO) << "spill gc thread started"; + + _init_metrics(); + return Status::OK(); } +void SpillStreamManager::_init_metrics() { + _entity = DorisMetrics::instance()->metric_registry()->register_entity("spill", + {{"name", "spill"}}); + + _spill_write_bytes_metric = std::make_unique( + doris::MetricType::COUNTER, doris::MetricUnit::BYTES, "spill_write_bytes"); + _spill_write_bytes_counter = (IntAtomicCounter*)(_entity->register_metric( + _spill_write_bytes_metric.get())); + + _spill_read_bytes_metric = std::make_unique( + doris::MetricType::COUNTER, doris::MetricUnit::BYTES, "spill_read_bytes"); + _spill_read_bytes_counter = (IntAtomicCounter*)(_entity->register_metric( + _spill_read_bytes_metric.get())); +} + // clean up stale spilled files void SpillStreamManager::_spill_gc_thread_callback() { while (!_stop_background_threads_latch.wait_for( @@ -108,45 +129,23 @@ Status SpillStreamManager::_init_spill_store_map() { std::vector SpillStreamManager::_get_stores_for_spill( TStorageMedium::type storage_medium) { - std::vector stores; + std::vector> stores_with_usage; for (auto& [_, store] : _spill_store_map) { if (store->storage_medium() == storage_medium && !store->reach_capacity_limit(0)) { - stores.push_back(store.get()); + stores_with_usage.emplace_back(store.get(), store->_get_disk_usage(0)); } } - if (stores.empty()) { - return stores; + if (stores_with_usage.empty()) { + return {}; } - std::sort(stores.begin(), stores.end(), [](SpillDataDir* a, SpillDataDir* b) { - return a->_get_disk_usage(0) < b->_get_disk_usage(0); - }); - - size_t seventy_percent_index = stores.size(); - size_t eighty_five_percent_index = stores.size(); - for (size_t index = 0; index < stores.size(); index++) { - // If the usage of the store is less than 70%, we choose disk randomly. - if (stores[index]->_get_disk_usage(0) > 0.7 && seventy_percent_index == stores.size()) { - seventy_percent_index = index; - } - if (stores[index]->_get_disk_usage(0) > 0.85 && - eighty_five_percent_index == stores.size()) { - eighty_five_percent_index = index; - break; - } - } + std::sort(stores_with_usage.begin(), stores_with_usage.end(), + [](auto&& a, auto&& b) { return a.second < b.second; }); - std::random_device rd; - std::mt19937 g(rd()); - std::shuffle(stores.begin(), stores.begin() + seventy_percent_index, g); - if (seventy_percent_index != stores.size()) { - std::shuffle(stores.begin() + seventy_percent_index, - stores.begin() + eighty_five_percent_index, g); - } - if (eighty_five_percent_index != stores.size()) { - std::shuffle(stores.begin() + eighty_five_percent_index, stores.end(), g); + std::vector stores; + for (const auto& [store, _] : stores_with_usage) { + stores.emplace_back(store); } - return stores; } @@ -169,6 +168,7 @@ Status SpillStreamManager::register_spill_stream(RuntimeState* state, SpillStrea 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); diff --git a/be/src/vec/spill/spill_stream_manager.h b/be/src/vec/spill/spill_stream_manager.h index 7bcfe9500979b9..53ae89e9111d0a 100644 --- a/be/src/vec/spill/spill_stream_manager.h +++ b/be/src/vec/spill/spill_stream_manager.h @@ -30,6 +30,14 @@ namespace doris { #include "common/compile_check_begin.h" class RuntimeProfile; +template +class AtomicCounter; +using IntAtomicCounter = AtomicCounter; +template +class AtomicGauge; +using UIntGauge = AtomicGauge; +class MetricEntity; +struct MetricPrototype; namespace vectorized { @@ -106,6 +114,7 @@ class SpillDataDir { }; class SpillStreamManager { public: + ~SpillStreamManager(); SpillStreamManager(std::unordered_map>&& spill_store_map); @@ -133,7 +142,12 @@ class SpillStreamManager { ThreadPool* get_spill_io_thread_pool() const { return _spill_io_thread_pool.get(); } + void update_spill_write_bytes(int64_t bytes) { _spill_write_bytes_counter->increment(bytes); } + + void update_spill_read_bytes(int64_t bytes) { _spill_read_bytes_counter->increment(bytes); } + private: + void _init_metrics(); Status _init_spill_store_map(); void _spill_gc_thread_callback(); std::vector _get_stores_for_spill(TStorageMedium::type storage_medium); @@ -145,6 +159,14 @@ class SpillStreamManager { scoped_refptr _spill_gc_thread; std::atomic_uint64_t id_ = 0; + + std::shared_ptr _entity {nullptr}; + + std::unique_ptr _spill_write_bytes_metric {nullptr}; + std::unique_ptr _spill_read_bytes_metric {nullptr}; + + IntAtomicCounter* _spill_write_bytes_counter {nullptr}; + IntAtomicCounter* _spill_read_bytes_counter {nullptr}; }; } // namespace vectorized } // namespace doris diff --git a/be/src/vec/spill/spill_writer.cpp b/be/src/vec/spill/spill_writer.cpp index bf755e76452f5d..3a576004091f83 100644 --- a/be/src/vec/spill/spill_writer.cpp +++ b/be/src/vec/spill/spill_writer.cpp @@ -46,14 +46,20 @@ Status SpillWriter::close() { // meta: block1 offset, block2 offset, ..., blockn offset, max_sub_block_size, n { - SCOPED_TIMER(write_timer_); + SCOPED_TIMER(_write_file_timer); RETURN_IF_ERROR(file_writer_->append(meta_)); } total_written_bytes_ += meta_.size(); - COUNTER_UPDATE(write_bytes_counter_, meta_.size()); - + COUNTER_UPDATE(_write_file_total_size, meta_.size()); + if (_query_statistics) { + _query_statistics->add_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()); @@ -65,6 +71,8 @@ Status SpillWriter::write(RuntimeState* state, const Block& block, size_t& writt 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); @@ -85,6 +93,9 @@ Status SpillWriter::write(RuntimeState* state, const Block& block, size_t& writt } }); + 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; @@ -98,22 +109,29 @@ Status SpillWriter::_write_internal(const Block& block, size_t& written_bytes) { Status status; std::string buff; + int64_t buff_size {0}; if (block.rows() > 0) { { PBlock pblock; - SCOPED_TIMER(serialize_timer_); + SCOPED_TIMER(_serialize_timer); status = block.serialize( BeExecVersionManager::get_newest_version(), &pblock, &uncompressed_bytes, &compressed_bytes, 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())) { + if (data_dir_->reach_capacity_limit(buff_size)) { return Status::Error( "spill data total size exceed limit, path: {}, size limit: {}, spill data " "size: {}", @@ -123,30 +141,36 @@ Status SpillWriter::_write_internal(const Block& block, size_t& written_bytes) { } { - auto buff_size = buff.size(); 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_, 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_bytes_counter_, buff_size); - COUNTER_UPDATE(write_block_counter_, 1); + COUNTER_UPDATE(_write_file_total_size, buff_size); + if (_query_statistics) { + _query_statistics->add_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_timer_); + SCOPED_TIMER(_write_file_timer); status = file_writer_->append(buff); RETURN_IF_ERROR(status); } } } - return Status::OK(); + return status; } } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/spill/spill_writer.h b/be/src/vec/spill/spill_writer.h index 921bd8ea874c7f..ab1ea34a062bf2 100644 --- a/be/src/vec/spill/spill_writer.h +++ b/be/src/vec/spill/spill_writer.h @@ -22,6 +22,7 @@ #include #include "io/fs/file_writer.h" +#include "runtime/query_statistics.h" #include "util/runtime_profile.h" #include "vec/core/block.h" namespace doris { @@ -32,9 +33,17 @@ namespace vectorized { class SpillDataDir; class SpillWriter { public: - SpillWriter(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) { - file_path_ = dir + "/" + std::to_string(file_index_); + SpillWriter(std::shared_ptr query_statistics, 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), + _query_statistics(std::move(query_statistics)) { + // 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"; + + _memory_used_counter = profile->get_counter("MemoryUsage"); } Status open(); @@ -49,19 +58,17 @@ class SpillWriter { const std::string& get_file_path() const { return file_path_; } - void set_counters(RuntimeProfile::Counter* serialize_timer, - RuntimeProfile::Counter* write_block_counter, - RuntimeProfile::Counter* write_bytes_counter, - RuntimeProfile::Counter* write_timer) { - serialize_timer_ = serialize_timer; - write_block_counter_ = write_block_counter; - write_bytes_counter_ = write_bytes_counter; - write_timer_ = write_timer; + void set_counters(RuntimeProfile* profile) { + _write_file_timer = profile->get_counter("SpillWriteFileTime"); + _serialize_timer = profile->get_counter("SpillWriteSerializeBlockTime"); + _write_block_counter = profile->get_counter("SpillWriteBlockCount"); + _write_block_bytes_counter = profile->get_counter("SpillWriteBlockBytes"); + _write_file_total_size = profile->get_counter("SpillWriteFileBytes"); + _write_file_current_size = profile->get_counter("SpillWriteFileCurrentBytes"); + _write_rows_counter = profile->get_counter("SpillWriteRows"); } private: - void _init_profile(); - Status _write_internal(const Block& block, size_t& written_bytes); // not owned, point to the data dir of this rowset @@ -71,7 +78,6 @@ class SpillWriter { int64_t stream_id_; size_t batch_size_; size_t max_sub_block_size_ = 0; - int file_index_ = 0; std::string file_path_; std::unique_ptr file_writer_; @@ -79,10 +85,16 @@ class SpillWriter { int64_t total_written_bytes_ = 0; std::string meta_; - RuntimeProfile::Counter* write_bytes_counter_; - RuntimeProfile::Counter* serialize_timer_; - RuntimeProfile::Counter* write_timer_; - RuntimeProfile::Counter* write_block_counter_; + 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 _query_statistics = nullptr; }; using SpillWriterUPtr = std::unique_ptr; } // namespace vectorized diff --git a/be/test/runtime/memory/mem_tracker_test.cpp b/be/test/runtime/memory/mem_tracker_test.cpp index eb66635ce072d2..be5c8ebdf6c3df 100644 --- a/be/test/runtime/memory/mem_tracker_test.cpp +++ b/be/test/runtime/memory/mem_tracker_test.cpp @@ -34,23 +34,18 @@ TEST(MemTrackerTest, SingleTrackerNoLimit) { EXPECT_EQ(t->consumption(), 20); t->release(15); EXPECT_EQ(t->consumption(), 5); - EXPECT_FALSE(t->limit_exceeded()); t->release(5); } TEST(MemTrackerTest, SingleTrackerWithLimit) { auto t = std::make_unique(MemTrackerLimiter::Type::GLOBAL, "limit tracker", 11); - EXPECT_TRUE(t->has_limit()); t->consume(10); EXPECT_EQ(t->consumption(), 10); - EXPECT_FALSE(t->limit_exceeded()); t->consume(10); EXPECT_EQ(t->consumption(), 20); - EXPECT_TRUE(t->limit_exceeded()); t->release(15); EXPECT_EQ(t->consumption(), 5); - EXPECT_FALSE(t->limit_exceeded()); t->release(5); } diff --git a/be/test/runtime/workload_group/workload_group_manager_test.cpp b/be/test/runtime/workload_group/workload_group_manager_test.cpp new file mode 100644 index 00000000000000..17f4569f39b62c --- /dev/null +++ b/be/test/runtime/workload_group/workload_group_manager_test.cpp @@ -0,0 +1,416 @@ +// 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 "runtime/workload_group/workload_group_manager.h" + +#include +#include +#include +#include +#include + +#include +#include + +#include "common/status.h" +#include "olap/olap_define.h" +#include "pipeline/pipeline_tracing.h" +#include "runtime/exec_env.h" +#include "runtime/query_context.h" +#include "runtime/runtime_query_statistics_mgr.h" +#include "runtime/workload_group/workload_group.h" +#include "vec/common/pod_array.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris { + +class WorkloadGroupManagerTest : public testing::Test { +public: +protected: + void SetUp() override { + _wg_manager = std::make_unique(); + EXPECT_EQ(system("rm -rf ./wg_test_run && mkdir -p ./wg_test_run"), 0); + + std::vector paths; + std::string path = std::filesystem::absolute("./wg_test_run").string(); + auto olap_res = doris::parse_conf_store_paths(path, &paths); + EXPECT_TRUE(olap_res.ok()) << olap_res.to_string(); + + std::vector spill_paths; + olap_res = doris::parse_conf_store_paths(path, &spill_paths); + ASSERT_TRUE(olap_res.ok()) << olap_res.to_string(); + std::unordered_map> spill_store_map; + for (const auto& spill_path : spill_paths) { + spill_store_map.emplace( + spill_path.path, + std::make_unique( + spill_path.path, spill_path.capacity_bytes, spill_path.storage_medium)); + } + + ExecEnv::GetInstance()->_runtime_query_statistics_mgr = new RuntimeQueryStatisticsMgr(); + ExecEnv::GetInstance()->_spill_stream_mgr = + new vectorized::SpillStreamManager(std::move(spill_store_map)); + auto st = ExecEnv::GetInstance()->_spill_stream_mgr->init(); + EXPECT_TRUE(st.ok()) << "init spill stream manager failed: " << st.to_string(); + ExecEnv::GetInstance()->_pipeline_tracer_ctx = + std::make_unique(); + } + void TearDown() override { + _wg_manager.reset(); + ExecEnv::GetInstance()->_runtime_query_statistics_mgr->stop_report_thread(); + SAFE_DELETE(ExecEnv::GetInstance()->_runtime_query_statistics_mgr); + + EXPECT_EQ(system("rm -rf ./wg_test_run"), 0); + } + +private: + std::shared_ptr _generate_on_query(std::shared_ptr& wg) { + TQueryOptions query_options; + query_options.query_type = TQueryType::SELECT; + query_options.mem_limit = 1024L * 1024 * 128; + query_options.query_slot_count = 1; + TNetworkAddress fe_address; + fe_address.hostname = "127.0.0.1"; + fe_address.port = 8060; + auto query_context = QueryContext::create_shared( + generate_uuid(), ExecEnv::GetInstance(), query_options, TNetworkAddress {}, true, + fe_address, QuerySource::INTERNAL_FRONTEND); + + auto st = wg->add_query(query_context->query_id(), query_context); + EXPECT_TRUE(st.ok()) << "add query to workload group failed: " << st.to_string(); + + st = query_context->set_workload_group(wg); + EXPECT_TRUE(st.ok()) << "set query's workload group failed: " << st.to_string(); + return query_context; + } + + void _run_checking_loop(const std::shared_ptr& wg) { + CountDownLatch latch(1); + size_t check_times = 100; + while (--check_times > 0) { + _wg_manager->handle_paused_queries(); + if (!_wg_manager->_paused_queries_list.contains(wg) || + _wg_manager->_paused_queries_list[wg].empty()) { + break; + } + latch.wait_for(std::chrono::milliseconds(config::memory_maintenance_sleep_time_ms)); + } + } + + std::unique_ptr _wg_manager; +}; + +TEST_F(WorkloadGroupManagerTest, get_or_create_workload_group) { + auto wg = _wg_manager->get_or_create_workload_group({}); + ASSERT_EQ(wg->id(), 0); +} + +TEST_F(WorkloadGroupManagerTest, add_paused_query) { + auto wg = _wg_manager->get_or_create_workload_group({}); + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason(Status::Error(ErrorCode::QUERY_MEMORY_EXCEEDED, "test")); + query_context->set_mem_limit(1024 * 1024); + + _wg_manager->add_paused_query(query_context, 1024); + + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + SCOPED_ATTACH_TASK(query_context->query_mem_tracker); + query_context->query_mem_tracker->consume(1024 * 1024 * 2); + } + + _run_checking_loop(wg); + + query_context->set_mem_limit(-1024 * 1024); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; + ASSERT_EQ(query_context->is_cancelled(), true) << "query should be canceled"; +} + +TEST_F(WorkloadGroupManagerTest, query_exceed) { + auto wg = _wg_manager->get_or_create_workload_group({}); + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason(Status::Error(ErrorCode::QUERY_MEMORY_EXCEEDED, "test")); + query_context->set_mem_limit(1024 * 1024); + query_context->query_mem_tracker->consume(1024 * 4); + + _wg_manager->add_paused_query(query_context, 1024L * 1024 * 1024); + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + } + + _run_checking_loop(wg); + + query_context->query_mem_tracker->consume(-1024 * 4); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; + ASSERT_EQ(query_context->is_cancelled(), true) << "query should be canceled"; +} + +// if (query_ctx->expected_mem_limit() < +// query_ctx->get_mem_tracker()->consumption() + query_it->reserve_size_) +TEST_F(WorkloadGroupManagerTest, wg_exceed1) { + auto wg = _wg_manager->get_or_create_workload_group({}); + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test")); + query_context->query_mem_tracker->consume(1024L * 1024 * 1024 * 4); + + _wg_manager->add_paused_query(query_context, 1024L); + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + } + + _run_checking_loop(wg); + + query_context->query_mem_tracker->consume(-1024 * 4); + ASSERT_TRUE(query_context->paused_reason().ok()); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; + ASSERT_EQ(query_context->is_cancelled(), false) << "query should be canceled"; +} + +// TWgSlotMemoryPolicy::DISABLED +// query_ctx->workload_group()->exceed_limit() == false +TEST_F(WorkloadGroupManagerTest, wg_exceed2) { + auto wg = _wg_manager->get_or_create_workload_group({}); + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test")); + query_context->query_mem_tracker->consume(1024L * 4); + + _wg_manager->add_paused_query(query_context, 1024L); + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + } + + _run_checking_loop(wg); + + query_context->query_mem_tracker->consume(-1024 * 4); + ASSERT_TRUE(query_context->paused_reason().ok()); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; + ASSERT_EQ(query_context->is_cancelled(), false) << "query should be canceled"; +} + +// TWgSlotMemoryPolicy::DISABLED +// query_ctx->workload_group()->exceed_limit() == true +TEST_F(WorkloadGroupManagerTest, wg_exceed3) { + WorkloadGroupInfo wg_info {.id = 1, .memory_limit = 1024L * 1024}; + auto wg = _wg_manager->get_or_create_workload_group(wg_info); + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test")); + query_context->query_mem_tracker->consume(1024L * 1024 * 4); + + _wg_manager->add_paused_query(query_context, 1024L); + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + } + + wg->refresh_memory_usage(); + _run_checking_loop(wg); + + query_context->query_mem_tracker->consume(-1024L * 1024 * 4); + ASSERT_TRUE(query_context->is_cancelled()); + ASSERT_EQ(query_context->exec_status().code(), ErrorCode::MEM_LIMIT_EXCEEDED); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; +} + +// TWgSlotMemoryPolicy::FIXED +TEST_F(WorkloadGroupManagerTest, wg_exceed4) { + WorkloadGroupInfo wg_info {.id = 1, + .memory_limit = 1024L * 1024 * 100, + .memory_low_watermark = 80, + .memory_high_watermark = 95, + .total_query_slot_count = 5, + .slot_mem_policy = TWgSlotMemoryPolicy::FIXED}; + auto wg = _wg_manager->get_or_create_workload_group(wg_info); + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test")); + query_context->query_mem_tracker->consume(1024L * 1024 * 4); + + _wg_manager->add_paused_query(query_context, 1024L); + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + } + + _wg_manager->refresh_wg_weighted_memory_limit(); + LOG(INFO) << "***** wg usage " << wg->refresh_memory_usage(); + _run_checking_loop(wg); + + query_context->query_mem_tracker->consume(-1024L * 1024 * 4); + ASSERT_TRUE(query_context->paused_reason().ok()); + LOG(INFO) << "***** query_context->get_mem_limit(): " << query_context->get_mem_limit(); + const auto delta = std::abs(query_context->get_mem_limit() - + ((1024L * 1024 * 100 * 95) / 100 - 10 * 1024 * 1024) / 5); + ASSERT_LE(delta, 1); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; +} + +// TWgSlotMemoryPolicy::DYNAMIC +TEST_F(WorkloadGroupManagerTest, wg_exceed5) { + WorkloadGroupInfo wg_info {.id = 1, + .memory_limit = 1024L * 1024 * 100, + .memory_low_watermark = 80, + .memory_high_watermark = 95, + .total_query_slot_count = 5, + .slot_mem_policy = TWgSlotMemoryPolicy::DYNAMIC}; + auto wg = _wg_manager->get_or_create_workload_group(wg_info); + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test")); + query_context->query_mem_tracker->consume(1024L * 1024 * 4); + + _wg_manager->add_paused_query(query_context, 1024L); + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + } + + _wg_manager->refresh_wg_weighted_memory_limit(); + LOG(INFO) << "***** wg usage " << wg->refresh_memory_usage(); + _run_checking_loop(wg); + + query_context->query_mem_tracker->consume(-1024L * 1024 * 4); + ASSERT_TRUE(query_context->paused_reason().ok()); + LOG(INFO) << "***** query_context->get_mem_limit(): " << query_context->get_mem_limit(); + ASSERT_LE(query_context->get_mem_limit(), (1024L * 1024 * 100 * 95) / 100); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; +} + +TEST_F(WorkloadGroupManagerTest, overcommit) { + WorkloadGroupInfo wg_info {.id = 1, .enable_memory_overcommit = true}; + auto wg = _wg_manager->get_or_create_workload_group(wg_info); + EXPECT_EQ(wg->id(), wg_info.id); + EXPECT_EQ(wg->enable_memory_overcommit(), true); + + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test")); + + _wg_manager->add_paused_query(query_context, 1024L * 1024 * 1024); + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + } + + _run_checking_loop(wg); + + query_context->query_mem_tracker->consume(-1024 * 4); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; + ASSERT_EQ(query_context->is_cancelled(), false) << "query should be canceled"; +} + +TEST_F(WorkloadGroupManagerTest, slot_memory_policy_disabled) { + WorkloadGroupInfo wg_info {.id = 1, + .enable_memory_overcommit = false, + .slot_mem_policy = TWgSlotMemoryPolicy::DISABLED}; + auto wg = _wg_manager->get_or_create_workload_group(wg_info); + EXPECT_EQ(wg->id(), wg_info.id); + EXPECT_EQ(wg->enable_memory_overcommit(), false); + EXPECT_EQ(wg->slot_memory_policy(), TWgSlotMemoryPolicy::DISABLED); + + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test")); + + _wg_manager->add_paused_query(query_context, 1024L); + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 1) + << "pasued queue should not be empty"; + } + + _run_checking_loop(wg); + + query_context->query_mem_tracker->consume(-1024 * 4); + + ASSERT_TRUE(query_context->paused_reason().ok()); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; + ASSERT_EQ(query_context->is_cancelled(), false) << "query should be canceled"; +} + +TEST_F(WorkloadGroupManagerTest, query_released) { + auto wg = _wg_manager->get_or_create_workload_group({}); + auto query_context = _generate_on_query(wg); + + query_context->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test")); + query_context->set_mem_limit(1024 * 1024); + + auto canceled_query = _generate_on_query(wg); + canceled_query->update_paused_reason( + Status::Error(ErrorCode::WORKLOAD_GROUP_MEMORY_EXCEEDED, "test for canceled")); + + _wg_manager->add_paused_query(query_context, 1024L * 1024 * 1024); + _wg_manager->add_paused_query(canceled_query, 1024L * 1024 * 1024); + canceled_query->cancel(Status::InternalError("for test")); + + { + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_EQ(_wg_manager->_paused_queries_list[wg].size(), 2) + << "pasued queue should not be empty"; + } + + query_context = nullptr; + + _run_checking_loop(wg); + + std::unique_lock lock(_wg_manager->_paused_queries_lock); + ASSERT_TRUE(_wg_manager->_paused_queries_list[wg].empty()) << "pasued queue should be empty"; +} + +} // namespace doris \ No newline at end of file diff --git a/be/test/vec/columns/column_nullable_seriazlization_test.cpp b/be/test/vec/columns/column_nullable_seriazlization_test.cpp index 70702369cd31af..07316c210f5057 100644 --- a/be/test/vec/columns/column_nullable_seriazlization_test.cpp +++ b/be/test/vec/columns/column_nullable_seriazlization_test.cpp @@ -45,11 +45,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_vector) { data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnInt64::create(), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { EXPECT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -70,11 +70,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_vector_all_null) { data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnInt64::create(), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { EXPECT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -96,11 +96,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_vector_all_not_null data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnInt64::create(), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { EXPECT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -121,11 +121,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_string) { data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -146,11 +146,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_string_all_null) { data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -172,11 +172,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_string_all_not_null data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -197,11 +197,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_decimal) { data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnDecimal64::create(0, 6), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -222,11 +222,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_decimal_all_null) { data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnDecimal64::create(0, 6), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -248,11 +248,11 @@ TEST(ColumnNullableSerializationTest, column_nullable_column_decimal_all_not_nul data_strs[i] = StringRef(data + i * column_nullable->get_max_row_byte_size(), 0); } - column_nullable->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column = ColumnNullable::create(ColumnDecimal64::create(0, 6), ColumnUInt8::create()); - result_column->deserialize_vec(data_strs, input_rows_count); + result_column->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable->get_data_at(i), result_column->get_data_at(i)) @@ -273,16 +273,16 @@ TEST(ColumnNullableSerializationTest, multiple_columns) { max_row_byte_size += column_nullable_string->get_max_row_byte_size(); Arena arena(4096); - auto data = arena.alloc(input_rows_count * max_row_byte_size); + auto* data = arena.alloc(input_rows_count * max_row_byte_size); std::vector data_strs(input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { data_strs[i] = StringRef(data + i * max_row_byte_size, 0); } - column_nullable_decimal64->serialize_vec(data_strs, input_rows_count, max_row_byte_size); - column_nullable_int64->serialize_vec(data_strs, input_rows_count, max_row_byte_size); - column_nullable_string->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable_decimal64->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); + column_nullable_int64->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); + column_nullable_string->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column_decimal64 = ColumnNullable::create(ColumnDecimal64::create(0, 6), ColumnUInt8::create()); @@ -291,9 +291,9 @@ TEST(ColumnNullableSerializationTest, multiple_columns) { ColumnNullable::MutablePtr result_column_string = ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()); - result_column_decimal64->deserialize_vec(data_strs, input_rows_count); - result_column_int64->deserialize_vec(data_strs, input_rows_count); - result_column_string->deserialize_vec(data_strs, input_rows_count); + result_column_decimal64->deserialize_vec(data_strs.data(), input_rows_count); + result_column_int64->deserialize_vec(data_strs.data(), input_rows_count); + result_column_string->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable_decimal64->get_data_at(i), @@ -321,16 +321,16 @@ TEST(ColumnNullableSerializationTest, multiple_columns_all_null) { max_row_byte_size += column_nullable_string->get_max_row_byte_size(); Arena arena(4096); - auto data = arena.alloc(input_rows_count * max_row_byte_size); + auto* data = arena.alloc(input_rows_count * max_row_byte_size); std::vector data_strs(input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { data_strs[i] = StringRef(data + i * max_row_byte_size, 0); } - column_nullable_decimal64->serialize_vec(data_strs, input_rows_count, max_row_byte_size); - column_nullable_int64->serialize_vec(data_strs, input_rows_count, max_row_byte_size); - column_nullable_string->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable_decimal64->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); + column_nullable_int64->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); + column_nullable_string->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column_decimal64 = ColumnNullable::create(ColumnDecimal64::create(0, 6), ColumnUInt8::create()); @@ -339,9 +339,9 @@ TEST(ColumnNullableSerializationTest, multiple_columns_all_null) { ColumnNullable::MutablePtr result_column_string = ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()); - result_column_decimal64->deserialize_vec(data_strs, input_rows_count); - result_column_int64->deserialize_vec(data_strs, input_rows_count); - result_column_string->deserialize_vec(data_strs, input_rows_count); + result_column_decimal64->deserialize_vec(data_strs.data(), input_rows_count); + result_column_int64->deserialize_vec(data_strs.data(), input_rows_count); + result_column_string->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable_decimal64->get_data_at(i), @@ -377,9 +377,9 @@ TEST(ColumnNullableSerializationTest, multiple_columns_all_not_null) { data_strs[i] = StringRef(data + i * max_row_byte_size, 0); } - column_nullable_decimal64->serialize_vec(data_strs, input_rows_count, max_row_byte_size); - column_nullable_int64->serialize_vec(data_strs, input_rows_count, max_row_byte_size); - column_nullable_string->serialize_vec(data_strs, input_rows_count, max_row_byte_size); + column_nullable_decimal64->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); + column_nullable_int64->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); + column_nullable_string->serialize_vec(data_strs.data(), input_rows_count, max_row_byte_size); ColumnNullable::MutablePtr result_column_decimal64 = ColumnNullable::create(ColumnDecimal64::create(0, 6), ColumnUInt8::create()); @@ -388,9 +388,9 @@ TEST(ColumnNullableSerializationTest, multiple_columns_all_not_null) { ColumnNullable::MutablePtr result_column_string = ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()); - result_column_decimal64->deserialize_vec(data_strs, input_rows_count); - result_column_int64->deserialize_vec(data_strs, input_rows_count); - result_column_string->deserialize_vec(data_strs, input_rows_count); + result_column_decimal64->deserialize_vec(data_strs.data(), input_rows_count); + result_column_int64->deserialize_vec(data_strs.data(), input_rows_count); + result_column_string->deserialize_vec(data_strs.data(), input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { ASSERT_EQ(column_nullable_decimal64->get_data_at(i), diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java index 17942dd04a2102..dbadc7bb5f3662 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java @@ -451,6 +451,7 @@ public class SchemaTable extends Table { new SchemaTable(SystemIdGenerator.getNextId(), "backend_active_tasks", TableType.SCHEMA, builder().column("BE_ID", ScalarType.createType(PrimitiveType.BIGINT)) .column("FE_HOST", ScalarType.createVarchar(256)) + .column("WORKLOAD_GROUP_ID", ScalarType.createType(PrimitiveType.BIGINT)) .column("QUERY_ID", ScalarType.createVarchar(256)) .column("TASK_TIME_MS", ScalarType.createType(PrimitiveType.BIGINT)) .column("TASK_CPU_TIME_MS", ScalarType.createType(PrimitiveType.BIGINT)) @@ -461,6 +462,10 @@ public class SchemaTable extends Table { .column("SHUFFLE_SEND_BYTES", ScalarType.createType(PrimitiveType.BIGINT)) .column("SHUFFLE_SEND_ROWS", ScalarType.createType(PrimitiveType.BIGINT)) .column("QUERY_TYPE", ScalarType.createVarchar(256)) + .column("SPILL_WRITE_BYTES_TO_LOCAL_STORAGE", + ScalarType.createType(PrimitiveType.BIGINT)) + .column("SPILL_READ_BYTES_FROM_LOCAL_STORAGE", + ScalarType.createType(PrimitiveType.BIGINT)) .build())) .put("active_queries", new SchemaTable(SystemIdGenerator.getNextId(), "active_queries", TableType.SCHEMA, builder().column("QUERY_ID", ScalarType.createVarchar(256)) @@ -556,6 +561,7 @@ public class SchemaTable extends Table { .column("CPU_USAGE_PERCENT", ScalarType.createType(PrimitiveType.DOUBLE)) .column("LOCAL_SCAN_BYTES_PER_SECOND", ScalarType.createType(PrimitiveType.BIGINT)) .column("REMOTE_SCAN_BYTES_PER_SECOND", ScalarType.createType(PrimitiveType.BIGINT)) + .column("WRITE_BUFFER_USAGE_BYTES", ScalarType.createType(PrimitiveType.BIGINT)) .build()) ) .put("file_cache_statistics", diff --git a/fe/fe-core/src/main/java/org/apache/doris/plugin/AuditEvent.java b/fe/fe-core/src/main/java/org/apache/doris/plugin/AuditEvent.java index 20c05d982f806e..84d58b28d4888a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/plugin/AuditEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/plugin/AuditEvent.java @@ -113,6 +113,10 @@ public enum EventType { public long scanBytesFromLocalStorage = -1; @AuditField(value = "ScanBytesFromRemoteStorage") public long scanBytesFromRemoteStorage = -1; + @AuditField(value = "SpillWriteBytesToLocalStorage") + public long spillWriteBytesToLocalStorage = -1; + @AuditField(value = "SpillReadBytesFromLocalStorage") + public long spillReadBytesFromLocalStorage = -1; public long pushToAuditLogQueueTime; @@ -277,6 +281,16 @@ public AuditEventBuilder setCommandType(String commandType) { return this; } + public AuditEventBuilder setSpillWriteBytesToLocalStorage(long bytes) { + auditEvent.spillWriteBytesToLocalStorage = bytes; + return this; + } + + public AuditEventBuilder setSpillReadBytesFromLocalStorage(long bytes) { + auditEvent.spillReadBytesFromLocalStorage = bytes; + return this; + } + public AuditEvent build() { return this.auditEvent; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java index 446960f9d56415..acc48accfc584d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/AuditLogHelper.java @@ -205,6 +205,10 @@ private static void logAuditLogImpl(ConnectContext ctx, String origStmt, Stateme .setQueryTime(elapseMs) .setScanBytes(statistics == null ? 0 : statistics.getScanBytes()) .setScanRows(statistics == null ? 0 : statistics.getScanRows()) + .setSpillWriteBytesToLocalStorage(statistics == null ? 0 : + statistics.getSpillWriteBytesToLocalStorage()) + .setSpillReadBytesFromLocalStorage(statistics == null ? 0 : + statistics.getSpillReadBytesFromLocalStorage()) .setCpuTimeMs(statistics == null ? 0 : statistics.getCpuMs()) .setPeakMemoryBytes(statistics == null ? 0 : statistics.getMaxPeakMemoryBytes()) .setReturnRows(ctx.getReturnRows()) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 472f2462e4d39d..777e36de3abcc1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -668,7 +668,7 @@ public void exec() throws Exception { // throw exception during workload group manager. throw new UserException("could not find query queue"); } - queueToken = queryQueue.getToken(); + queueToken = queryQueue.getToken(context.getSessionVariable().wgQuerySlotCount); queueToken.get(DebugUtil.printId(queryId), this.queryOptions.getExecutionTimeout() * 1000); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java index a363aedc0d4824..dd67fe68bd761c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java @@ -475,7 +475,7 @@ private void enqueue(ConnectContext context) throws UserException { // throw exception during workload group manager. throw new UserException("could not find query queue"); } - QueueToken queueToken = queryQueue.getToken(); + QueueToken queueToken = queryQueue.getToken(context.getSessionVariable().wgQuerySlotCount); int queryTimeout = coordinatorContext.queryOptions.getExecutionTimeout() * 1000; coordinatorContext.setQueueInfo(queryQueue, queueToken); queueToken.get(DebugUtil.printId(coordinatorContext.queryId), queryTimeout); 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 0b5a6f785ec4c2..92a01fbf9a21d3 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 @@ -404,9 +404,6 @@ public class SessionVariable implements Serializable, Writable { public static final String INTERNAL_SESSION = "internal_session"; - public static final String PARTITIONED_HASH_JOIN_ROWS_THRESHOLD = "partitioned_hash_join_rows_threshold"; - public static final String PARTITIONED_HASH_AGG_ROWS_THRESHOLD = "partitioned_hash_agg_rows_threshold"; - public static final String PARTITION_PRUNING_EXPAND_THRESHOLD = "partition_pruning_expand_threshold"; public static final String ENABLE_SHARE_HASH_TABLE_FOR_BROADCAST_JOIN @@ -568,15 +565,20 @@ public class SessionVariable implements Serializable, Writable { public static final String HUGE_TABLE_LOWER_BOUND_SIZE_IN_BYTES = "huge_table_lower_bound_size_in_bytes"; // for spill to disk - public static final String EXTERNAL_SORT_BYTES_THRESHOLD = "external_sort_bytes_threshold"; - public static final String EXTERNAL_AGG_PARTITION_BITS = "external_agg_partition_bits"; - public static final String SPILL_STREAMING_AGG_MEM_LIMIT = "spill_streaming_agg_mem_limit"; - public static final String MIN_REVOCABLE_MEM = "min_revocable_mem"; - public static final String ENABLE_JOIN_SPILL = "enable_join_spill"; - public static final String ENABLE_SORT_SPILL = "enable_sort_spill"; - public static final String ENABLE_AGG_SPILL = "enable_agg_spill"; + public static final String ENABLE_SPILL = "enable_spill"; 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_REVOCABLE_MEMORY_HIGH_WATERMARK_PERCENT = + "spill_revocable_memory_high_watermark_percent"; public static final String DATA_QUEUE_MAX_BLOCKS = "data_queue_max_blocks"; + public static final String LOW_MEMORY_MODE_BUFFER_LIMIT = "low_memory_mode_buffer_limit"; public static final String FUZZY_DISABLE_RUNTIME_FILTER_IN_BE = "fuzzy_disable_runtime_filter_in_be"; @@ -654,6 +656,10 @@ public class SessionVariable implements Serializable, Writable { public static final String BYPASS_WORKLOAD_GROUP = "bypass_workload_group"; + public static final String QUERY_SLOT_COUNT = "query_slot_count"; + + public static final String ENABLE_MEM_OVERCOMMIT = "enable_mem_overcommit"; + public static final String MAX_COLUMN_READER_NUM = "max_column_reader_num"; public static final String USE_MAX_LENGTH_OF_VARCHAR_IN_CTAS = "use_max_length_of_varchar_in_ctas"; @@ -696,6 +702,8 @@ public class SessionVariable implements Serializable, Writable { "adaptive_pipeline_task_serial_read_on_limit"; public static final String REQUIRE_SEQUENCE_IN_INSERT = "require_sequence_in_insert"; + public static final String MINIMUM_OPERATOR_MEMORY_REQUIRED_KB = "minimum_operator_memory_required_kb"; + public static final String ENABLE_PHRASE_QUERY_SEQUENYIAL_OPT = "enable_phrase_query_sequential_opt"; public static final String ENABLE_COOLDOWN_REPLICA_AFFINITY = @@ -750,7 +758,7 @@ public class SessionVariable implements Serializable, Writable { public long insertVisibleTimeoutMs = DEFAULT_INSERT_VISIBLE_TIMEOUT_MS; // max memory used on every backend. - @VariableMgr.VarAttr(name = EXEC_MEM_LIMIT) + @VariableMgr.VarAttr(name = EXEC_MEM_LIMIT, needForward = true) public long maxExecMemByte = 2147483648L; @VariableMgr.VarAttr(name = SCAN_QUEUE_MEM_LIMIT, @@ -844,6 +852,25 @@ public class SessionVariable implements Serializable, Writable { "whether bypass workload group's limitation, currently only support bypass query queue"}) public boolean bypassWorkloadGroup = false; + @VariableMgr.VarAttr(name = QUERY_SLOT_COUNT, needForward = true, checker = "checkQuerySlotCount", + description = { + "每个查询占用的slot的数量,workload group的query slot的总数等于设置的最大并发数", + "Number of slots occupied by each query, the total number of query slots " + + "of the workload group equals the maximum number of concurrent requests"}) + public int wgQuerySlotCount = 1; + + public void checkQuerySlotCount(String slotCnt) { + Long slotCount = Long.valueOf(slotCnt); + if (slotCount < 1 || slotCount > 1025) { + throw new InvalidParameterException("query_slot_count should be between 1 and 1024)"); + } + } + + @VariableMgr.VarAttr(name = ENABLE_MEM_OVERCOMMIT, needForward = true, description = { + "是否通过硬限的方式来计算每个Query的内存资源", + "Whether to calculate the memory resources of each query by hard limit"}) + public boolean enableMemOvercommit = true; + @VariableMgr.VarAttr(name = MAX_COLUMN_READER_NUM) public int maxColumnReaderNum = 20000; @@ -1611,14 +1638,6 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = INTERNAL_SESSION) public boolean internalSession = false; - // Use partitioned hash join if build side row count >= the threshold . 0 - the threshold is not set. - @VariableMgr.VarAttr(name = PARTITIONED_HASH_JOIN_ROWS_THRESHOLD, fuzzy = true) - public int partitionedHashJoinRowsThreshold = 0; - - // Use partitioned hash join if build side row count >= the threshold . 0 - the threshold is not set. - @VariableMgr.VarAttr(name = PARTITIONED_HASH_AGG_ROWS_THRESHOLD, fuzzy = true) - public int partitionedHashAggRowsThreshold = 0; - @VariableMgr.VarAttr(name = PARTITION_PRUNING_EXPAND_THRESHOLD, fuzzy = true) public int partitionPruningExpandThreshold = 10; @@ -2134,6 +2153,11 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { description = {"对外表启用 count(*) 下推优化", "enable count(*) pushdown optimization for external table"}) private boolean enableCountPushDownForExternalTable = true; + @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 static final String IGNORE_RUNTIME_FILTER_IDS = "ignore_runtime_filter_ids"; public Set getIgnoredRuntimeFilterIds() { @@ -2215,10 +2239,6 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { public boolean disableEmptyPartitionPrune = false; // CLOUD_VARIABLES_END - // for spill to disk - @VariableMgr.VarAttr(name = MIN_REVOCABLE_MEM, fuzzy = true) - public long minRevocableMem = 32 * 1024 * 1024; - // fetch remote schema rpc timeout @VariableMgr.VarAttr(name = FETCH_REMOTE_SCHEMA_TIMEOUT_SECONDS, fuzzy = true) public long fetchRemoteSchemaTimeoutSeconds = 120; @@ -2226,22 +2246,6 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { @VariableMgr.VarAttr(name = MAX_FETCH_REMOTE_TABLET_COUNT, fuzzy = true) public int maxFetchRemoteTabletCount = 512; - @VariableMgr.VarAttr( - name = ENABLE_JOIN_SPILL, - description = {"控制是否启用join算子落盘。默认为 false。", - "Controls whether to enable spill to disk of join operation. " - + "The default value is false."}, - needForward = true, fuzzy = true) - public boolean enableJoinSpill = false; - - @VariableMgr.VarAttr( - name = ENABLE_SORT_SPILL, - description = {"控制是否启用排序算子落盘。默认为 false。", - "Controls whether to enable spill to disk of sort operation. " - + "The default value is false."}, - needForward = true, fuzzy = true) - public boolean enableSortSpill = false; - @VariableMgr.VarAttr( name = "enable_compress_materialize", description = {"控制是否启用compress materialize。", @@ -2252,12 +2256,20 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { public boolean enableCompressMaterialize = false; @VariableMgr.VarAttr( - name = ENABLE_AGG_SPILL, - description = {"控制是否启用聚合算子落盘。默认为 false。", - "Controls whether to enable spill to disk of aggregation operation. " + name = DATA_QUEUE_MAX_BLOCKS, + description = {"DataQueue 中每个子队列允许最大的 block 个数", + "Max blocks in DataQueue."}, + needForward = true, fuzzy = true) + public long dataQueueMaxBlocks = 1; + + // for spill to disk + @VariableMgr.VarAttr( + name = ENABLE_SPILL, + description = {"控制是否启用查询算子落盘。默认为 false。", + "Controls whether to enable spill to disk for query. " + "The default value is false."}, needForward = true, fuzzy = true) - public boolean enableAggSpill = false; + public boolean enableSpill = false; @VariableMgr.VarAttr( name = ENABLE_FORCE_SPILL, @@ -2269,11 +2281,25 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { public boolean enableForceSpill = false; @VariableMgr.VarAttr( - name = DATA_QUEUE_MAX_BLOCKS, - description = {"DataQueue 中每个子队列允许最大的 block 个数", - "Max blocks in DataQueue."}, + name = ENABLE_RESERVE_MEMORY, + description = {"控制是否启用分配内存前先reverve memory的功能。默认为 true。", + "Controls whether to enable reserve memory before allocating memory. " + + "The default value is true."}, needForward = true, fuzzy = true) - public long dataQueueMaxBlocks = 1; + public boolean enableReserveMemory = true; + + @VariableMgr.VarAttr(name = SPILL_MIN_REVOCABLE_MEM, fuzzy = true) + public long spillMinRevocableMem = 32 * 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 = FUZZY_DISABLE_RUNTIME_FILTER_IN_BE, @@ -2282,23 +2308,22 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { needForward = true, fuzzy = false) public boolean fuzzyDisableRuntimeFilterInBE = false; - // If the memory consumption of sort node exceed this limit, will trigger spill to disk; - // Set to 0 to disable; min: 128M - public static final long MIN_EXTERNAL_SORT_BYTES_THRESHOLD = 2097152; - @VariableMgr.VarAttr(name = EXTERNAL_SORT_BYTES_THRESHOLD, - checker = "checkExternalSortBytesThreshold", varType = VariableAnnotation.DEPRECATED) - public long externalSortBytesThreshold = 0; + @VariableMgr.VarAttr(name = SPILL_AGGREGATION_PARTITION_COUNT, fuzzy = true) + public int spillAggregationPartitionCount = 32; + + @VariableMgr.VarAttr(name = LOW_MEMORY_MODE_BUFFER_LIMIT, fuzzy = false) + public long lowMemoryModeBufferLimit = 33554432; // The memory limit of streaming agg when spilling is enabled // NOTE: streaming agg operator will not spill to disk. @VariableMgr.VarAttr(name = SPILL_STREAMING_AGG_MEM_LIMIT, fuzzy = true) public long spillStreamingAggMemLimit = 268435456; //256MB - public static final int MIN_EXTERNAL_AGG_PARTITION_BITS = 4; - public static final int MAX_EXTERNAL_AGG_PARTITION_BITS = 20; - @VariableMgr.VarAttr(name = EXTERNAL_AGG_PARTITION_BITS, - checker = "checkExternalAggPartitionBits", fuzzy = true) - public int externalAggPartitionBits = 5; // means that the hash table will be partitioned into 32 blocks. + @VariableMgr.VarAttr(name = SPILL_HASH_JOIN_PARTITION_COUNT, fuzzy = true) + public int spillHashJoinPartitionCount = 32; + + @VariableMgr.VarAttr(name = SPILL_REVOCABLE_MEMORY_HIGH_WATERMARK_PERCENT, fuzzy = true) + public int spillRevocableMemoryHighWatermarkPercent = -1; @VariableMgr.VarAttr(name = USE_MAX_LENGTH_OF_VARCHAR_IN_CTAS, needForward = true, description = { "在CTAS中,如果 CHAR / VARCHAR 列不来自于源表,是否是将这一列的长度设置为 MAX,即65533。默认为 true。", @@ -2416,18 +2441,6 @@ public boolean isEnableESParallelScroll() { return enableESParallelScroll; } - public boolean isEnableJoinSpill() { - return enableJoinSpill; - } - - public void setEnableJoinSpill(boolean enableJoinSpill) { - this.enableJoinSpill = enableJoinSpill; - } - - public boolean isEnableSortSpill() { - return enableSortSpill; - } - // If this fe is in fuzzy mode, then will use initFuzzyModeVariables to generate some variables, // not the default value set in the code. @SuppressWarnings("checkstyle:Indentation") @@ -2444,8 +2457,6 @@ public void initFuzzyModeVariables() { // this.disableJoinReorder = random.nextBoolean(); this.enableCommonExpPushDownForInvertedIndex = random.nextBoolean(); this.disableStreamPreaggregations = random.nextBoolean(); - this.partitionedHashJoinRowsThreshold = random.nextBoolean() ? 8 : 1048576; - this.partitionedHashAggRowsThreshold = random.nextBoolean() ? 8 : 1048576; this.enableShareHashTableForBroadcastJoin = random.nextBoolean(); // this.enableHashJoinEarlyStartProbe = random.nextBoolean(); this.enableParallelResultSink = random.nextBoolean(); @@ -2465,23 +2476,23 @@ public void initFuzzyModeVariables() { /* switch (randomInt) { case 0: - this.externalSortBytesThreshold = 0; + this.spillSortBytesThreshold = 0; this.externalAggBytesThreshold = 0; break; case 1: - this.externalSortBytesThreshold = 1; + this.spillSortBytesThreshold = 1; this.externalAggBytesThreshold = 1; - this.externalAggPartitionBits = 6; + this.spillAggregationPartitionCount = 6; break; case 2: - this.externalSortBytesThreshold = 1024 * 1024; + this.spillSortBytesThreshold = 1024 * 1024; this.externalAggBytesThreshold = 1024 * 1024; - this.externalAggPartitionBits = 8; + this.spillAggregationPartitionCount = 8; break; default: - this.externalSortBytesThreshold = 100 * 1024 * 1024 * 1024; + this.spillSortBytesThreshold = 100 * 1024 * 1024 * 1024; this.externalAggBytesThreshold = 100 * 1024 * 1024 * 1024; - this.externalAggPartitionBits = 4; + this.spillAggregationPartitionCount = 4; break; } */ @@ -2561,29 +2572,26 @@ public void initFuzzyModeVariables() { // for spill to disk if (Config.pull_request_id > 10000) { if (Config.pull_request_id % 2 == 0) { - this.enableJoinSpill = true; - this.enableSortSpill = true; - this.enableAggSpill = true; - + this.enableSpill = true; + this.enableReserveMemory = true; randomInt = random.nextInt(4); switch (randomInt) { case 0: - this.minRevocableMem = 0; + this.spillMinRevocableMem = 0; break; case 1: - this.minRevocableMem = 1; + this.spillMinRevocableMem = 1; break; case 2: - this.minRevocableMem = 1024 * 1024; + this.spillMinRevocableMem = 1024 * 1024; break; default: - this.minRevocableMem = 100L * 1024 * 1024 * 1024; + this.spillMinRevocableMem = 100L * 1024 * 1024 * 1024; break; } } else { - this.enableJoinSpill = false; - this.enableSortSpill = false; - this.enableAggSpill = false; + this.enableSpill = false; + this.enableReserveMemory = false; } } } @@ -3145,14 +3153,6 @@ public void setQueryCacheEntryMaxRows(long queryCacheEntryMaxRows) { this.queryCacheEntryMaxRows = queryCacheEntryMaxRows; } - public int getPartitionedHashJoinRowsThreshold() { - return partitionedHashJoinRowsThreshold; - } - - public void setPartitionedHashJoinRowsThreshold(int threshold) { - this.partitionedHashJoinRowsThreshold = threshold; - } - // Serialize to thrift object public boolean getForwardToMaster() { return forwardToMaster; @@ -3767,24 +3767,6 @@ public boolean isDropTableIfCtasFailed() { return dropTableIfCtasFailed; } - public void checkExternalSortBytesThreshold(String externalSortBytesThreshold) { - long value = Long.valueOf(externalSortBytesThreshold); - if (value > 0 && value < MIN_EXTERNAL_SORT_BYTES_THRESHOLD) { - LOG.warn("external sort bytes threshold: {}, min: {}", value, MIN_EXTERNAL_SORT_BYTES_THRESHOLD); - throw new UnsupportedOperationException("minimum value is " + MIN_EXTERNAL_SORT_BYTES_THRESHOLD); - } - } - - public void checkExternalAggPartitionBits(String externalAggPartitionBits) { - int value = Integer.valueOf(externalAggPartitionBits); - if (value < MIN_EXTERNAL_AGG_PARTITION_BITS || value > MAX_EXTERNAL_AGG_PARTITION_BITS) { - LOG.warn("external agg bytes threshold: {}, min: {}, max: {}", - value, MIN_EXTERNAL_AGG_PARTITION_BITS, MAX_EXTERNAL_AGG_PARTITION_BITS); - throw new UnsupportedOperationException("min value is " + MIN_EXTERNAL_AGG_PARTITION_BITS + " max value is " - + MAX_EXTERNAL_AGG_PARTITION_BITS); - } - } - public void checkQueryTimeoutValid(String newQueryTimeout) { int value = Integer.valueOf(newQueryTimeout); if (value <= 0) { @@ -4004,15 +3986,6 @@ public TQueryOptions toThrift() { tResult.setSkipDeleteBitmap(skipDeleteBitmap); - tResult.setPartitionedHashJoinRowsThreshold(partitionedHashJoinRowsThreshold); - tResult.setPartitionedHashAggRowsThreshold(partitionedHashAggRowsThreshold); - - tResult.setExternalSortBytesThreshold(externalSortBytesThreshold); - - tResult.setExternalAggBytesThreshold(0); // disable for now - - tResult.setSpillStreamingAggMemLimit(spillStreamingAggMemLimit); - tResult.setEnableFileCache(enableFileCache); tResult.setEnablePageCache(enablePageCache); @@ -4050,13 +4023,22 @@ public TQueryOptions toThrift() { tResult.setParallelScanMinRowsPerScanner(parallelScanMinRowsPerScanner); tResult.setSkipBadTablet(skipBadTablet); tResult.setDisableFileCache(disableFileCache); - tResult.setEnableJoinSpill(enableJoinSpill); - tResult.setEnableSortSpill(enableSortSpill); - tResult.setEnableAggSpill(enableAggSpill); + + // for spill + tResult.setEnableSpill(enableSpill); tResult.setEnableForceSpill(enableForceSpill); - tResult.setMinRevocableMem(minRevocableMem); + 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.setDataQueueMaxBlocks(dataQueueMaxBlocks); tResult.setFuzzyDisableRuntimeFilterInBe(fuzzyDisableRuntimeFilterInBE); + tResult.setLowMemoryModeBufferLimit(lowMemoryModeBufferLimit); tResult.setEnableLocalMergeSort(enableLocalMergeSort); tResult.setEnableSharedExchangeSinkBuffer(enableSharedExchangeSinkBuffer); @@ -4072,6 +4054,9 @@ public TQueryOptions toThrift() { tResult.setEnableInvertedIndexQueryCache(enableInvertedIndexQueryCache); tResult.setHiveOrcUseColumnNames(hiveOrcUseColumnNames); tResult.setHiveParquetUseColumnNames(hiveParquetUseColumnNames); + tResult.setQuerySlotCount(wgQuerySlotCount); + tResult.setEnableMemOvercommit(enableMemOvercommit); + tResult.setKeepCarriageReturn(keepCarriageReturn); tResult.setEnableSegmentCache(enableSegmentCache); @@ -4088,6 +4073,7 @@ public TQueryOptions toThrift() { tResult.setIgnoreRuntimeFilterError(ignoreRuntimeFilterError); tResult.setEnableFixedLenToUint32V2(enableFixedLenToUint32V2); + tResult.setMinimumOperatorMemoryRequiredKb(minimumOperatorMemoryRequiredKB); return tResult; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueryQueue.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueryQueue.java index ba2d2526f2e751..82822c05a0d995 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueryQueue.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueryQueue.java @@ -26,6 +26,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Iterator; import java.util.LinkedList; import java.util.PriorityQueue; import java.util.Queue; @@ -98,24 +99,43 @@ public String debugString() { + runningQueryQueue.size() + ", currentWaitingQueryNum=" + waitingQueryQueue.size(); } - public QueueToken getToken() throws UserException { + public int usedSlotCount() { + int cnt = 0; + for (Iterator iterator = runningQueryQueue.iterator(); iterator.hasNext();) { + QueueToken queueToken = (QueueToken) iterator.next(); + cnt += queueToken.getQuerySlotCount(); + } + return cnt; + } + + public QueueToken getToken(int querySlotCount) throws UserException { + if (maxConcurrency > 0 && (querySlotCount > maxConcurrency || querySlotCount < 1)) { + throw new UserException("query slot count " + querySlotCount + + " should be smaller than workload group's max concurrency " + + maxConcurrency + " and > 0"); + } AdmissionControl admissionControl = Env.getCurrentEnv().getAdmissionControl(); queueLock.lock(); try { if (LOG.isDebugEnabled()) { LOG.info(this.debugString()); } - QueueToken queueToken = new QueueToken(queueTimeout, this); + QueueToken queueToken = new QueueToken(queueTimeout, querySlotCount, this); boolean isReachMaxCon = runningQueryQueue.size() >= maxConcurrency; + boolean hasFreeSlot = queueToken.getQuerySlotCount() <= maxConcurrency - usedSlotCount(); boolean isResourceAvailable = admissionControl.checkResourceAvailable(queueToken); - if (!isReachMaxCon && isResourceAvailable) { + if (!isReachMaxCon && isResourceAvailable && hasFreeSlot) { runningQueryQueue.offer(queueToken); queueToken.complete(); return queueToken; } else if (waitingQueryQueue.size() >= maxQueueSize) { - throw new UserException("query waiting queue is full, queue length=" + maxQueueSize); + throw new UserException("query waiting queue is full, queue capacity=" + maxQueueSize + + ", waiting num=" + waitingQueryQueue.size()); } else { + if (!hasFreeSlot) { + queueToken.setQueueMsg("NO_FREE_SLOT"); + } if (isReachMaxCon) { queueToken.setQueueMsg("WAIT_IN_QUEUE"); } @@ -145,14 +165,19 @@ public void releaseAndNotify(QueueToken releaseToken) { AdmissionControl admissionControl = Env.getCurrentEnv().getAdmissionControl(); queueLock.lock(); try { - runningQueryQueue.remove(releaseToken); - waitingQueryQueue.remove(releaseToken); - admissionControl.removeQueueToken(releaseToken); + if (releaseToken != null) { + runningQueryQueue.remove(releaseToken); + waitingQueryQueue.remove(releaseToken); + admissionControl.removeQueueToken(releaseToken); + } while (runningQueryQueue.size() < maxConcurrency) { QueueToken queueToken = waitingQueryQueue.peek(); if (queueToken == null) { break; } + if (queueToken.getQuerySlotCount() > maxConcurrency - usedSlotCount()) { + break; + } if (admissionControl.checkResourceAvailable(queueToken)) { queueToken.complete(); runningQueryQueue.offer(queueToken); diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueueToken.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueueToken.java index 748c0c21bda1ba..67afcca1340333 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueueToken.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/QueueToken.java @@ -54,6 +54,7 @@ public enum TokenState { private long queueStartTime = -1; private long queueEndTime = -1; + private final int querySlotCount; private volatile String queueMsg = ""; @@ -62,10 +63,11 @@ public enum TokenState { // Object is just a placeholder, it's meaningless now private CompletableFuture future; - public QueueToken(long queueWaitTimeout, QueryQueue queryQueue) { + public QueueToken(long queueWaitTimeout, int querySlotCount, QueryQueue queryQueue) { this.tokenId = tokenIdGenerator.addAndGet(1); this.queueWaitTimeout = queueWaitTimeout; this.queueStartTime = System.currentTimeMillis(); + this.querySlotCount = querySlotCount; this.queryQueue = queryQueue; this.future = new CompletableFuture<>(); } @@ -146,4 +148,8 @@ public boolean equals(Object obj) { public long getTokenId() { return tokenId; } + + public int getQuerySlotCount() { + return querySlotCount; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroup.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroup.java index 2bd4a394d84dac..7852d3b6302cfb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroup.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroup.java @@ -29,6 +29,7 @@ import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.thrift.TPipelineWorkloadGroup; +import org.apache.doris.thrift.TWgSlotMemoryPolicy; import org.apache.doris.thrift.TWorkloadGroupInfo; import org.apache.doris.thrift.TWorkloadType; import org.apache.doris.thrift.TopicInfo; @@ -63,6 +64,8 @@ public class WorkloadGroup implements Writable, GsonPostProcessable { public static final String ENABLE_MEMORY_OVERCOMMIT = "enable_memory_overcommit"; + public static final String WRITE_BUFFER_RATIO = "write_buffer_ratio"; + public static final String MAX_CONCURRENCY = "max_concurrency"; public static final String MAX_QUEUE_SIZE = "max_queue_size"; @@ -79,6 +82,8 @@ public class WorkloadGroup implements Writable, GsonPostProcessable { public static final String MEMORY_HIGH_WATERMARK = "memory_high_watermark"; + public static final String SLOT_MEMORY_POLICY = "slot_memory_policy"; + public static final String TAG = "tag"; public static final String READ_BYTES_PER_SECOND = "read_bytes_per_second"; @@ -102,7 +107,8 @@ public class WorkloadGroup implements Writable, GsonPostProcessable { .add(MAX_QUEUE_SIZE).add(QUEUE_TIMEOUT).add(CPU_HARD_LIMIT).add(SCAN_THREAD_NUM) .add(MAX_REMOTE_SCAN_THREAD_NUM).add(MIN_REMOTE_SCAN_THREAD_NUM) .add(MEMORY_LOW_WATERMARK).add(MEMORY_HIGH_WATERMARK) - .add(TAG).add(READ_BYTES_PER_SECOND).add(REMOTE_READ_BYTES_PER_SECOND).add(INTERNAL_TYPE).build(); + .add(TAG).add(READ_BYTES_PER_SECOND).add(REMOTE_READ_BYTES_PER_SECOND) + .add(WRITE_BUFFER_RATIO).add(SLOT_MEMORY_POLICY).add(INTERNAL_TYPE).build(); private static final ImmutableMap DEPRECATED_PROPERTIES_NAME = new ImmutableMap.Builder() @@ -112,8 +118,8 @@ public class WorkloadGroup implements Writable, GsonPostProcessable { public static final ImmutableMap WORKLOAD_TYPE_MAP = new ImmutableMap.Builder() .put(TWorkloadType.INTERNAL.toString().toLowerCase(), TWorkloadType.INTERNAL.getValue()).build(); - public static final int MEMORY_LOW_WATERMARK_DEFAULT_VALUE = 50; - public static final int MEMORY_HIGH_WATERMARK_DEFAULT_VALUE = 80; + public static final int MEMORY_LOW_WATERMARK_DEFAULT_VALUE = 80; + public static final int MEMORY_HIGH_WATERMARK_DEFAULT_VALUE = 95; private static final Map ALL_PROPERTIES_DEFAULT_VALUE_MAP = Maps.newHashMap(); @@ -128,13 +134,21 @@ public class WorkloadGroup implements Writable, GsonPostProcessable { ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(SCAN_THREAD_NUM, "-1"); ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(MAX_REMOTE_SCAN_THREAD_NUM, "-1"); ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(MIN_REMOTE_SCAN_THREAD_NUM, "-1"); - ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(MEMORY_LOW_WATERMARK, "50%"); - ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(MEMORY_HIGH_WATERMARK, "80%"); + ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(MEMORY_LOW_WATERMARK, "80%"); + ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(MEMORY_HIGH_WATERMARK, "95%"); ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(TAG, ""); ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(READ_BYTES_PER_SECOND, "-1"); ALL_PROPERTIES_DEFAULT_VALUE_MAP.put(REMOTE_READ_BYTES_PER_SECOND, "-1"); } + public static final int WRITE_BUFFER_RATIO_DEFAULT_VALUE = 20; + public static final String SLOT_MEMORY_POLICY_DEFAULT_VALUE = "none"; + public static final HashSet AVAILABLE_SLOT_MEMORY_POLICY_VALUES = new HashSet() {{ + add("none"); + add("fixed"); + add("dynamic"); + }}; + @SerializedName(value = "id") private long id; @@ -167,6 +181,24 @@ private WorkloadGroup(long id, String name, Map properties, long if (properties.containsKey(MEMORY_LIMIT)) { setMemLimitPercent(properties); } + + if (properties.containsKey(WRITE_BUFFER_RATIO)) { + String loadBufLimitStr = properties.get(WRITE_BUFFER_RATIO); + if (loadBufLimitStr.endsWith("%")) { + loadBufLimitStr = loadBufLimitStr.substring(0, loadBufLimitStr.length() - 1); + } + this.properties.put(WRITE_BUFFER_RATIO, loadBufLimitStr); + } else { + this.properties.put(WRITE_BUFFER_RATIO, WRITE_BUFFER_RATIO_DEFAULT_VALUE + ""); + } + + if (properties.containsKey(SLOT_MEMORY_POLICY)) { + String slotPolicy = properties.get(SLOT_MEMORY_POLICY); + this.properties.put(SLOT_MEMORY_POLICY, slotPolicy); + } else { + this.properties.put(SLOT_MEMORY_POLICY, SLOT_MEMORY_POLICY_DEFAULT_VALUE); + } + if (properties.containsKey(ENABLE_MEMORY_OVERCOMMIT)) { properties.put(ENABLE_MEMORY_OVERCOMMIT, properties.get(ENABLE_MEMORY_OVERCOMMIT).toLowerCase()); } @@ -307,6 +339,25 @@ private static void checkProperties(Map properties) throws DdlEx } } + if (properties.containsKey(WRITE_BUFFER_RATIO)) { + String writeBufSizeStr = properties.get(WRITE_BUFFER_RATIO); + String memLimitErr = WRITE_BUFFER_RATIO + " " + writeBufSizeStr + + " requires a positive int number."; + if (writeBufSizeStr.endsWith("%")) { + writeBufSizeStr = writeBufSizeStr.substring(0, writeBufSizeStr.length() - 1); + } + try { + if (Integer.parseInt(writeBufSizeStr) < 0) { + throw new DdlException(memLimitErr); + } + } catch (NumberFormatException e) { + if (LOG.isDebugEnabled()) { + LOG.debug(memLimitErr, e); + } + throw new DdlException(memLimitErr); + } + } + if (properties.containsKey(ENABLE_MEMORY_OVERCOMMIT)) { String inputValue = properties.get(ENABLE_MEMORY_OVERCOMMIT); String value = inputValue.toLowerCase(); @@ -317,6 +368,14 @@ private static void checkProperties(Map properties) throws DdlEx } } + if (properties.containsKey(SLOT_MEMORY_POLICY)) { + String value = properties.get(SLOT_MEMORY_POLICY).toLowerCase(); + if (!AVAILABLE_SLOT_MEMORY_POLICY_VALUES.contains(value)) { + throw new DdlException("The value of '" + SLOT_MEMORY_POLICY + + "' must be one of none, fixed, dynamic."); + } + } + if (properties.containsKey(SCAN_THREAD_NUM)) { String value = properties.get(SCAN_THREAD_NUM); try { @@ -607,6 +666,18 @@ public TPipelineWorkloadGroup toThrift() { return new TPipelineWorkloadGroup().setId(id); } + public static TWgSlotMemoryPolicy findSlotPolicyValueByString(String slotPolicy) { + if (slotPolicy.equalsIgnoreCase("none")) { + return TWgSlotMemoryPolicy.NONE; + } else if (slotPolicy.equalsIgnoreCase("fixed")) { + return TWgSlotMemoryPolicy.FIXED; + } else if (slotPolicy.equalsIgnoreCase("dynamic")) { + return TWgSlotMemoryPolicy.DYNAMIC; + } else { + throw new RuntimeException("Could not find policy using " + slotPolicy); + } + } + public TopicInfo toTopicInfo() { TWorkloadGroupInfo tWorkloadGroupInfo = new TWorkloadGroupInfo(); long wgId = this.id; @@ -633,6 +704,14 @@ public TopicInfo toTopicInfo() { if (memLimitStr != null) { tWorkloadGroupInfo.setMemLimit(memLimitStr); } + String writeBufferRatioStr = properties.get(WRITE_BUFFER_RATIO); + if (writeBufferRatioStr != null) { + tWorkloadGroupInfo.setWriteBufferRatio(Integer.parseInt(writeBufferRatioStr)); + } + String slotMemoryPolicyStr = properties.get(SLOT_MEMORY_POLICY); + if (slotMemoryPolicyStr != null) { + tWorkloadGroupInfo.setSlotMemoryPolicy(findSlotPolicyValueByString(slotMemoryPolicyStr)); + } String memOvercommitStr = properties.get(ENABLE_MEMORY_OVERCOMMIT); if (memOvercommitStr != null) { tWorkloadGroupInfo.setEnableMemoryOvercommit(Boolean.valueOf(memOvercommitStr)); @@ -686,6 +765,11 @@ public TopicInfo toTopicInfo() { tWorkloadGroupInfo.setTag(tagStr); } + String totalQuerySlotCountStr = properties.get(MAX_CONCURRENCY); + if (totalQuerySlotCountStr != null) { + tWorkloadGroupInfo.setTotalQuerySlotCount(Integer.parseInt(totalQuerySlotCountStr)); + } + TopicInfo topicInfo = new TopicInfo(); topicInfo.setWorkloadGroupInfo(tWorkloadGroupInfo); return topicInfo; diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java index 94695cc8d5b8ac..f2b500e8260609 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadgroup/WorkloadGroupMgr.java @@ -81,6 +81,8 @@ public class WorkloadGroupMgr extends MasterDaemon implements Writable, GsonPost public static final ImmutableList WORKLOAD_GROUP_PROC_NODE_TITLE_NAMES = new ImmutableList.Builder() .add("Id").add("Name").add(WorkloadGroup.CPU_SHARE).add(WorkloadGroup.MEMORY_LIMIT) .add(WorkloadGroup.ENABLE_MEMORY_OVERCOMMIT) + .add(WorkloadGroup.WRITE_BUFFER_RATIO) + .add(WorkloadGroup.SLOT_MEMORY_POLICY) .add(WorkloadGroup.MAX_CONCURRENCY).add(WorkloadGroup.MAX_QUEUE_SIZE) .add(WorkloadGroup.QUEUE_TIMEOUT).add(WorkloadGroup.CPU_HARD_LIMIT) .add(WorkloadGroup.SCAN_THREAD_NUM).add(WorkloadGroup.MAX_REMOTE_SCAN_THREAD_NUM) diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadRuntimeStatusMgr.java b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadRuntimeStatusMgr.java index 695bf983dc6b2f..d671e97271a754 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadRuntimeStatusMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/resource/workloadschedpolicy/WorkloadRuntimeStatusMgr.java @@ -86,6 +86,8 @@ protected void runAfterCatalogReady() { auditEvent.cpuTimeMs = queryStats.cpu_ms; auditEvent.shuffleSendBytes = queryStats.shuffle_send_bytes; auditEvent.shuffleSendRows = queryStats.shuffle_send_rows; + auditEvent.spillWriteBytesToLocalStorage = queryStats.spill_write_bytes_to_local_storage; + auditEvent.spillReadBytesFromLocalStorage = queryStats.spill_read_bytes_from_local_storage; } boolean ret = Env.getCurrentAuditEventProcessor().handleAuditEvent(auditEvent, true); if (!ret) { @@ -226,6 +228,8 @@ private void mergeQueryStatistics(TQueryStatistics dst, TQueryStatistics src) { if (dst.max_peak_memory_bytes < src.max_peak_memory_bytes) { dst.max_peak_memory_bytes = src.max_peak_memory_bytes; } + dst.spill_write_bytes_to_local_storage += src.spill_write_bytes_to_local_storage; + dst.spill_read_bytes_from_local_storage += src.spill_read_bytes_from_local_storage; } private void queryAuditEventLogWriteLock() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index 2becdf802d283a..8ee897819cc3e7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -616,23 +616,25 @@ private static TFetchSchemaTableDataResult workloadGroupsMetadataResult(TSchemaT trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(2)))); // cpu_share trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(3))); // mem_limit trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(4))); // mem overcommit - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(5)))); // max concurrent - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(6)))); // max queue size - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(7)))); // queue timeout - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(8))); // cpu hard limit - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(9)))); // scan thread num + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(5)))); // write_buffer_ratio + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(6))); // slot_memory_policy + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(7)))); // max concurrent + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(8)))); // max queue size + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(9)))); // queue timeout + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(10))); // cpu hard limit + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(11)))); // scan thread num // max remote scan thread num - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(10)))); + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(12)))); // min remote scan thread num - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(11)))); - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(12))); // spill low watermark - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(13))); // spill high watermark - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(14))); // tag - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(15)))); // read bytes per second + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(13)))); + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(14))); // spill low watermark + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(15))); // spill high watermark + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(16))); // tag + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(17)))); // read bytes per second trow.addToColumnValue( - new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(16)))); // remote read bytes per second - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(17))); // running query num - trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(18))); // waiting query num + new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(18)))); // remote read bytes per second + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(19))); // running query num + trow.addToColumnValue(new TCell().setStringVal(rGroupsInfo.get(20))); // waiting query num dataBatch.add(trow); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/resource/workloadgroup/WorkloadGroupTest.java b/fe/fe-core/src/test/java/org/apache/doris/resource/workloadgroup/WorkloadGroupTest.java index 121f46fe75be99..cc6ae036c2cf06 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/resource/workloadgroup/WorkloadGroupTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/resource/workloadgroup/WorkloadGroupTest.java @@ -19,6 +19,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.proc.BaseProcResult; +import org.apache.doris.thrift.TWgSlotMemoryPolicy; import com.google.common.collect.Maps; import org.junit.Assert; @@ -87,4 +88,23 @@ public void testGetProcNodeData() throws DdlException { List> rows = result.getRows(); Assert.assertEquals(1, rows.size()); } + + @Test + public void testPolicyToString() { + TWgSlotMemoryPolicy p1 = WorkloadGroup.findSlotPolicyValueByString("fixed"); + Assert.assertEquals(p1, TWgSlotMemoryPolicy.FIXED); + TWgSlotMemoryPolicy p2 = WorkloadGroup.findSlotPolicyValueByString("dynamic"); + Assert.assertEquals(p2, TWgSlotMemoryPolicy.DYNAMIC); + TWgSlotMemoryPolicy p3 = WorkloadGroup.findSlotPolicyValueByString("none"); + Assert.assertEquals(p3, TWgSlotMemoryPolicy.NONE); + TWgSlotMemoryPolicy p4 = WorkloadGroup.findSlotPolicyValueByString("none"); + Assert.assertEquals(p4, TWgSlotMemoryPolicy.NONE); + boolean hasException = false; + try { + WorkloadGroup.findSlotPolicyValueByString("disableDa"); + } catch (RuntimeException e) { + hasException = true; + } + Assert.assertEquals(hasException, true); + } } diff --git a/gensrc/proto/data.proto b/gensrc/proto/data.proto index 9b3824db3dc17d..95fb522289ee5c 100644 --- a/gensrc/proto/data.proto +++ b/gensrc/proto/data.proto @@ -37,6 +37,8 @@ message PQueryStatistics { repeated PNodeStatistics nodes_statistics = 6; optional int64 scan_bytes_from_local_storage = 7; optional int64 scan_bytes_from_remote_storage = 8; + optional int64 spill_write_bytes_to_local_storage = 9; + optional int64 spill_read_bytes_from_local_storage = 10; } message PRowBatch { diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index aed248adfe891a..0e17e2def87740 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -258,6 +258,12 @@ enum TTopicInfoType { WORKLOAD_SCHED_POLICY = 2 } +enum TWgSlotMemoryPolicy { + NONE = 0, + FIXED = 1, + DYNAMIC = 2 +} + struct TWorkloadGroupInfo { 1: optional i64 id 2: optional string name @@ -275,6 +281,9 @@ struct TWorkloadGroupInfo { 14: optional i64 read_bytes_per_second 15: optional i64 remote_read_bytes_per_second 16: optional string tag + 17: optional i32 total_query_slot_count + 18: optional i32 write_buffer_ratio + 19: optional TWgSlotMemoryPolicy slot_memory_policy } enum TWorkloadMetricType { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 8f7e3432a78bd6..aa4c8fc5dba1cf 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -415,6 +415,8 @@ struct TQueryStatistics { 9: optional i64 shuffle_send_rows 10: optional i64 scan_bytes_from_local_storage 11: optional i64 scan_bytes_from_remote_storage + 12: optional i64 spill_write_bytes_to_local_storage + 13: optional i64 spill_read_bytes_from_local_storage } struct TReportWorkloadRuntimeStatusParams { diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 39f3b65818f2b9..32d4f0bea5c610 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -182,6 +182,7 @@ struct TQueryOptions { 52: optional i32 be_exec_version = 0 + // not used any more 53: optional i32 partitioned_hash_join_rows_threshold = 0 54: optional bool enable_share_hash_table_for_broadcast_join @@ -195,9 +196,10 @@ struct TQueryOptions { 58: optional i32 repeat_max_num = 0 // Deprecated + // deprecated, use spill_sort_mem_limit 59: optional i64 external_sort_bytes_threshold = 0 - // deprecated + // Not used any more 60: optional i32 partitioned_hash_agg_rows_threshold = 0 61: optional bool enable_file_cache = false @@ -213,9 +215,10 @@ struct TQueryOptions { 66: optional i32 parallel_instance = 1 // Indicate where useServerPrepStmts enabled 67: optional bool mysql_row_binary_format = false; + // Not used anymore 68: optional i64 external_agg_bytes_threshold = 0 - // partition count(1 << external_agg_partition_bits) when spill aggregation data into disk + // Not used anymore, use spill_aggregation_partition_count 69: optional i32 external_agg_partition_bits = 4 // Specify base path for file cache @@ -281,10 +284,13 @@ struct TQueryOptions { 100: optional bool enable_distinct_streaming_aggregation = true; + // deprecated 101: optional bool enable_join_spill = false + // deprecated 102: optional bool enable_sort_spill = false + // deprecated 103: optional bool enable_agg_spill = false 104: optional i64 min_revocable_mem = 0 @@ -333,9 +339,7 @@ struct TQueryOptions { 125: optional bool enable_segment_cache = true; 126: optional i32 runtime_bloom_filter_max_size = 16777216; - 127: optional i32 in_list_value_count_threshold = 10; - // We need this two fields to make sure thrift id on master is compatible with other branch. 128: optional bool enable_verbose_profile = false; 129: optional i32 rpc_verbose_profile_max_instance_count = 0; @@ -350,7 +354,7 @@ struct TQueryOptions { 135: optional bool enable_parallel_outfile = false; 136: optional bool enable_phrase_query_sequential_opt = true; - + 137: optional bool enable_auto_create_when_overwrite = false; 138: optional i64 orc_tiny_stripe_threshold_bytes = 8388608; @@ -358,6 +362,7 @@ struct TQueryOptions { 140: optional i64 orc_max_merge_distance_bytes = 1048576; 141: optional bool ignore_runtime_filter_error = false; + 142: optional bool enable_fixed_len_to_uint32_v2 = false; 143: optional bool enable_shared_exchange_sink_buffer = true; @@ -365,6 +370,21 @@ struct TQueryOptions { 145: optional bool enable_inverted_index_query_cache = true; 146: optional bool fuzzy_disable_runtime_filter_in_be = false; + // The minimum memory that an operator required to run. + 147: optional i32 minimum_operator_memory_required_kb = 1024; + + 148: optional bool enable_mem_overcommit = true; + 149: optional i32 query_slot_count = 0; + 150: optional bool enable_spill = false + 151: optional bool enable_reserve_memory = true + 152: optional i32 revocable_memory_high_watermark_percent = -1 + + 153: optional i64 spill_sort_mem_limit = 134217728 + 154: optional i64 spill_sort_batch_bytes = 8388608 + 155: optional i32 spill_aggregation_partition_count = 32 + 156: optional i32 spill_hash_join_partition_count = 32 + 157: optional i64 low_memory_mode_buffer_limit = 33554432 + // 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. diff --git a/regression-test/data/workload_manager_p0/test_curd_wlg.out b/regression-test/data/workload_manager_p0/test_curd_wlg.out index aa922615dfe6d1..0e05adff526544 100644 --- a/regression-test/data/workload_manager_p0/test_curd_wlg.out +++ b/regression-test/data/workload_manager_p0/test_curd_wlg.out @@ -144,11 +144,11 @@ test_wg_priv_role1 test_wg_priv_g1 Usage_priv NO -- !select_wgp_12 -- -- !select_default_val_wg_1 -- -default_val_wg -1 -1 true 2147483647 0 0 -1 -1 -1 -1 50% 80% -1 -1 +default_val_wg -1 -1 true 2147483647 0 0 -1 -1 -1 -1 80% 95% -1 -1 -- !select_default_val_wg_2 -- -default_val_wg 1024 1% true 100 1 123 1% 1 12 10 50% 80% abc 123 10 +default_val_wg 1024 1% true 100 1 123 1% 1 12 10 80% 95% abc 123 10 -- !select_default_val_wg_3 -- -default_val_wg -1 -1 true 2147483647 0 0 -1 -1 -1 -1 50% 80% -1 -1 +default_val_wg -1 -1 true 2147483647 0 0 -1 -1 -1 -1 80% 95% -1 -1 diff --git a/regression-test/suites/datatype_p0/nested_types/query/test_nested_type_with_resize.groovy b/regression-test/suites/datatype_p0/nested_types/query/test_nested_type_with_resize.groovy index 2a599c1cc1ad20..5a158b53e81869 100644 --- a/regression-test/suites/datatype_p0/nested_types/query/test_nested_type_with_resize.groovy +++ b/regression-test/suites/datatype_p0/nested_types/query/test_nested_type_with_resize.groovy @@ -65,9 +65,9 @@ suite("test_nested_type_with_resize") { } } - order_qt_sql """ /*set ShuffleSendBytes=0|ShuffleSendRows=0|FuzzyVariables=batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=5,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,parallel_prepare_threshold=13,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=true,sort_phase_num=0,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=true,enable_local_exchange=false,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=1048576,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5*/ select col4 from test_array_resize order by col1,col2,col3 limit 10; """ - order_qt_sql """ /*set ShuffleSendBytes=0|ShuffleSendRows=0|FuzzyVariables=batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=5,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,parallel_prepare_threshold=13,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=true,sort_phase_num=0,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=true,enable_local_exchange=false,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=1048576,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5*/ select col22 from test_array_resize order by col1,col2,col3 limit 10; """ - order_qt_sql """ /*set ShuffleSendBytes=0|ShuffleSendRows=0|FuzzyVariables=batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=5,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,parallel_prepare_threshold=13,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=true,sort_phase_num=0,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=true,enable_local_exchange=false,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=1048576,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5*/ select col32 from test_array_resize order by col1,col2,col3 limit 10; """ - order_qt_sql """ /*set ShuffleSendBytes=0|ShuffleSendRows=0|FuzzyVariables=batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=5,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,parallel_prepare_threshold=13,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=true,sort_phase_num=0,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=true,enable_local_exchange=false,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=1048576,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5*/ select col74 from test_array_resize order by col1,col2,col3 limit 10; """ + order_qt_sql """ /*set ShuffleSendBytes=0|ShuffleSendRows=0|FuzzyVariables=batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=5,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,parallel_prepare_threshold=13,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=true,sort_phase_num=0,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=true,enable_local_exchange=false,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=false,enable_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5*/ select col4 from test_array_resize order by col1,col2,col3 limit 10; """ + order_qt_sql """ /*set ShuffleSendBytes=0|ShuffleSendRows=0|FuzzyVariables=batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=5,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,parallel_prepare_threshold=13,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=true,sort_phase_num=0,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=true,enable_local_exchange=false,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=false,enable_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5*/ select col22 from test_array_resize order by col1,col2,col3 limit 10; """ + order_qt_sql """ /*set ShuffleSendBytes=0|ShuffleSendRows=0|FuzzyVariables=batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=5,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,parallel_prepare_threshold=13,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=true,sort_phase_num=0,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=true,enable_local_exchange=false,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=false,enable_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5*/ select col32 from test_array_resize order by col1,col2,col3 limit 10; """ + order_qt_sql """ /*set ShuffleSendBytes=0|ShuffleSendRows=0|FuzzyVariables=batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=5,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,parallel_prepare_threshold=13,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=true,sort_phase_num=0,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=true,enable_local_exchange=false,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=false,enable_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5*/ select col74 from test_array_resize order by col1,col2,col3 limit 10; """ } diff --git a/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy b/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy index fae32e914ed031..33affd8a7cd52d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/variant/variant_mv.groovy @@ -574,7 +574,7 @@ suite("variant_mv") { where g2.actor['id'] > 34259289; """ def query3_6 = """ - SELECT /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=0,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=32,parallel_scan_min_rows_per_scanner=64,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=1,enable_parallel_result_sink=false,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=false,enable_local_exchange=true,partitioned_hash_join_rows_threshold=8,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=true,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=false,enable_delete_sub_predicate_v2=false,min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_join_spill=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ + SELECT /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=3,parallel_pipeline_task_num=0,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=32,parallel_scan_min_rows_per_scanner=64,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=1,enable_parallel_result_sink=false,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=100000,enable_function_pushdown=false,enable_common_expr_pushdown=false,enable_local_exchange=true,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=true,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=false,enable_delete_sub_predicate_v2=false,spill_min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5) */ g1.id, g2.type, floor(cast(g1.actor['id'] as int) + 100.5), diff --git a/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy b/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy index 676cdd06274a68..33a2595de2d9d9 100644 --- a/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy +++ b/regression-test/suites/query_p0/join/test_partitioned_hash_join.groovy @@ -65,7 +65,7 @@ suite("test_partitioned_hash_join", "query,p0,arrow_flight_sql") { qt_partitioned_hash_join1 """ select - /*+SET_VAR(disable_join_reorder=true,experimental_enable_pipeline_engine=false, parallel_fragment_exec_instance_num=1, partitioned_hash_join_rows_threshold = 1)*/ + /*+SET_VAR(disable_join_reorder=true,experimental_enable_pipeline_engine=false, parallel_fragment_exec_instance_num=1)*/ kl1 from test_partitioned_hash_join_l diff --git a/regression-test/suites/query_p0/limit/test_group_by_limit.groovy b/regression-test/suites/query_p0/limit/test_group_by_limit.groovy index d6f49f9d7a397f..145c08a3ece675 100644 --- a/regression-test/suites/query_p0/limit/test_group_by_limit.groovy +++ b/regression-test/suites/query_p0/limit/test_group_by_limit.groovy @@ -17,7 +17,7 @@ suite("test_group_by_limit", "query") { -sql 'set enable_agg_spill=false' +sql 'set enable_spill=false' sql 'set enable_force_spill=false' diff --git a/regression-test/suites/spill_p0/aggregate_spill.groovy b/regression-test/suites/spill_p0/aggregate_spill.groovy index 180ab37200f171..2a4127d177c9de 100644 --- a/regression-test/suites/spill_p0/aggregate_spill.groovy +++ b/regression-test/suites/spill_p0/aggregate_spill.groovy @@ -17,13 +17,13 @@ suite("aggregate_spill") { sql """ - set enable_agg_spill = true; + set enable_spill = true; """ sql """ set enable_force_spill = true; """ sql """ - set min_revocable_mem = 1; + set spill_min_revocable_mem = 1; """ sql """ set parallel_pipeline_task_num = 4; diff --git a/regression-test/suites/tpcds_sf1_p1/spill_test/q23.groovy b/regression-test/suites/tpcds_sf1_p1/spill_test/q23.groovy index cc5408d6b81a54..08f1e04d9b87ba 100644 --- a/regression-test/suites/tpcds_sf1_p1/spill_test/q23.groovy +++ b/regression-test/suites/tpcds_sf1_p1/spill_test/q23.groovy @@ -17,7 +17,7 @@ suite("q23") { sql """ set enable_force_spill =true; """ - sql """ set min_revocable_mem = 65536; """ + sql """ set spill_min_revocable_mem = 65536; """ sql """ use regression_test_tpcds_sf1_p1; """ qt_select1 """ diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q01.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q01.groovy index b32601cf600547..4bb49ea12e05f1 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q01.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q01.groovy @@ -19,7 +19,7 @@ suite("q01_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q02.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q02.groovy index 005b5ea6e9f3c2..838ae626e3a4ff 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q02.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q02.groovy @@ -19,7 +19,7 @@ suite("q02_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q03.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q03.groovy index dcb40b0a1258d6..10005476014a5f 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q03.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q03.groovy @@ -19,7 +19,7 @@ suite("q03_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q04.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q04.groovy index ad203cf3ee1321..850a5c20776e64 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q04.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q04.groovy @@ -19,7 +19,7 @@ suite("q04_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=500; + set spill_min_revocable_mem=500; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q05.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q05.groovy index f743b03fc7fdf1..c19f657d25e562 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q05.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q05.groovy @@ -19,7 +19,7 @@ suite("q05_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q06.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q06.groovy index 9d7ff51f39f9e4..e50d4aaec6d2f0 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q06.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q06.groovy @@ -19,7 +19,7 @@ suite("q06_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q07.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q07.groovy index 7439ed1472931a..ef37208782adba 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q07.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q07.groovy @@ -19,7 +19,7 @@ suite("q07_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q08.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q08.groovy index 549e3af9e6ee46..2ba2478d185d0d 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q08.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q08.groovy @@ -19,7 +19,7 @@ suite("q08_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q09.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q09.groovy index 1f4e0b7a061fd4..0ea7b5e860f117 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q09.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q09.groovy @@ -19,7 +19,7 @@ suite("q09_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q10.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q10.groovy index 0b4c0de0adb2bb..f2caa94ab39e90 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q10.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q10.groovy @@ -19,7 +19,7 @@ suite("q10_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q11.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q11.groovy index 8c9ba7431f9749..84f9a4ee328845 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q11.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q11.groovy @@ -19,7 +19,7 @@ suite("q11_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q12.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q12.groovy index 2958676568aace..49008eca758de4 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q12.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q12.groovy @@ -19,7 +19,7 @@ suite("q12_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q13.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q13.groovy index 192de5525da7e6..6fe75c18c169cc 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q13.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q13.groovy @@ -19,7 +19,7 @@ suite("q13_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q14_1.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q14_1.groovy index c4f7f5c0bb5da9..08bd47a996ec00 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q14_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q14_1.groovy @@ -19,7 +19,7 @@ suite("q14_1_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q14_2.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q14_2.groovy index 7381131713baef..60539a0920eb44 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q14_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q14_2.groovy @@ -19,7 +19,7 @@ suite("q14_2_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q15.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q15.groovy index 7ab60727c5b77d..3787e84ada655f 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q15.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q15.groovy @@ -19,7 +19,7 @@ suite("q15_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q16.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q16.groovy index af93a322c872e2..265fa9f64f7a1c 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q16.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q16.groovy @@ -19,7 +19,7 @@ suite("q16_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q17.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q17.groovy index ade102222cc91e..6db3ce88b709a8 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q17.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q17.groovy @@ -19,7 +19,7 @@ suite("q17_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q18.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q18.groovy index 311c0a2d5dfc5d..e65507319033ee 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q18.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q18.groovy @@ -19,7 +19,7 @@ suite("q18_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q19.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q19.groovy index d61da8d37548ac..f73d22051e4adc 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q19.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q19.groovy @@ -19,7 +19,7 @@ suite("q19_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q20.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q20.groovy index dbd6382fac0b5d..ad44d54d9f17ca 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q20.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q20.groovy @@ -19,7 +19,7 @@ suite("q20_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q21.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q21.groovy index 3dc9caac152867..6d5cb1435c77e0 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q21.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q21.groovy @@ -19,7 +19,7 @@ suite("q21_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q22.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q22.groovy index 5030f6a338d542..b5dc33c0dbe274 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q22.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q22.groovy @@ -19,7 +19,7 @@ suite("q22_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q23_1.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q23_1.groovy index cbb9962b63d1f2..ba5f83fc26f60c 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q23_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q23_1.groovy @@ -19,7 +19,7 @@ suite("q23_1_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q23_2.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q23_2.groovy index f74cb1cd85ee52..afcef33bb05d7c 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q23_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q23_2.groovy @@ -19,7 +19,7 @@ suite("q23_2_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy index 1847ff8a5c65d1..3725d665c4617d 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy @@ -19,7 +19,7 @@ suite("q24_1_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy index 69d40f764c5777..b609cf9cc99959 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy @@ -19,7 +19,7 @@ suite("q24_2_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=500; + set spill_min_revocable_mem=500; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q25.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q25.groovy index f6732db8aacffc..f4dce66555b80e 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q25.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q25.groovy @@ -19,7 +19,7 @@ suite("q25_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q26.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q26.groovy index 6c51cc77a5cfac..d3b36339072b8d 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q26.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q26.groovy @@ -19,7 +19,7 @@ suite("q26_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q27.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q27.groovy index 1a9a10d73e604a..084f7045041af3 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q27.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q27.groovy @@ -19,7 +19,7 @@ suite("q27_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q28.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q28.groovy index 3efc9b3dc01010..85e0502adc042f 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q28.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q28.groovy @@ -19,7 +19,7 @@ suite("q28_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q29.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q29.groovy index df21a5e2ae111c..e5d6cbf5612211 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q29.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q29.groovy @@ -19,7 +19,7 @@ suite("q29_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q30.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q30.groovy index 040b98791f4686..4eb2530e3e11fc 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q30.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q30.groovy @@ -19,7 +19,7 @@ suite("q30_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q31.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q31.groovy index 0e22e379d52daa..fb638d506f7225 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q31.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q31.groovy @@ -19,7 +19,7 @@ suite("q31_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q32.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q32.groovy index 1254dd7c61adcf..11eaa781bfcc73 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q32.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q32.groovy @@ -19,7 +19,7 @@ suite("q32_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q33.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q33.groovy index 31ff1f96b7422e..5e89f9f497c2dd 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q33.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q33.groovy @@ -19,7 +19,7 @@ suite("q33_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q34.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q34.groovy index 4cecd47df8278e..9a7a71866fcdf0 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q34.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q34.groovy @@ -19,7 +19,7 @@ suite("q34_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q35.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q35.groovy index ed264485a06f24..b2643fb4de9ec6 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q35.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q35.groovy @@ -19,7 +19,7 @@ suite("q35_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q36.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q36.groovy index 313ffbcd92ff5e..05802c810f0ad2 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q36.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q36.groovy @@ -19,7 +19,7 @@ suite("q36_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q37.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q37.groovy index a2d83e3c6f627c..6c54188f34eb0a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q37.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q37.groovy @@ -19,7 +19,7 @@ suite("q37_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q38.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q38.groovy index 103306336c1a20..505208f755c2bd 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q38.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q38.groovy @@ -19,7 +19,7 @@ suite("q38_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q39_1.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q39_1.groovy index c7c94b9f847814..32ed089a956fde 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q39_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q39_1.groovy @@ -19,7 +19,7 @@ suite("q39_1_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q39_2.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q39_2.groovy index 0f6f14654a2039..029fbade5d7b27 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q39_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q39_2.groovy @@ -19,7 +19,7 @@ suite("q39_2_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q40.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q40.groovy index 7c33af6a957f5c..34577c9bfccc5f 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q40.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q40.groovy @@ -19,7 +19,7 @@ suite("q40_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q41.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q41.groovy index 16ad8981ef7e24..870dec6e3f0a6a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q41.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q41.groovy @@ -19,7 +19,7 @@ suite("q41_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q42.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q42.groovy index f9c4171ac6031f..2440976d8e15e8 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q42.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q42.groovy @@ -19,7 +19,7 @@ suite("q42_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q43.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q43.groovy index efc863bf5c8907..ed51ebef5c026a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q43.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q43.groovy @@ -19,7 +19,7 @@ suite("q43_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q44.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q44.groovy index 05f4c8b6779cd3..9342e098333c55 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q44.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q44.groovy @@ -19,7 +19,7 @@ suite("q44_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q45.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q45.groovy index 70129815a1d677..91625f3b874eca 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q45.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q45.groovy @@ -19,7 +19,7 @@ suite("q45_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q47.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q47.groovy index d4934077bec92c..d19b95058b923a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q47.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q47.groovy @@ -19,7 +19,7 @@ suite("q47_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q48.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q48.groovy index 5824dad703e1b7..bf052dc9dbf520 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q48.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q48.groovy @@ -19,7 +19,7 @@ suite("q48_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q49.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q49.groovy index ec1bae6277a88e..217e6d4123e12e 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q49.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q49.groovy @@ -19,7 +19,7 @@ suite("q49_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q49_rewrite.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q49_rewrite.groovy index 593025caecaee3..4d2a3c7c680504 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q49_rewrite.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q49_rewrite.groovy @@ -19,7 +19,7 @@ suite("q49_rewrite_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q50.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q50.groovy index 03c75f92df857c..0720cb8fb7b3ab 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q50.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q50.groovy @@ -19,7 +19,7 @@ suite("q50_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q51.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q51.groovy index a246962d2752d1..866b2af0ee7f53 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q51.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q51.groovy @@ -19,7 +19,7 @@ suite("q51_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q52.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q52.groovy index a51c260047f2de..cd501ac7439dfa 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q52.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q52.groovy @@ -19,7 +19,7 @@ suite("q52_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q53.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q53.groovy index a92cb45d097628..57d76897157d02 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q53.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q53.groovy @@ -19,7 +19,7 @@ suite("q53_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q54.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q54.groovy index 4db6568da95e7d..ac0ef507581ace 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q54.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q54.groovy @@ -19,7 +19,7 @@ suite("q54_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q55.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q55.groovy index 1825422f5a8590..1b2a829ca7c3be 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q55.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q55.groovy @@ -19,7 +19,7 @@ suite("q55_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q56.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q56.groovy index ede5d54db3bcdb..d5ce621d36b74a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q56.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q56.groovy @@ -19,7 +19,7 @@ suite("q56_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q57.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q57.groovy index bf126dd2325040..de693026b62a00 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q57.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q57.groovy @@ -19,7 +19,7 @@ suite("q57_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q58.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q58.groovy index fde8e03ea566d5..68d80ab02606ea 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q58.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q58.groovy @@ -19,7 +19,7 @@ suite("q58_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q59.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q59.groovy index e435834e07aeb1..20c1a4977687a0 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q59.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q59.groovy @@ -19,7 +19,7 @@ suite("q59_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q60.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q60.groovy index ca7ee275442431..16f85bb9900a03 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q60.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q60.groovy @@ -19,7 +19,7 @@ suite("q60_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q61.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q61.groovy index 933c026f6d6a28..24acaea1ad3332 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q61.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q61.groovy @@ -19,7 +19,7 @@ suite("q61_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q62.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q62.groovy index febe9fb842940e..eeea52d314c0e1 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q62.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q62.groovy @@ -19,7 +19,7 @@ suite("q62_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q63.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q63.groovy index c1ff1e6dd1576c..bbb2e6fa9152af 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q63.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q63.groovy @@ -19,7 +19,7 @@ suite("q63_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q64.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q64.groovy index 10c446f43205d1..4fd8dfaa8b36e2 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q64.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q64.groovy @@ -19,7 +19,7 @@ suite("q64_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q65.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q65.groovy index e16237a3882cac..0f20dce7666b67 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q65.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q65.groovy @@ -19,7 +19,7 @@ suite("q65_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q66.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q66.groovy index dac5da4977a0f8..162ad7beaa65a4 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q66.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q66.groovy @@ -19,7 +19,7 @@ suite("q66_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q67.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q67.groovy index 971881431e581b..e297d8b01d6e9b 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q67.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q67.groovy @@ -19,7 +19,7 @@ suite("q67_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q68.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q68.groovy index 4994673ce2f746..ab3ad61c2e3323 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q68.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q68.groovy @@ -19,7 +19,7 @@ suite("q68_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q69.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q69.groovy index 9219e5e634340b..87ab225b1c2982 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q69.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q69.groovy @@ -19,7 +19,7 @@ suite("q69_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q70.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q70.groovy index ac90c4dfddc773..f2f8932fb52731 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q70.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q70.groovy @@ -19,7 +19,7 @@ suite("q70_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q71.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q71.groovy index 09612747fc5c39..af986e3e810b5e 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q71.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q71.groovy @@ -19,7 +19,7 @@ suite("q71_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q72.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q72.groovy index 7b03dd9f573637..f535a3715e773a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q72.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q72.groovy @@ -19,7 +19,7 @@ suite("q72_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q73.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q73.groovy index 0c818fe0fff421..121582d8000840 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q73.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q73.groovy @@ -19,7 +19,7 @@ suite("q73_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q74.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q74.groovy index 5e0c82b8dc1dda..2a75a5e6b04210 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q74.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q74.groovy @@ -19,7 +19,7 @@ suite("q74_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q75.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q75.groovy index 7a245f3116389b..1c5bd5b262e3e3 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q75.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q75.groovy @@ -19,7 +19,7 @@ suite("q75_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q76.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q76.groovy index ad0a9e803ab055..89edf0abe1a93d 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q76.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q76.groovy @@ -19,7 +19,7 @@ suite("q76_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q77.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q77.groovy index 08de73daceb308..dd1fd34c1bff5d 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q77.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q77.groovy @@ -19,7 +19,7 @@ suite("q77_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q78.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q78.groovy index 5092857d6af166..e99ee2a67c0e0a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q78.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q78.groovy @@ -19,7 +19,7 @@ suite("q78_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q79.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q79.groovy index 6a6f46d4909631..596f9f2bbcc951 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q79.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q79.groovy @@ -19,7 +19,7 @@ suite("q79_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q80.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q80.groovy index a1cc15cb896535..c592a8427e92f8 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q80.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q80.groovy @@ -19,7 +19,7 @@ suite("q80_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q81.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q81.groovy index 40bbe33eb2f51f..437fe1beffd7f6 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q81.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q81.groovy @@ -19,7 +19,7 @@ suite("q81_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q82.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q82.groovy index edd3f7a4247d9a..13f8fdc6379c77 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q82.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q82.groovy @@ -19,7 +19,7 @@ suite("q82_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q83.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q83.groovy index 12859e25595baf..e8a83ec34e76ab 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q83.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q83.groovy @@ -19,7 +19,7 @@ suite("q83_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q84.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q84.groovy index 0b462d141ee839..8c004720d5d32f 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q84.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q84.groovy @@ -19,7 +19,7 @@ suite("q84_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q85.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q85.groovy index fdc7d3db86e0af..da478a70824460 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q85.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q85.groovy @@ -19,7 +19,7 @@ suite("q85_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q86.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q86.groovy index 05360c66e84365..0a47fe6a5fa999 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q86.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q86.groovy @@ -19,7 +19,7 @@ suite("q86_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q87.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q87.groovy index 78c7e304ce4783..a51821b010e4ab 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q87.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q87.groovy @@ -19,7 +19,7 @@ suite("q87_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q88.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q88.groovy index 0036488937093c..cb8b9e615dced1 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q88.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q88.groovy @@ -19,7 +19,7 @@ suite("q88_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q89.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q89.groovy index a99c3f559db668..8ad30ac16490c7 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q89.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q89.groovy @@ -19,7 +19,7 @@ suite("q89_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q90.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q90.groovy index 637fdd4cb0558e..f558a72621a48a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q90.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q90.groovy @@ -19,7 +19,7 @@ suite("q90_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q91.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q91.groovy index 79e5b0997eed35..dbb925c19c2762 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q91.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q91.groovy @@ -19,7 +19,7 @@ suite("q91_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q92.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q92.groovy index 149bf92041a1f3..b80df399d8f0d3 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q92.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q92.groovy @@ -19,7 +19,7 @@ suite("q92_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ multi_sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q93.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q93.groovy index 0a375e0a4299e3..13c6aa8f82326a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q93.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q93.groovy @@ -19,7 +19,7 @@ suite("q93_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q94.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q94.groovy index 5aa3db3a344f48..427e563db3f875 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q94.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q94.groovy @@ -19,7 +19,7 @@ suite("q94_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q95.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q95.groovy index aa60089dd9fa55..50d8591b412594 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q95.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q95.groovy @@ -19,7 +19,7 @@ suite("q95_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q96.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q96.groovy index 0ee2566e690a48..054ddf1d72ec00 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q96.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q96.groovy @@ -19,7 +19,7 @@ suite("q96_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q97.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q97.groovy index be34b583df53f1..c69a1fa5f9313d 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q97.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q97.groovy @@ -19,7 +19,7 @@ suite("q97_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q98.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q98.groovy index ba05ea62635134..3e11a022d69b05 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q98.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q98.groovy @@ -19,7 +19,7 @@ suite("q98_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q99.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q99.groovy index 899cf068e582c8..4f1cdf723fe73a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q99.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q99.groovy @@ -19,7 +19,7 @@ suite("q99_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_ck_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q01.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q01.groovy index a0e3b6547bcffa..1e9649fe571a42 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q01.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q01.groovy @@ -19,7 +19,7 @@ suite("q01_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q02.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q02.groovy index d7ce6bae65ddb5..088db6ede82681 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q02.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q02.groovy @@ -19,7 +19,7 @@ suite("q02_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q03.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q03.groovy index 1eb8e702da3551..15f94958de3bc2 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q03.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q03.groovy @@ -19,7 +19,7 @@ suite("q03_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q04.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q04.groovy index 47b73e434342c2..6b0e877e681d08 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q04.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q04.groovy @@ -19,7 +19,7 @@ suite("q04_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=500; + set spill_min_revocable_mem=500; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q05.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q05.groovy index 30532794e68d19..7cc8630bb97086 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q05.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q05.groovy @@ -19,7 +19,7 @@ suite("q05_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q06.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q06.groovy index 7dfbeb1941cff4..1577eee94dd138 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q06.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q06.groovy @@ -19,7 +19,7 @@ suite("q06_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q07.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q07.groovy index 2b615703d24cb9..380a0010d0ff61 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q07.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q07.groovy @@ -19,7 +19,7 @@ suite("q07_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q08.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q08.groovy index edc8e89851f937..e037c9a2a10114 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q08.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q08.groovy @@ -19,7 +19,7 @@ suite("q08_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q09.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q09.groovy index fc15a914000f1c..bf48659860d8ad 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q09.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q09.groovy @@ -19,7 +19,7 @@ suite("q09_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q10.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q10.groovy index ad2453499b58d4..e6a91e1007d2ab 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q10.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q10.groovy @@ -19,7 +19,7 @@ suite("q10_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q11.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q11.groovy index c2af13e200267b..c3763114678995 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q11.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q11.groovy @@ -19,7 +19,7 @@ suite("q11_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q12.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q12.groovy index aa1a3d44c81185..c15fda0f45b2e6 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q12.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q12.groovy @@ -19,7 +19,7 @@ suite("q12_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q13.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q13.groovy index f5e3f363d08983..979b103efb20e8 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q13.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q13.groovy @@ -19,7 +19,7 @@ suite("q13_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q14_1.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q14_1.groovy index 222773d9cf9133..14e1b8e36a81d0 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q14_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q14_1.groovy @@ -19,7 +19,7 @@ suite("q14_1_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q14_2.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q14_2.groovy index f3cae690200ecf..daf10874c3a448 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q14_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q14_2.groovy @@ -19,7 +19,7 @@ suite("q14_2_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q15.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q15.groovy index a7c57e15f1fcbf..480b66cb209b7b 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q15.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q15.groovy @@ -19,7 +19,7 @@ suite("q15_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q16.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q16.groovy index 868ccdcea334b5..7a353353afdc39 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q16.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q16.groovy @@ -19,7 +19,7 @@ suite("q16_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q17.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q17.groovy index 91bb1011474a06..73652f6a0c1b5c 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q17.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q17.groovy @@ -19,7 +19,7 @@ suite("q17_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q18.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q18.groovy index d9b562cc4f1f0a..062cb569be07ec 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q18.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q18.groovy @@ -19,7 +19,7 @@ suite("q18_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q19.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q19.groovy index 6667110fe9f8d1..88beee4e4330f7 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q19.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q19.groovy @@ -19,7 +19,7 @@ suite("q19_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q20.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q20.groovy index 96c978d2a05798..6992d0ce012efc 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q20.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q20.groovy @@ -19,7 +19,7 @@ suite("q20_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q21.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q21.groovy index 78634b748dd17d..92002103ea1027 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q21.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q21.groovy @@ -19,7 +19,7 @@ suite("q21_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q22.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q22.groovy index 5badf3dfed6da4..39681cb34cad00 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q22.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q22.groovy @@ -19,7 +19,7 @@ suite("q22_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q23_1.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q23_1.groovy index bfaf1b022ab8ff..7282bd38dfa10e 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q23_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q23_1.groovy @@ -19,7 +19,7 @@ suite("q23_1_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q23_2.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q23_2.groovy index 39056dd258d043..1a29aac8855fe1 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q23_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q23_2.groovy @@ -19,7 +19,7 @@ suite("q23_2_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy index 1c3df66e9df461..7eb2dcd0e5d0e6 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy @@ -19,7 +19,7 @@ suite("q24_1_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy index ee4e3407e9f3ee..be07ca4c6ccb1d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy @@ -19,7 +19,7 @@ suite("q24_2_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=500; + set spill_min_revocable_mem=500; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q25.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q25.groovy index a5f28fd4d54a4b..cc3ecfa628b0c6 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q25.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q25.groovy @@ -19,7 +19,7 @@ suite("q25_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q26.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q26.groovy index 7ba39f7341075c..a6387bf646ea94 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q26.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q26.groovy @@ -19,7 +19,7 @@ suite("q26_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q27.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q27.groovy index fb0479f83bd211..a6956e083c7724 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q27.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q27.groovy @@ -19,7 +19,7 @@ suite("q27_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q28.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q28.groovy index 75299e9f875a73..0130216a522466 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q28.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q28.groovy @@ -19,7 +19,7 @@ suite("q28_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q29.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q29.groovy index afb9e0b3206ed9..f6839c5fb89df5 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q29.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q29.groovy @@ -19,7 +19,7 @@ suite("q29_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q30.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q30.groovy index b4ca8d5959c74b..527fa3add10509 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q30.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q30.groovy @@ -19,7 +19,7 @@ suite("q30_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q31.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q31.groovy index 7ac8da05982f8e..20db957d8d9158 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q31.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q31.groovy @@ -19,7 +19,7 @@ suite("q31_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q32.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q32.groovy index 61752998ac1d65..8ababd68ab6b74 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q32.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q32.groovy @@ -19,7 +19,7 @@ suite("q32_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q33.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q33.groovy index 20abeef48cf3a5..efa775c5fe9061 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q33.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q33.groovy @@ -19,7 +19,7 @@ suite("q33_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q34.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q34.groovy index aad3cdc275864f..4f16cf0511ae08 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q34.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q34.groovy @@ -19,7 +19,7 @@ suite("q34_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q35.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q35.groovy index cf623eb634e4c6..f3cb4a222dbd0a 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q35.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q35.groovy @@ -19,7 +19,7 @@ suite("q35_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q36.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q36.groovy index 8da71f4b555cc0..ac3d8606e64562 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q36.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q36.groovy @@ -19,7 +19,7 @@ suite("q36_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q37.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q37.groovy index 7bdc85c62713c1..e491c41a1cf462 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q37.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q37.groovy @@ -19,7 +19,7 @@ suite("q37_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q38.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q38.groovy index f873cdaede0bef..c80b7d53e65eee 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q38.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q38.groovy @@ -19,7 +19,7 @@ suite("q38_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q39_1.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q39_1.groovy index 7eea2bd291d4d0..ccd8ca379af401 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q39_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q39_1.groovy @@ -19,7 +19,7 @@ suite("q39_1_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q39_2.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q39_2.groovy index e46c4917515c4d..05a3122f4f0a1d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q39_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q39_2.groovy @@ -19,7 +19,7 @@ suite("q39_2_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q40.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q40.groovy index dca463d78d0eb9..cad7707c2ec027 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q40.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q40.groovy @@ -19,7 +19,7 @@ suite("q40_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q41.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q41.groovy index 88c61037b37701..6228aebb1384b0 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q41.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q41.groovy @@ -19,7 +19,7 @@ suite("q41_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q42.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q42.groovy index 31716d6a3c5272..9fd51830fc4b4d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q42.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q42.groovy @@ -19,7 +19,7 @@ suite("q42_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q43.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q43.groovy index 6a7bcc37f0359f..d456b67176d76a 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q43.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q43.groovy @@ -19,7 +19,7 @@ suite("q43_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q44.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q44.groovy index e3dc65bb7392e3..03100cb0ae40ce 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q44.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q44.groovy @@ -19,7 +19,7 @@ suite("q44_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q45.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q45.groovy index b1adb93f8244ff..6213201ada480d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q45.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q45.groovy @@ -19,7 +19,7 @@ suite("q45_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q47.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q47.groovy index af2c8be0f28076..f25fe0c481e3a7 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q47.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q47.groovy @@ -19,7 +19,7 @@ suite("q47_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q48.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q48.groovy index 76126f3ae34667..0c473f9f665d0b 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q48.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q48.groovy @@ -19,7 +19,7 @@ suite("q48_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q49.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q49.groovy index fbaddf96b4956f..95ccd2a1549fc4 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q49.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q49.groovy @@ -19,7 +19,7 @@ suite("q49_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q49_rewrite.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q49_rewrite.groovy index 9ab0b51ec02d8b..622d492f120e74 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q49_rewrite.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q49_rewrite.groovy @@ -19,7 +19,7 @@ suite("q49_rewrite_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q50.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q50.groovy index 38c40367c27a5e..bae703858767a2 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q50.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q50.groovy @@ -19,7 +19,7 @@ suite("q50_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q51.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q51.groovy index 651d0619358502..8b45df390dd9e9 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q51.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q51.groovy @@ -19,7 +19,7 @@ suite("q51_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q52.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q52.groovy index 71f73d4fa6ad89..5de5a103e5b579 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q52.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q52.groovy @@ -19,7 +19,7 @@ suite("q52_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q53.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q53.groovy index f84eb890174026..02bd6a1e2f9206 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q53.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q53.groovy @@ -19,7 +19,7 @@ suite("q53_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q54.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q54.groovy index 6ce489b8e834c3..ad53d1f361b18e 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q54.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q54.groovy @@ -19,7 +19,7 @@ suite("q54_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q55.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q55.groovy index 53787e629e040e..e25d16ecb02af3 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q55.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q55.groovy @@ -19,7 +19,7 @@ suite("q55_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q56.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q56.groovy index 41f38eb763b856..1685ec947e8853 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q56.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q56.groovy @@ -19,7 +19,7 @@ suite("q56_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q57.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q57.groovy index a320a4df279cee..37d06f98b39277 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q57.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q57.groovy @@ -19,7 +19,7 @@ suite("q57_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q58.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q58.groovy index 637d632c6a651b..e6296ccce3156b 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q58.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q58.groovy @@ -19,7 +19,7 @@ suite("q58_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q59.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q59.groovy index eb065d22fd6700..da486030252710 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q59.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q59.groovy @@ -19,7 +19,7 @@ suite("q59_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q60.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q60.groovy index 0335d8cb589685..254b584d71cb3e 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q60.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q60.groovy @@ -19,7 +19,7 @@ suite("q60_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q61.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q61.groovy index 8ede55f9e0daa4..e6e52c634e9baa 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q61.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q61.groovy @@ -19,7 +19,7 @@ suite("q61_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q62.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q62.groovy index 4190b32d43be87..17edf56ceedcc1 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q62.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q62.groovy @@ -19,7 +19,7 @@ suite("q62_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q63.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q63.groovy index 079e8ed610089f..114a2e19c87380 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q63.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q63.groovy @@ -19,7 +19,7 @@ suite("q63_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q64.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q64.groovy index b3ae5ad3caba77..5e79261a529e19 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q64.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q64.groovy @@ -19,7 +19,7 @@ suite("q64_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q65.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q65.groovy index 132e6b0bfaaa32..fa83c6614a7f9a 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q65.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q65.groovy @@ -19,7 +19,7 @@ suite("q65_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q66.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q66.groovy index e91825e267f524..baf2205ca33cc3 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q66.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q66.groovy @@ -19,7 +19,7 @@ suite("q66_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q67.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q67.groovy index a51232f4da46a0..eef17357003e65 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q67.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q67.groovy @@ -19,7 +19,7 @@ suite("q67_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q68.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q68.groovy index d52f42c6ee58ab..9f7d2d5eed7a54 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q68.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q68.groovy @@ -19,7 +19,7 @@ suite("q68_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q69.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q69.groovy index 36bc9171c91d70..1fcbdd40918618 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q69.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q69.groovy @@ -19,7 +19,7 @@ suite("q69_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q70.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q70.groovy index 4382906f148a59..a1405302022f7d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q70.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q70.groovy @@ -19,7 +19,7 @@ suite("q70_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q71.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q71.groovy index 40d0940e0af17a..94c871f784a9ae 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q71.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q71.groovy @@ -19,7 +19,7 @@ suite("q71_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q72.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q72.groovy index 54b04396fd5d2d..f80cc3008414fc 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q72.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q72.groovy @@ -19,7 +19,7 @@ suite("q72_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q73.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q73.groovy index e6d36b6d3a46c5..f3cc0ce591442d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q73.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q73.groovy @@ -19,7 +19,7 @@ suite("q73_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q74.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q74.groovy index ce8fea2afd4378..278c780e8bfc94 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q74.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q74.groovy @@ -19,7 +19,7 @@ suite("q74_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q75.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q75.groovy index 7d795447d73756..dd7333444ce476 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q75.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q75.groovy @@ -19,7 +19,7 @@ suite("q75_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=100; + set spill_min_revocable_mem=100; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q76.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q76.groovy index a99e8165bde247..2be430b975e5f4 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q76.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q76.groovy @@ -19,7 +19,7 @@ suite("q76_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q77.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q77.groovy index c869b8b9c1b8f5..59877d37da7283 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q77.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q77.groovy @@ -19,7 +19,7 @@ suite("q77_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q78.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q78.groovy index 6ee9f33cb9b867..8956e8c83a7b5d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q78.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q78.groovy @@ -19,7 +19,7 @@ suite("q78_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q79.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q79.groovy index e22beae644640e..b9381a852ba06e 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q79.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q79.groovy @@ -19,7 +19,7 @@ suite("q79_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q80.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q80.groovy index ccd40c0edf9db0..133971b96fdc07 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q80.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q80.groovy @@ -19,7 +19,7 @@ suite("q80_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q81.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q81.groovy index 7f5d625229aebc..893afa4000265a 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q81.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q81.groovy @@ -19,7 +19,7 @@ suite("q81_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q82.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q82.groovy index d8afc0e1860165..b7585861eedd9d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q82.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q82.groovy @@ -19,7 +19,7 @@ suite("q82_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q83.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q83.groovy index 9146c1d36bd795..4bfceae1d23059 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q83.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q83.groovy @@ -19,7 +19,7 @@ suite("q83_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q84.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q84.groovy index d64ab5d4f7d202..d1b8aa0ceb7fb1 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q84.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q84.groovy @@ -19,7 +19,7 @@ suite("q84_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q85.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q85.groovy index 11330cd1dc7a0e..6c61281f87f78f 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q85.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q85.groovy @@ -19,7 +19,7 @@ suite("q85_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q86.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q86.groovy index bbc956a82fce27..447a965f9784cb 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q86.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q86.groovy @@ -19,7 +19,7 @@ suite("q86_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q87.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q87.groovy index c6479dbebe65ad..f7d303881049cb 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q87.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q87.groovy @@ -19,7 +19,7 @@ suite("q87_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q88.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q88.groovy index 8077197a61ae75..28462ebcd530af 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q88.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q88.groovy @@ -19,7 +19,7 @@ suite("q88_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q89.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q89.groovy index d6e6bf9ae70fbb..57cec480023a1d 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q89.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q89.groovy @@ -19,7 +19,7 @@ suite("q89_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q90.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q90.groovy index 7724c6d0eb816e..1032699123fbd9 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q90.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q90.groovy @@ -19,7 +19,7 @@ suite("q90_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q91.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q91.groovy index bc56a712531d2d..a0f7b91c82edea 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q91.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q91.groovy @@ -19,7 +19,7 @@ suite("q91_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q92.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q92.groovy index 3194c15030ef27..957a78575d6a01 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q92.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q92.groovy @@ -19,7 +19,7 @@ suite("q92_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ multi_sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q93.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q93.groovy index a2f4d8c3b90045..af2371dc2c624f 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q93.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q93.groovy @@ -19,7 +19,7 @@ suite("q93_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q94.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q94.groovy index ce5ed847ce571c..9c7a6ed5380e95 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q94.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q94.groovy @@ -19,7 +19,7 @@ suite("q94_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q95.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q95.groovy index 9bc5e5b2bbae1b..46cdb7301b5e4a 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q95.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q95.groovy @@ -19,7 +19,7 @@ suite("q95_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q96.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q96.groovy index 474073b10b59dd..e0e68fc3b4c68c 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q96.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q96.groovy @@ -19,7 +19,7 @@ suite("q96_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q97.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q97.groovy index 2a8ac3e21015ac..97adde154ef70c 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q97.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q97.groovy @@ -19,7 +19,7 @@ suite("q97_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q98.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q98.groovy index b28776964b7341..aec36c9c6de8ae 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q98.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q98.groovy @@ -19,7 +19,7 @@ suite("q98_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q99.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q99.groovy index 5bc0ee63ce4358..20300a1b671519 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q99.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q99.groovy @@ -19,7 +19,7 @@ suite("q99_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpcds_sf1_unique_p1; diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q01.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q01.groovy index dc8390eb487263..30e47e823dea80 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q01.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q01.groovy @@ -19,7 +19,7 @@ suite("q01_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q02.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q02.groovy index 1d32eae44dc282..7857ac2ccde9f6 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q02.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q02.groovy @@ -19,7 +19,7 @@ suite("q02_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q03.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q03.groovy index 80d5f905f1a6b4..3750e383f4532f 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q03.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q03.groovy @@ -19,7 +19,7 @@ suite("q03_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q03_spill") { qt_q03 """ -- tables: customer,orders,lineitem SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ l_orderkey, sum(l_extendedprice * (1 - l_discount)) AS revenue, o_orderdate, diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q04.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q04.groovy index c87e6fb13781ac..6559e12454bbb1 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q04.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q04.groovy @@ -19,7 +19,7 @@ suite("q04_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q04_spill") { qt_q04 """ -- tables: orders,lineitem SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ o_orderpriority, count(*) AS order_count FROM orders diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q05.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q05.groovy index 84c14a49c49fa3..a32adfa3107bf0 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q05.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q05.groovy @@ -19,7 +19,7 @@ suite("q05_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q05_spill") { qt_q05 """ -- tables: customer,orders,lineitem,supplier,nation,region SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ n_name, sum(l_extendedprice * (1 - l_discount)) AS revenue FROM diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q06.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q06.groovy index a63c041b137fdc..5b83fad0bdad70 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q06.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q06.groovy @@ -19,7 +19,7 @@ suite("q06_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q06_spill") { qt_q06 """ -- tables: lineitem SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ sum(l_extendedprice * l_discount) AS revenue FROM lineitem diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q07.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q07.groovy index 3a09e72b7ef7a2..0d72f4875c462d 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q07.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q07.groovy @@ -19,7 +19,7 @@ suite("q07_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q07_spill") { qt_q07 """ -- tables: supplier,lineitem,orders,customer,nation SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ supp_nation, cust_nation, l_year, diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q08.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q08.groovy index 0bdf04392922eb..a9fbc45e8f0ec4 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q08.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q08.groovy @@ -19,7 +19,7 @@ suite("q08_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q08_spill") { qt_q08 """ -- tables: part,supplier,lineitem,orders,customer,nation,region SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ o_year, sum(CASE WHEN nation = 'BRAZIL' diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q09.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q09.groovy index 282e20b9bffc76..6ef0ed78177524 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q09.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q09.groovy @@ -19,7 +19,7 @@ suite("q09_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q10.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q10.groovy index 2626bff0fea6ed..1b4b64427a9d0c 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q10.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q10.groovy @@ -19,7 +19,7 @@ suite("q10_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q10_spill") { qt_q10 """ -- tables: customer,orders,lineitem,nation SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) AS revenue, diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q11.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q11.groovy index 07b8a3b97715c9..7687eb21854609 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q11.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q11.groovy @@ -19,7 +19,7 @@ suite("q11_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q11_spill") { qt_q11 """ -- tables: partsupp,supplier,nation SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ ps_partkey, sum(ps_supplycost * ps_availqty) AS value FROM diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q12.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q12.groovy index 73f5b19bdb2cfb..4f08be43b85769 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q12.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q12.groovy @@ -19,7 +19,7 @@ suite("q12_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q12_spill") { qt_q12 """ -- tables: orders,lineitem SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ l_shipmode, sum(CASE WHEN o_orderpriority = '1-URGENT' diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q13.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q13.groovy index 64cd5e0babd008..3eb7c07c0badb3 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q13.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q13.groovy @@ -19,7 +19,7 @@ suite("q13_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q13_spill") { qt_q13 """ -- tables: customer SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ c_count, count(*) AS custdist FROM ( diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q14.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q14.groovy index 3bb423b92f1da1..cd3ea166dd4ed5 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q14.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q14.groovy @@ -19,7 +19,7 @@ suite("q14_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q14_spill") { qt_q14 """ -- tables: lineitem,part SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q15.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q15.groovy index ffe0e68c5e8d90..290e2cb55ba1f3 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q15.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q15.groovy @@ -19,14 +19,14 @@ suite("q15_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; """ qt_q15 """ SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ s_suppkey, s_name, s_address, diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q16.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q16.groovy index b8f5347dacb6dc..d4743a215990f9 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q16.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q16.groovy @@ -19,7 +19,7 @@ suite("q16_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q16_spill") { qt_q16 """ -- tables: partsupp,part,supplier SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ p_brand, p_type, p_size, diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q17.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q17.groovy index 01aeaa49e3d217..88eb0fcc207fbb 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q17.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q17.groovy @@ -19,7 +19,7 @@ suite("q17_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q17_spill") { qt_q17 """ -- tables: lineitem,part SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q18.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q18.groovy index b8d712928cf351..87549541ba8f49 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q18.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q18.groovy @@ -19,7 +19,7 @@ suite("q18_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q18_spill") { qt_q18 """ -- tables: customer,orders,lineitem SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ c_name, c_custkey, o_orderkey, diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q19.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q19.groovy index 511f875a6544f8..a29a0711647894 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q19.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q19.groovy @@ -19,7 +19,7 @@ suite("q19_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q19_spill") { qt_q19 """ -- tables: lineitem,part SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ sum(l_extendedprice * (1 - l_discount)) AS revenue FROM lineitem, diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q20.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q20.groovy index 21a57bcbbe02db..02b809e79706da 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q20.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q20.groovy @@ -19,7 +19,7 @@ suite("q20_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q20_spill") { qt_q20 """ -- tables: supplier,nation,partsupp,lineitem,part SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ s_name, s_address FROM diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q21.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q21.groovy index c32546362ec0bc..7d82a4c8875724 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q21.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q21.groovy @@ -19,7 +19,7 @@ suite("q21_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q21_spill") { qt_q21 """ -- tables: supplier,lineitem,orders,nation SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ s_name, count(*) AS numwait FROM diff --git a/regression-test/suites/tpch_sf0.1_p1/spill/q22.groovy b/regression-test/suites/tpch_sf0.1_p1/spill/q22.groovy index 26a15704d9b9ec..45a26a0db68e3c 100644 --- a/regression-test/suites/tpch_sf0.1_p1/spill/q22.groovy +++ b/regression-test/suites/tpch_sf0.1_p1/spill/q22.groovy @@ -19,7 +19,7 @@ suite("q22_spill") { set enable_force_spill=true; """ sql """ - set min_revocable_mem=1; + set spill_min_revocable_mem=1; """ sql """ use regression_test_tpch_sf0_1_p1; @@ -27,7 +27,7 @@ suite("q22_spill") { qt_q22 """ -- tables: orders,customer SELECT -/*+SET_VAR(enable_force_spill=true, min_revocable_mem=1)*/ +/*+SET_VAR(enable_force_spill=true, spill_min_revocable_mem=1)*/ cntrycode, count(*) AS numcust, sum(c_acctbal) AS totacctbal diff --git a/regression-test/suites/tpch_sf1_p2/spill/partitioned_agg_fault_injection.groovy b/regression-test/suites/tpch_sf1_p2/spill/partitioned_agg_fault_injection.groovy index f80b0a987bd134..f568e5d8f04220 100644 --- a/regression-test/suites/tpch_sf1_p2/spill/partitioned_agg_fault_injection.groovy +++ b/regression-test/suites/tpch_sf1_p2/spill/partitioned_agg_fault_injection.groovy @@ -19,7 +19,7 @@ suite("partitioned_agg_fault_injection", "p2, nonConcurrent") { multi_sql """ use regression_test_tpch_sf1_p2; set enable_force_spill=true; - set min_revocable_mem=1024; + set spill_min_revocable_mem=1024; """ def test_sql = """ select diff --git a/regression-test/suites/tpch_sf1_p2/spill/partitioned_hash_join_fault_injection.groovy b/regression-test/suites/tpch_sf1_p2/spill/partitioned_hash_join_fault_injection.groovy index b79e584ca97b5f..6f5e9d58e77b1c 100644 --- a/regression-test/suites/tpch_sf1_p2/spill/partitioned_hash_join_fault_injection.groovy +++ b/regression-test/suites/tpch_sf1_p2/spill/partitioned_hash_join_fault_injection.groovy @@ -19,7 +19,7 @@ suite("partitioned_hash_join_fault_injection", "p2, nonConcurrent") { multi_sql """ use regression_test_tpch_sf1_p2; set enable_force_spill=true; - set min_revocable_mem=1024; + set spill_min_revocable_mem=1024; """ def test_sql = """ SELECT diff --git a/regression-test/suites/tpch_sf1_p2/spill/spill_sort_fault_injection.groovy b/regression-test/suites/tpch_sf1_p2/spill/spill_sort_fault_injection.groovy index 3030b123e45170..37870221243526 100644 --- a/regression-test/suites/tpch_sf1_p2/spill/spill_sort_fault_injection.groovy +++ b/regression-test/suites/tpch_sf1_p2/spill/spill_sort_fault_injection.groovy @@ -19,7 +19,7 @@ suite("spill_sort_fault_injection", "p2, nonConcurrent") { multi_sql """ use regression_test_tpch_sf1_p2; set enable_force_spill=true; - set min_revocable_mem=1024; + set spill_min_revocable_mem=1024; """ def test_sql = """ select diff --git a/regression-test/suites/variant_p0/load.groovy b/regression-test/suites/variant_p0/load.groovy index cd5e9ee523d922..2a260b959d1960 100644 --- a/regression-test/suites/variant_p0/load.groovy +++ b/regression-test/suites/variant_p0/load.groovy @@ -290,7 +290,7 @@ suite("regression_test_variant", "p0"){ sql """insert into ${table_name} values (5, '{"i" : 1}'), (1, '{"a" : 1}')""" sql """insert into ${table_name} values (6, '{"j" : 1}'), (1, '{"a" : 1}')""" sql """insert into ${table_name} values (6, '{"k" : 1}'), (1, '{"a" : 1}')""" - sql "select /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=1,parallel_pipeline_task_num=4,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=16,parallel_scan_min_rows_per_scanner=128,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=2,enable_parallel_result_sink=false,sort_phase_num=0,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=true,enable_local_exchange=true,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=true,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=false,enable_delete_sub_predicate_v2=true,min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_join_spill=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ * from ${table_name}" + sql "select /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=1,parallel_pipeline_task_num=4,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=true,parallel_scan_max_scanners_count=16,parallel_scan_min_rows_per_scanner=128,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=2,enable_parallel_result_sink=false,sort_phase_num=0,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=true,enable_local_exchange=true,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=true,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=false,enable_delete_sub_predicate_v2=true,spill_min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5) */ * from ${table_name}" qt_sql_36_1 "select cast(v['a'] as int), cast(v['b'] as int), cast(v['c'] as int) from ${table_name} order by k limit 10" sql "DELETE FROM ${table_name} WHERE k=1" sql "select * from ${table_name}" diff --git a/regression-test/suites/variant_p0/nested.groovy b/regression-test/suites/variant_p0/nested.groovy index 7235386960fb61..229f1b57559528 100644 --- a/regression-test/suites/variant_p0/nested.groovy +++ b/regression-test/suites/variant_p0/nested.groovy @@ -88,7 +88,7 @@ suite("regression_test_variant_nested", "p0"){ qt_sql """select /*+SET_VAR(batch_size=1024,broker_load_batch_size=16352,disable_streaming_preaggregations=true,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_ parallel_pipeline_task_num=7,parallel_fragment_exec_instance_num=4,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=false,parallel_scan_max_scanners_count=16 -,parallel_scan_min_rows_per_scanner=128,enable_fold_constant_by_be=false,enable_rewrite_element_at_to_slot=true,runtime_filter_type=2,enable_parallel_result_sink=true,sort_phase_num=0,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=false,enable_local_exchange=true,partitioned_hash_join_rows_threshold=8,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=true,enable_delete_sub_predicate_v2=true,min_revocable_mem=1,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_join_spill=true,enable_sort_spill=true,enable_agg_spill=true,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ * from var_nested where v['k2'] = 'some' order by k limit 10""" +,parallel_scan_min_rows_per_scanner=128,enable_fold_constant_by_be=false,enable_rewrite_element_at_to_slot=true,runtime_filter_type=2,enable_parallel_result_sink=true,sort_phase_num=0,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=false,enable_local_exchange=true,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=true,enable_delete_sub_predicate_v2=true,spill_min_revocable_mem=1,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_spill=true,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5) */ * from var_nested where v['k2'] = 'some' order by k limit 10""" qt_sql """select * from var_nested where v['k2'] = 'some' and array_contains(cast(v['nested1']['nested2']['a'] as array), 10) order by k limit 1;""" sql """INSERT INTO var_nested SELECT *, '{"k1":1, "k2": "some", "k3" : [1234], "k4" : 1.10000, "k5" : [[123]], "nested1" : {"nested2" : [{"a" : 10, "b" : 1.1, "c" : "1111"}]}}' FROM numbers("number" = "4096") where number > 1024 limit 1024;""" @@ -117,7 +117,7 @@ parallel_pipeline_task_num=7,parallel_fragment_exec_instance_num=4,profile_level properties("replication_num" = "1", "disable_auto_compaction" = "false", "enable_unique_key_merge_on_write" = "true", "variant_enable_flatten_nested" = "true"); """ sql """insert into var_nested2 select * from var_nested order by k limit 1024""" - qt_sql """select /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=true,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=5,parallel_pipeline_task_num=1,profile_level=1,enable_pipeline_engine=false,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=false,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=false,enable_local_exchange=false,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=true,enable_delete_sub_predicate_v2=true,min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_join_spill=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ * from var_nested2 order by k limit 10;""" + qt_sql """select /*+SET_VAR(batch_size=4064,broker_load_batch_size=16352,disable_streaming_preaggregations=true,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=5,parallel_pipeline_task_num=1,profile_level=1,enable_pipeline_engine=false,enable_parallel_scan=true,parallel_scan_max_scanners_count=48,parallel_scan_min_rows_per_scanner=16384,enable_fold_constant_by_be=true,enable_rewrite_element_at_to_slot=true,runtime_filter_type=12,enable_parallel_result_sink=false,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=false,enable_local_exchange=false,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_common_expr_pushdown_for_inverted_index=true,enable_delete_sub_predicate_v2=true,spill_min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,max_fetch_remote_schema_tablet_count=512,enable_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5) */ * from var_nested2 order by k limit 10;""" qt_sql """select v['nested'] from var_nested2 where k < 10 order by k limit 10;""" // 0. nomal explode variant array order_qt_explode_sql """select count(),cast(vv['xx'] as int) from var_nested lateral view explode_variant_array(v['nested']) tmp as vv where vv['xx'] = 10 group by cast(vv['xx'] as int)""" diff --git a/regression-test/suites/variant_p0/test_sub_path_pruning.groovy b/regression-test/suites/variant_p0/test_sub_path_pruning.groovy index f09f4713ad29e5..9b8cfb8f38f0b5 100644 --- a/regression-test/suites/variant_p0/test_sub_path_pruning.groovy +++ b/regression-test/suites/variant_p0/test_sub_path_pruning.groovy @@ -139,7 +139,7 @@ suite("variant_sub_path_pruning", "variant_type"){ // two children order_qt_sql """ - select /*+SET_VAR(batch_size=50,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=6,parallel_pipeline_task_num=2,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=false,parallel_scan_max_scanners_count=16,parallel_scan_min_rows_per_scanner=128,enable_fold_constant_by_be=false,enable_rewrite_element_at_to_slot=true,runtime_filter_type=2,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=true,enable_local_exchange=false,partitioned_hash_join_rows_threshold=1048576,partitioned_hash_agg_rows_threshold=8,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=true,min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,enable_join_spill=false,enable_sort_spill=false,enable_agg_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,external_agg_partition_bits=5) */ dt['a'] as c1 from pruning_test union all select dt['a'] as c1 from pruning_test; + select /*+SET_VAR(batch_size=50,disable_streaming_preaggregations=false,enable_distinct_streaming_aggregation=true,parallel_fragment_exec_instance_num=6,parallel_pipeline_task_num=2,profile_level=1,enable_pipeline_engine=true,enable_parallel_scan=false,parallel_scan_max_scanners_count=16,parallel_scan_min_rows_per_scanner=128,enable_fold_constant_by_be=false,enable_rewrite_element_at_to_slot=true,runtime_filter_type=2,enable_nereids_planner=true,rewrite_or_to_in_predicate_threshold=2,enable_function_pushdown=true,enable_common_expr_pushdown=true,enable_local_exchange=false,partition_pruning_expand_threshold=10,enable_share_hash_table_for_broadcast_join=false,enable_two_phase_read_opt=true,enable_delete_sub_predicate_v2=true,spill_min_revocable_mem=33554432,fetch_remote_schema_timeout_seconds=120,enable_spill=false,enable_force_spill=false,data_queue_max_blocks=1,spill_streaming_agg_mem_limit=268435456,spill_aggregation_partition_count=5) */ dt['a'] as c1 from pruning_test union all select dt['a'] as c1 from pruning_test; """ order_qt_sql """select c1['a'] from (select dt as c1 from pruning_test union all select dt as c1 from pruning_test) v1;""" order_qt_sql """select c1['b'] from (select dt['a'] as c1 from pruning_test union all select dt['a'] as c1 from pruning_test) v1;""" diff --git a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy index b1bbae2e3ce477..836efd88e96f82 100644 --- a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy +++ b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy @@ -451,7 +451,7 @@ suite("test_crud_wlg") { // test workload spill property // 1 create group test { - sql "create workload group if not exists spill_group_test_failed properties ( 'memory_low_watermark'='90%');" + sql "create workload group if not exists spill_group_test_failed properties ( 'memory_low_watermark'='96%');" exception "should bigger than memory_low_watermark" } sql "create workload group if not exists spill_group_test properties ( 'memory_low_watermark'='10%','memory_high_watermark'='10%');"