From 2e3d35f4f37b63b3cde89739f02a664aab89b228 Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Thu, 14 Mar 2024 14:56:47 +0800 Subject: [PATCH 01/10] [refactor](rename) rename task group to workload group in be (#32204) --------- Co-authored-by: yiguolei --- be/src/agent/cgroup_cpu_ctl.cpp | 2 +- be/src/agent/workload_group_listener.cpp | 35 +++--- be/src/common/daemon.cpp | 2 +- be/src/pipeline/pipeline_task.h | 2 +- be/src/pipeline/pipeline_tracing.cpp | 10 +- be/src/pipeline/pipeline_tracing.h | 4 +- be/src/pipeline/pipeline_x/pipeline_x_task.h | 2 +- be/src/pipeline/task_queue.h | 2 +- be/src/pipeline/task_scheduler.h | 2 +- be/src/runtime/exec_env.h | 8 +- be/src/runtime/exec_env_init.cpp | 10 +- be/src/runtime/fragment_mgr.cpp | 16 +-- be/src/runtime/memory/mem_tracker_limiter.cpp | 8 +- be/src/runtime/memory/mem_tracker_limiter.h | 14 +-- be/src/runtime/query_context.cpp | 30 ++--- be/src/runtime/query_context.h | 6 +- .../workload_group.cpp} | 108 +++++++++--------- .../workload_group.h} | 31 +++-- .../workload_group_manager.cpp} | 75 ++++++------ .../workload_group_manager.h} | 23 ++-- be/src/util/mem_info.cpp | 46 ++++---- be/src/vec/exec/scan/scanner_context.h | 5 +- 22 files changed, 213 insertions(+), 228 deletions(-) rename be/src/runtime/{task_group/task_group.cpp => workload_group/workload_group.cpp} (81%) rename be/src/runtime/{task_group/task_group.h => workload_group/workload_group.h} (87%) rename be/src/runtime/{task_group/task_group_manager.cpp => workload_group/workload_group_manager.cpp} (59%) rename be/src/runtime/{task_group/task_group_manager.h => workload_group/workload_group_manager.h} (70%) diff --git a/be/src/agent/cgroup_cpu_ctl.cpp b/be/src/agent/cgroup_cpu_ctl.cpp index 5263c0605306d8..3fe0778ecbaff1 100644 --- a/be/src/agent/cgroup_cpu_ctl.cpp +++ b/be/src/agent/cgroup_cpu_ctl.cpp @@ -132,7 +132,7 @@ Status CgroupV1CpuCtl::init() { if (_tg_id == -1) { // means current cgroup cpu ctl is just used to clear dir, - // it does not contains task group. + // it does not contains workload group. // todo(wb) rethinking whether need to refactor cgroup_cpu_ctl _init_succ = true; LOG(INFO) << "init cgroup cpu query path succ, path=" << _cgroup_v1_cpu_query_path; diff --git a/be/src/agent/workload_group_listener.cpp b/be/src/agent/workload_group_listener.cpp index 1d5a8544e1106e..f98315fa4332e5 100644 --- a/be/src/agent/workload_group_listener.cpp +++ b/be/src/agent/workload_group_listener.cpp @@ -17,8 +17,8 @@ #include "agent/workload_group_listener.h" -#include "runtime/task_group/task_group.h" -#include "runtime/task_group/task_group_manager.h" +#include "runtime/workload_group/workload_group.h" +#include "runtime/workload_group/workload_group_manager.h" #include "util/mem_info.h" #include "util/parse_util.h" @@ -32,36 +32,37 @@ void WorkloadGroupListener::handle_topic_info(const std::vector& topi } // 1 parse topicinfo to group info - taskgroup::TaskGroupInfo task_group_info; - Status ret = taskgroup::TaskGroupInfo::parse_topic_info(topic_info.workload_group_info, - &task_group_info); + WorkloadGroupInfo workload_group_info; + Status ret = WorkloadGroupInfo::parse_topic_info(topic_info.workload_group_info, + &workload_group_info); if (!ret.ok()) { - LOG(INFO) << "parse topic info failed, tg_id=" << task_group_info.id + LOG(INFO) << "parse topic info failed, tg_id=" << workload_group_info.id << ", reason:" << ret.to_string(); continue; } - current_wg_ids.insert(task_group_info.id); + current_wg_ids.insert(workload_group_info.id); - // 2 update task group - auto tg = _exec_env->task_group_manager()->get_or_create_task_group(task_group_info); + // 2 update workload group + auto tg = + _exec_env->workload_group_mgr()->get_or_create_workload_group(workload_group_info); // 3 set cpu soft hard limit switch - _exec_env->task_group_manager()->_enable_cpu_hard_limit.store( - task_group_info.enable_cpu_hard_limit); + _exec_env->workload_group_mgr()->_enable_cpu_hard_limit.store( + workload_group_info.enable_cpu_hard_limit); // 4 create and update task scheduler - tg->upsert_task_scheduler(&task_group_info, _exec_env); + tg->upsert_task_scheduler(&workload_group_info, _exec_env); - LOG(INFO) << "update task group finish, tg info=" << tg->debug_string() + LOG(INFO) << "update workload group finish, tg info=" << tg->debug_string() << ", enable_cpu_hard_limit=" - << (_exec_env->task_group_manager()->enable_cpu_hard_limit() ? "true" : "false") - << ", cgroup cpu_shares=" << task_group_info.cgroup_cpu_shares - << ", cgroup cpu_hard_limit=" << task_group_info.cgroup_cpu_hard_limit + << (_exec_env->workload_group_mgr()->enable_cpu_hard_limit() ? "true" : "false") + << ", cgroup cpu_shares=" << workload_group_info.cgroup_cpu_shares + << ", cgroup cpu_hard_limit=" << workload_group_info.cgroup_cpu_hard_limit << ", enable_cgroup_cpu_soft_limit=" << (config::enable_cgroup_cpu_soft_limit ? "true" : "false") << ", cgroup home path=" << config::doris_cgroup_cpu_path; } - _exec_env->task_group_manager()->delete_task_group_by_ids(current_wg_ids); + _exec_env->workload_group_mgr()->delete_workload_group_by_ids(current_wg_ids); } } // namespace doris \ No newline at end of file diff --git a/be/src/common/daemon.cpp b/be/src/common/daemon.cpp index d89800a19f60fa..eb130e1d19f62c 100644 --- a/be/src/common/daemon.cpp +++ b/be/src/common/daemon.cpp @@ -51,7 +51,7 @@ #include "runtime/memory/mem_tracker.h" #include "runtime/memory/mem_tracker_limiter.h" #include "runtime/runtime_query_statistics_mgr.h" -#include "runtime/task_group/task_group_manager.h" +#include "runtime/workload_group/workload_group_manager.h" #include "util/cpu_info.h" #include "util/debug_util.h" #include "util/disk_info.h" diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h index 517d6b8a8deee0..b9a5cb06ff872d 100644 --- a/be/src/pipeline/pipeline_task.h +++ b/be/src/pipeline/pipeline_task.h @@ -26,7 +26,7 @@ #include "common/status.h" #include "exec/operator.h" #include "pipeline.h" -#include "runtime/task_group/task_group.h" +#include "runtime/workload_group/workload_group.h" #include "util/runtime_profile.h" #include "util/stopwatch.hpp" #include "vec/core/block.h" diff --git a/be/src/pipeline/pipeline_tracing.cpp b/be/src/pipeline/pipeline_tracing.cpp index 94675f77f63f7d..f635f49682fca4 100644 --- a/be/src/pipeline/pipeline_tracing.cpp +++ b/be/src/pipeline/pipeline_tracing.cpp @@ -46,10 +46,10 @@ void PipelineTracerContext::record(ScheduleRecord record) { } } -void PipelineTracerContext::end_query(TUniqueId query_id, uint64_t task_group) { +void PipelineTracerContext::end_query(TUniqueId query_id, uint64_t workload_group) { { std::unique_lock l(_tg_lock); - _id_to_taskgroup[query_id] = task_group; + _id_to_workload_group[query_id] = workload_group; } if (_dump_type == RecordType::PerQuery) { _dump(query_id); @@ -113,7 +113,7 @@ void PipelineTracerContext::_dump(TUniqueId query_id) { uint64_t v = 0; { std::unique_lock l(_tg_lock); - v = _id_to_taskgroup[query_id]; + v = _id_to_workload_group[query_id]; } auto tmp_str = record.to_string(v); auto text = Slice {tmp_str}; @@ -140,7 +140,7 @@ void PipelineTracerContext::_dump(TUniqueId query_id) { uint64_t v = 0; { std::unique_lock l(_tg_lock); - v = _id_to_taskgroup[query_id]; + v = _id_to_workload_group[query_id]; } auto tmp_str = record.to_string(v); auto text = Slice {tmp_str}; @@ -156,7 +156,7 @@ void PipelineTracerContext::_dump(TUniqueId query_id) { _datas.erase(query_id); { std::unique_lock l(_tg_lock); - _id_to_taskgroup.erase(query_id); + _id_to_workload_group.erase(query_id); } } } // namespace doris::pipeline diff --git a/be/src/pipeline/pipeline_tracing.h b/be/src/pipeline/pipeline_tracing.h index 3160148c5707bc..adebd84b05fc6c 100644 --- a/be/src/pipeline/pipeline_tracing.h +++ b/be/src/pipeline/pipeline_tracing.h @@ -61,7 +61,7 @@ class PipelineTracerContext { }; void record(ScheduleRecord record); // record one schedule record void end_query(TUniqueId query_id, - uint64_t task_group); // tell context this query is end. may leads to dump. + uint64_t workload_group); // tell context this query is end. may leads to dump. Status change_record_params(const std::map& params); bool enabled() const { return !(_dump_type == RecordType::None); } @@ -72,7 +72,7 @@ class PipelineTracerContext { std::mutex _data_lock; // lock for map, not map items. phmap::flat_hash_map _datas; std::mutex _tg_lock; //TODO: use an lockfree DS - phmap::flat_hash_map _id_to_taskgroup; + phmap::flat_hash_map _id_to_workload_group; RecordType _dump_type = RecordType::None; std::filesystem::path _dir = config::pipeline_tracing_log_dir; diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.h b/be/src/pipeline/pipeline_x/pipeline_x_task.h index 8707de4b54c0ab..c754af645ef7c5 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.h +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.h @@ -28,7 +28,7 @@ #include "pipeline/pipeline.h" #include "pipeline/pipeline_task.h" #include "pipeline/pipeline_x/dependency.h" -#include "runtime/task_group/task_group.h" +#include "runtime/workload_group/workload_group.h" #include "util/runtime_profile.h" #include "util/stopwatch.hpp" #include "vec/core/block.h" diff --git a/be/src/pipeline/task_queue.h b/be/src/pipeline/task_queue.h index 9440fc18d83ff2..a6799fdb0c69d9 100644 --- a/be/src/pipeline/task_queue.h +++ b/be/src/pipeline/task_queue.h @@ -30,7 +30,7 @@ #include "common/status.h" #include "pipeline_task.h" -#include "runtime/task_group/task_group.h" +#include "runtime/workload_group/workload_group.h" namespace doris { namespace pipeline { diff --git a/be/src/pipeline/task_scheduler.h b/be/src/pipeline/task_scheduler.h index 3074cf02afc1f5..5bbf85fad452fd 100644 --- a/be/src/pipeline/task_scheduler.h +++ b/be/src/pipeline/task_scheduler.h @@ -30,7 +30,7 @@ #include "common/status.h" #include "gutil/ref_counted.h" #include "pipeline_task.h" -#include "runtime/task_group/task_group.h" +#include "runtime/workload_group/workload_group.h" #include "util/thread.h" namespace doris { diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 007d8380944463..3fdbf95855cd7a 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -51,9 +51,7 @@ class TaskScheduler; class BlockedTaskScheduler; struct RuntimeFilterTimerQueue; } // namespace pipeline -namespace taskgroup { -class TaskGroupManager; -} +class WorkloadGroupMgr; namespace io { class FileCacheFactory; } // namespace io @@ -152,7 +150,7 @@ class ExecEnv { ClientCache* broker_client_cache() { return _broker_client_cache; } pipeline::TaskScheduler* pipeline_task_scheduler() { return _without_group_task_scheduler; } - taskgroup::TaskGroupManager* task_group_manager() { return _task_group_manager; } + WorkloadGroupMgr* workload_group_mgr() { return _workload_group_manager; } WorkloadSchedPolicyMgr* workload_sched_policy_mgr() { return _workload_sched_mgr; } RuntimeQueryStatiticsMgr* runtime_query_statistics_mgr() { return _runtime_query_statistics_mgr; @@ -336,7 +334,7 @@ class ExecEnv { FragmentMgr* _fragment_mgr = nullptr; pipeline::TaskScheduler* _without_group_task_scheduler = nullptr; - taskgroup::TaskGroupManager* _task_group_manager = nullptr; + WorkloadGroupMgr* _workload_group_manager = nullptr; ResultCache* _result_cache = nullptr; TMasterInfo* _master_info = nullptr; diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index f405893b8630e0..aad24b068b4379 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -77,9 +77,9 @@ #include "runtime/small_file_mgr.h" #include "runtime/stream_load/new_load_stream_mgr.h" #include "runtime/stream_load/stream_load_executor.h" -#include "runtime/task_group/task_group_manager.h" #include "runtime/thread_context.h" #include "runtime/user_function_cache.h" +#include "runtime/workload_group/workload_group_manager.h" #include "runtime/workload_management/workload_sched_policy_mgr.h" #include "service/backend_options.h" #include "service/backend_service.h" @@ -217,7 +217,7 @@ Status ExecEnv::_init(const std::vector& store_paths, init_file_cache_factory(); _pipeline_tracer_ctx = std::make_unique(); // before query RETURN_IF_ERROR(init_pipeline_task_scheduler()); - _task_group_manager = new taskgroup::TaskGroupManager(); + _workload_group_manager = new WorkloadGroupMgr(); _scanner_scheduler = new doris::vectorized::ScannerScheduler(); _fragment_mgr = new FragmentMgr(this); _result_cache = new ResultCache(config::query_cache_max_size_mb, @@ -308,7 +308,7 @@ Status ExecEnv::init_pipeline_task_scheduler() { executors_size = CpuInfo::num_cores(); } - // TODO pipeline task group combie two blocked schedulers. + // TODO pipeline workload group combie two blocked schedulers. auto t_queue = std::make_shared(executors_size); _without_group_block_scheduler = std::make_shared("PipeNoGSchePool"); @@ -554,7 +554,7 @@ void ExecEnv::destroy() { SAFE_STOP(_without_group_task_scheduler); // stop pipline step 2, cgroup execution SAFE_SHUTDOWN(_global_block_scheduler.get()); - SAFE_STOP(_task_group_manager); + SAFE_STOP(_workload_group_manager); SAFE_STOP(_external_scan_context_mgr); SAFE_STOP(_fragment_mgr); @@ -620,7 +620,7 @@ void ExecEnv::destroy() { SAFE_DELETE(_result_cache); SAFE_DELETE(_fragment_mgr); SAFE_DELETE(_workload_sched_mgr); - SAFE_DELETE(_task_group_manager); + SAFE_DELETE(_workload_group_manager); SAFE_DELETE(_file_cache_factory); SAFE_DELETE(_runtime_filter_timer_queue); // TODO(zhiqiang): Maybe we should call shutdown before release thread pool? diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index 0f45973d2925d4..16cbcc9a4ccf2e 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -70,10 +70,10 @@ #include "runtime/stream_load/new_load_stream_mgr.h" #include "runtime/stream_load/stream_load_context.h" #include "runtime/stream_load/stream_load_executor.h" -#include "runtime/task_group/task_group.h" -#include "runtime/task_group/task_group_manager.h" #include "runtime/thread_context.h" #include "runtime/types.h" +#include "runtime/workload_group/workload_group.h" +#include "runtime/workload_group/workload_group_manager.h" #include "runtime/workload_management/workload_query_info.h" #include "service/backend_options.h" #include "util/debug_util.h" @@ -639,15 +639,15 @@ Status FragmentMgr::_get_query_ctx(const Params& params, TUniqueId query_id, boo if (params.__isset.workload_groups && !params.workload_groups.empty()) { uint64_t tg_id = params.workload_groups[0].id; - taskgroup::TaskGroupPtr task_group_ptr = - _exec_env->task_group_manager()->get_task_group_by_id(tg_id); - if (task_group_ptr != nullptr) { - RETURN_IF_ERROR(query_ctx->set_task_group(task_group_ptr)); + WorkloadGroupPtr workload_group_ptr = + _exec_env->workload_group_mgr()->get_task_group_by_id(tg_id); + if (workload_group_ptr != nullptr) { + RETURN_IF_ERROR(query_ctx->set_workload_group(workload_group_ptr)); _exec_env->runtime_query_statistics_mgr()->set_workload_group_id(print_id(query_id), tg_id); LOG(INFO) << "Query/load id: " << print_id(query_ctx->query_id()) - << ", use task group: " << task_group_ptr->debug_string() + << ", use workload group: " << workload_group_ptr->debug_string() << ", is pipeline: " << ((int)is_pipeline) << ", enable cgroup soft limit: " << ((int)config::enable_cgroup_cpu_soft_limit); @@ -657,7 +657,7 @@ Status FragmentMgr::_get_query_ctx(const Params& params, TUniqueId query_id, boo } } // There is some logic in query ctx's dctor, we could not check if exists and delete the - // temp query ctx now. For example, the query id maybe removed from task group's queryset. + // temp query ctx now. For example, the query id maybe removed from workload group's queryset. _query_ctx_map.insert(std::make_pair(query_ctx->query_id(), query_ctx)); LOG(INFO) << "Register query/load memory tracker, query/load id: " << print_id(query_ctx->query_id()) diff --git a/be/src/runtime/memory/mem_tracker_limiter.cpp b/be/src/runtime/memory/mem_tracker_limiter.cpp index 680c2917cb813f..83b81f5160dc02 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.cpp +++ b/be/src/runtime/memory/mem_tracker_limiter.cpp @@ -30,8 +30,8 @@ #include "olap/memtable_memory_limiter.h" #include "runtime/exec_env.h" #include "runtime/fragment_mgr.h" -#include "runtime/task_group/task_group.h" #include "runtime/thread_context.h" +#include "runtime/workload_group/workload_group.h" #include "service/backend_options.h" #include "util/mem_info.h" #include "util/perf_counters.h" @@ -396,8 +396,7 @@ int64_t MemTrackerLimiter::free_top_memory_query(int64_t min_free_mem, } int64_t MemTrackerLimiter::tg_free_top_memory_query( - int64_t min_free_mem, Type type, - std::vector& tracker_groups, + int64_t min_free_mem, Type type, std::vector& tracker_groups, const std::function& cancel_msg, RuntimeProfile* profile, GCType gctype) { return free_top_memory_query(min_free_mem, type, tracker_groups, cancel_msg, profile, gctype); @@ -531,8 +530,7 @@ int64_t MemTrackerLimiter::free_top_overcommit_query(int64_t min_free_mem, } int64_t MemTrackerLimiter::tg_free_top_overcommit_query( - int64_t min_free_mem, Type type, - std::vector& tracker_groups, + int64_t min_free_mem, Type type, std::vector& tracker_groups, const std::function& cancel_msg, RuntimeProfile* profile, GCType gctype) { return free_top_overcommit_query(min_free_mem, type, tracker_groups, cancel_msg, profile, diff --git a/be/src/runtime/memory/mem_tracker_limiter.h b/be/src/runtime/memory/mem_tracker_limiter.h index 9703981af6366d..b10f41f6691e07 100644 --- a/be/src/runtime/memory/mem_tracker_limiter.h +++ b/be/src/runtime/memory/mem_tracker_limiter.h @@ -45,11 +45,9 @@ class RuntimeProfile; constexpr auto MEM_TRACKER_GROUP_NUM = 1000; -namespace taskgroup { -struct TgTrackerLimiterGroup; -class TaskGroup; -using TaskGroupPtr = std::shared_ptr; -} // namespace taskgroup +struct WgTrackerLimiterGroup; +class WorkloadGroup; +using WorkloadGroupPtr = std::shared_ptr; // Track and limit the memory usage of process and query. // Contains an limit, arranged into a tree structure. @@ -195,8 +193,7 @@ class MemTrackerLimiter final : public MemTracker { RuntimeProfile* profile, GCType gctype); static int64_t tg_free_top_memory_query( - int64_t min_free_mem, Type type, - std::vector& tracker_groups, + int64_t min_free_mem, Type type, std::vector& tracker_groups, const std::function& cancel_msg, RuntimeProfile* profile, GCType gctype); @@ -219,8 +216,7 @@ class MemTrackerLimiter final : public MemTracker { RuntimeProfile* profile, GCType gctype); static int64_t tg_free_top_overcommit_query( - int64_t min_free_mem, Type type, - std::vector& tracker_groups, + int64_t min_free_mem, Type type, std::vector& tracker_groups, const std::function& cancel_msg, RuntimeProfile* profile, GCType gctype); diff --git a/be/src/runtime/query_context.cpp b/be/src/runtime/query_context.cpp index 7dce8488ecabd2..56001c4498c1d2 100644 --- a/be/src/runtime/query_context.cpp +++ b/be/src/runtime/query_context.cpp @@ -23,7 +23,7 @@ #include "pipeline/pipeline_fragment_context.h" #include "pipeline/pipeline_x/dependency.h" #include "runtime/runtime_query_statistics_mgr.h" -#include "runtime/task_group/task_group_manager.h" +#include "runtime/workload_group/workload_group_manager.h" #include "util/mem_info.h" namespace doris { @@ -102,10 +102,10 @@ QueryContext::~QueryContext() { MemTracker::print_bytes(query_mem_tracker->peak_consumption())); } uint64_t group_id = 0; - if (_task_group) { - group_id = _task_group->id(); // before remove - _task_group->remove_mem_tracker_limiter(query_mem_tracker); - _task_group->remove_query(_query_id); + if (_workload_group) { + group_id = _workload_group->id(); // before remove + _workload_group->remove_mem_tracker_limiter(query_mem_tracker); + _workload_group->remove_query(_query_id); } _exec_env->runtime_query_statistics_mgr()->set_query_finished(print_id(_query_id)); @@ -205,7 +205,7 @@ void QueryContext::register_cpu_statistics() { } doris::pipeline::TaskScheduler* QueryContext::get_pipe_exec_scheduler() { - if (_task_group) { + if (_workload_group) { if (_task_scheduler) { return _task_scheduler; } @@ -214,21 +214,21 @@ doris::pipeline::TaskScheduler* QueryContext::get_pipe_exec_scheduler() { } ThreadPool* QueryContext::get_non_pipe_exec_thread_pool() { - if (_task_group) { + if (_workload_group) { return _non_pipe_thread_pool; } else { return nullptr; } } -Status QueryContext::set_task_group(taskgroup::TaskGroupPtr& tg) { - _task_group = tg; - // Should add query first, then the task group will not be deleted. - // see task_group_manager::delete_task_group_by_ids - RETURN_IF_ERROR(_task_group->add_query(_query_id)); - _task_group->add_mem_tracker_limiter(query_mem_tracker); - _task_group->get_query_scheduler(&_task_scheduler, &_scan_task_scheduler, - &_non_pipe_thread_pool, &_remote_scan_task_scheduler); +Status QueryContext::set_workload_group(WorkloadGroupPtr& tg) { + _workload_group = tg; + // Should add query first, then the workload group will not be deleted. + // see task_group_manager::delete_workload_group_by_ids + RETURN_IF_ERROR(_workload_group->add_query(_query_id)); + _workload_group->add_mem_tracker_limiter(query_mem_tracker); + _workload_group->get_query_scheduler(&_task_scheduler, &_scan_task_scheduler, + &_non_pipe_thread_pool, &_remote_scan_task_scheduler); return Status::OK(); } diff --git a/be/src/runtime/query_context.h b/be/src/runtime/query_context.h index 36c4f6ae110179..e47c42b4394972 100644 --- a/be/src/runtime/query_context.h +++ b/be/src/runtime/query_context.h @@ -32,11 +32,11 @@ #include "runtime/query_statistics.h" #include "runtime/runtime_filter_mgr.h" #include "runtime/runtime_predicate.h" -#include "task_group/task_group.h" #include "util/threadpool.h" #include "vec/exec/scan/scanner_scheduler.h" #include "vec/runtime/shared_hash_table_controller.h" #include "vec/runtime/shared_scanner_controller.h" +#include "workload_group/workload_group.h" namespace doris { @@ -163,7 +163,7 @@ class QueryContext { } } - Status set_task_group(taskgroup::TaskGroupPtr& tg); + Status set_workload_group(WorkloadGroupPtr& tg); int execution_timeout() const { return _query_options.__isset.execution_timeout ? _query_options.execution_timeout @@ -292,7 +292,7 @@ class QueryContext { std::shared_ptr _shared_scanner_controller; std::unordered_map _runtime_predicates; - taskgroup::TaskGroupPtr _task_group = nullptr; + WorkloadGroupPtr _workload_group = nullptr; std::unique_ptr _runtime_filter_mgr; const TQueryOptions _query_options; diff --git a/be/src/runtime/task_group/task_group.cpp b/be/src/runtime/workload_group/workload_group.cpp similarity index 81% rename from be/src/runtime/task_group/task_group.cpp rename to be/src/runtime/workload_group/workload_group.cpp index 53534444a5d79c..88e5a1221eb774 100644 --- a/be/src/runtime/task_group/task_group.cpp +++ b/be/src/runtime/workload_group/workload_group.cpp @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -#include "task_group.h" +#include "workload_group.h" #include #include @@ -38,7 +38,6 @@ #include "vec/exec/scan/scanner_scheduler.h" namespace doris { -namespace taskgroup { const static uint64_t CPU_SHARE_DEFAULT_VALUE = 1024; const static std::string MEMORY_LIMIT_DEFAULT_VALUE = "0%"; @@ -46,7 +45,7 @@ const static bool ENABLE_MEMORY_OVERCOMMIT_DEFAULT_VALUE = true; const static int CPU_HARD_LIMIT_DEFAULT_VALUE = -1; const static uint64_t CPU_SOFT_LIMIT_DEFAULT_VALUE = 1024; -TaskGroup::TaskGroup(const TaskGroupInfo& tg_info) +WorkloadGroup::WorkloadGroup(const WorkloadGroupInfo& tg_info) : _id(tg_info.id), _name(tg_info.name), _version(tg_info.version), @@ -59,7 +58,7 @@ TaskGroup::TaskGroup(const TaskGroupInfo& tg_info) _max_remote_scan_thread_num(tg_info.max_remote_scan_thread_num), _min_remote_scan_thread_num(tg_info.min_remote_scan_thread_num) {} -std::string TaskGroup::debug_string() const { +std::string WorkloadGroup::debug_string() const { std::shared_lock rl {_mutex}; return fmt::format( "TG[id = {}, name = {}, cpu_share = {}, memory_limit = {}, enable_memory_overcommit = " @@ -70,7 +69,7 @@ std::string TaskGroup::debug_string() const { _scan_thread_num, _max_remote_scan_thread_num, _min_remote_scan_thread_num); } -void TaskGroup::check_and_update(const TaskGroupInfo& tg_info) { +void WorkloadGroup::check_and_update(const WorkloadGroupInfo& tg_info) { if (UNLIKELY(tg_info.id != _id)) { return; } @@ -98,7 +97,7 @@ void TaskGroup::check_and_update(const TaskGroupInfo& tg_info) { } } -int64_t TaskGroup::memory_used() { +int64_t WorkloadGroup::memory_used() { int64_t used_memory = 0; for (auto& mem_tracker_group : _mem_tracker_limiter_pool) { std::lock_guard l(mem_tracker_group.group_lock); @@ -109,19 +108,19 @@ int64_t TaskGroup::memory_used() { return used_memory; } -void TaskGroup::add_mem_tracker_limiter(std::shared_ptr mem_tracker_ptr) { +void WorkloadGroup::add_mem_tracker_limiter(std::shared_ptr mem_tracker_ptr) { auto group_num = mem_tracker_ptr->group_num(); std::lock_guard l(_mem_tracker_limiter_pool[group_num].group_lock); _mem_tracker_limiter_pool[group_num].trackers.insert(mem_tracker_ptr); } -void TaskGroup::remove_mem_tracker_limiter(std::shared_ptr mem_tracker_ptr) { +void WorkloadGroup::remove_mem_tracker_limiter(std::shared_ptr mem_tracker_ptr) { auto group_num = mem_tracker_ptr->group_num(); std::lock_guard l(_mem_tracker_limiter_pool[group_num].group_lock); _mem_tracker_limiter_pool[group_num].trackers.erase(mem_tracker_ptr); } -int64_t TaskGroup::gc_memory(int64_t need_free_mem, RuntimeProfile* profile) { +int64_t WorkloadGroup::gc_memory(int64_t need_free_mem, RuntimeProfile* profile) { if (need_free_mem <= 0) { return 0; } @@ -202,97 +201,97 @@ int64_t TaskGroup::gc_memory(int64_t need_free_mem, RuntimeProfile* profile) { return freed_mem; } -Status TaskGroupInfo::parse_topic_info(const TWorkloadGroupInfo& workload_group_info, - taskgroup::TaskGroupInfo* task_group_info) { +Status WorkloadGroupInfo::parse_topic_info(const TWorkloadGroupInfo& tworkload_group_info, + WorkloadGroupInfo* workload_group_info) { // 1 id int tg_id = 0; - if (workload_group_info.__isset.id) { - tg_id = workload_group_info.id; + if (tworkload_group_info.__isset.id) { + tg_id = tworkload_group_info.id; } else { return Status::InternalError("workload group id is required"); } - task_group_info->id = tg_id; + workload_group_info->id = tg_id; // 2 name std::string name = "INVALID_NAME"; - if (workload_group_info.__isset.name) { - name = workload_group_info.name; + if (tworkload_group_info.__isset.name) { + name = tworkload_group_info.name; } - task_group_info->name = name; + workload_group_info->name = name; // 3 version int version = 0; - if (workload_group_info.__isset.version) { - version = workload_group_info.version; + if (tworkload_group_info.__isset.version) { + version = tworkload_group_info.version; } else { return Status::InternalError("workload group version is required"); } - task_group_info->version = version; + workload_group_info->version = version; // 4 cpu_share uint64_t cpu_share = CPU_SHARE_DEFAULT_VALUE; - if (workload_group_info.__isset.cpu_share) { - cpu_share = workload_group_info.cpu_share; + if (tworkload_group_info.__isset.cpu_share) { + cpu_share = tworkload_group_info.cpu_share; } - task_group_info->cpu_share = cpu_share; + workload_group_info->cpu_share = cpu_share; // 5 cpu hard limit int cpu_hard_limit = CPU_HARD_LIMIT_DEFAULT_VALUE; - if (workload_group_info.__isset.cpu_hard_limit) { - cpu_hard_limit = workload_group_info.cpu_hard_limit; + if (tworkload_group_info.__isset.cpu_hard_limit) { + cpu_hard_limit = tworkload_group_info.cpu_hard_limit; } - task_group_info->cpu_hard_limit = cpu_hard_limit; + workload_group_info->cpu_hard_limit = cpu_hard_limit; // 6 mem_limit std::string mem_limit_str = MEMORY_LIMIT_DEFAULT_VALUE; - if (workload_group_info.__isset.mem_limit) { - mem_limit_str = workload_group_info.mem_limit; + if (tworkload_group_info.__isset.mem_limit) { + mem_limit_str = tworkload_group_info.mem_limit; } bool is_percent = true; int64_t mem_limit = ParseUtil::parse_mem_spec(mem_limit_str, -1, MemInfo::mem_limit(), &is_percent); - task_group_info->memory_limit = mem_limit; + workload_group_info->memory_limit = mem_limit; // 7 mem overcommit bool enable_memory_overcommit = ENABLE_MEMORY_OVERCOMMIT_DEFAULT_VALUE; - if (workload_group_info.__isset.enable_memory_overcommit) { - enable_memory_overcommit = workload_group_info.enable_memory_overcommit; + if (tworkload_group_info.__isset.enable_memory_overcommit) { + enable_memory_overcommit = tworkload_group_info.enable_memory_overcommit; } - task_group_info->enable_memory_overcommit = enable_memory_overcommit; + workload_group_info->enable_memory_overcommit = enable_memory_overcommit; // 8 cpu soft limit or hard limit bool enable_cpu_hard_limit = false; - if (workload_group_info.__isset.enable_cpu_hard_limit) { - enable_cpu_hard_limit = workload_group_info.enable_cpu_hard_limit; + if (tworkload_group_info.__isset.enable_cpu_hard_limit) { + enable_cpu_hard_limit = tworkload_group_info.enable_cpu_hard_limit; } - task_group_info->enable_cpu_hard_limit = enable_cpu_hard_limit; + workload_group_info->enable_cpu_hard_limit = enable_cpu_hard_limit; // 9 scan thread num - task_group_info->scan_thread_num = config::doris_scanner_thread_pool_thread_num; - if (workload_group_info.__isset.scan_thread_num && workload_group_info.scan_thread_num > 0) { - task_group_info->scan_thread_num = workload_group_info.scan_thread_num; + workload_group_info->scan_thread_num = config::doris_scanner_thread_pool_thread_num; + if (tworkload_group_info.__isset.scan_thread_num && tworkload_group_info.scan_thread_num > 0) { + workload_group_info->scan_thread_num = tworkload_group_info.scan_thread_num; } // 10 max remote scan thread num - task_group_info->max_remote_scan_thread_num = config::doris_scanner_thread_pool_thread_num; - if (workload_group_info.__isset.max_remote_scan_thread_num && - workload_group_info.max_remote_scan_thread_num > 0) { - task_group_info->max_remote_scan_thread_num = - workload_group_info.max_remote_scan_thread_num; + workload_group_info->max_remote_scan_thread_num = config::doris_scanner_thread_pool_thread_num; + if (tworkload_group_info.__isset.max_remote_scan_thread_num && + tworkload_group_info.max_remote_scan_thread_num > 0) { + workload_group_info->max_remote_scan_thread_num = + tworkload_group_info.max_remote_scan_thread_num; } // 11 min remote scan thread num - task_group_info->min_remote_scan_thread_num = config::doris_scanner_thread_pool_thread_num; - if (workload_group_info.__isset.min_remote_scan_thread_num && - workload_group_info.min_remote_scan_thread_num > 0) { - task_group_info->min_remote_scan_thread_num = - workload_group_info.min_remote_scan_thread_num; + workload_group_info->min_remote_scan_thread_num = config::doris_scanner_thread_pool_thread_num; + if (tworkload_group_info.__isset.min_remote_scan_thread_num && + tworkload_group_info.min_remote_scan_thread_num > 0) { + workload_group_info->min_remote_scan_thread_num = + tworkload_group_info.min_remote_scan_thread_num; } return Status::OK(); } -void TaskGroup::upsert_task_scheduler(taskgroup::TaskGroupInfo* tg_info, ExecEnv* exec_env) { +void WorkloadGroup::upsert_task_scheduler(WorkloadGroupInfo* tg_info, ExecEnv* exec_env) { uint64_t tg_id = tg_info->id; std::string tg_name = tg_info->name; int cpu_hard_limit = tg_info->cpu_hard_limit; @@ -407,10 +406,10 @@ void TaskGroup::upsert_task_scheduler(taskgroup::TaskGroupInfo* tg_info, ExecEnv } } -void TaskGroup::get_query_scheduler(doris::pipeline::TaskScheduler** exec_sched, - vectorized::SimplifiedScanScheduler** scan_sched, - ThreadPool** non_pipe_thread_pool, - vectorized::SimplifiedScanScheduler** remote_scan_sched) { +void WorkloadGroup::get_query_scheduler(doris::pipeline::TaskScheduler** exec_sched, + vectorized::SimplifiedScanScheduler** scan_sched, + ThreadPool** non_pipe_thread_pool, + vectorized::SimplifiedScanScheduler** remote_scan_sched) { std::shared_lock rlock(_task_sched_lock); *exec_sched = _task_sched.get(); *scan_sched = _scan_task_sched.get(); @@ -418,7 +417,7 @@ void TaskGroup::get_query_scheduler(doris::pipeline::TaskScheduler** exec_sched, *non_pipe_thread_pool = _non_pipe_thread_pool.get(); } -void TaskGroup::try_stop_schedulers() { +void WorkloadGroup::try_stop_schedulers() { std::shared_lock rlock(_task_sched_lock); if (_task_sched) { _task_sched->stop(); @@ -435,5 +434,4 @@ void TaskGroup::try_stop_schedulers() { } } -} // namespace taskgroup } // namespace doris diff --git a/be/src/runtime/task_group/task_group.h b/be/src/runtime/workload_group/workload_group.h similarity index 87% rename from be/src/runtime/task_group/task_group.h rename to be/src/runtime/workload_group/workload_group.h index 938f73ac20a31a..af77be493b676f 100644 --- a/be/src/runtime/task_group/task_group.h +++ b/be/src/runtime/workload_group/workload_group.h @@ -49,18 +49,16 @@ class PipelineTask; class TaskScheduler; } // namespace pipeline -namespace taskgroup { - -class TaskGroup; -struct TaskGroupInfo; -struct TgTrackerLimiterGroup { +class WorkloadGroup; +struct WorkloadGroupInfo; +struct WgTrackerLimiterGroup { std::unordered_set> trackers; std::mutex group_lock; }; -class TaskGroup : public std::enable_shared_from_this { +class WorkloadGroup : public std::enable_shared_from_this { public: - explicit TaskGroup(const TaskGroupInfo& tg_info); + explicit WorkloadGroup(const WorkloadGroupInfo& tg_info); int64_t version() const { return _version; } @@ -86,7 +84,7 @@ class TaskGroup : public std::enable_shared_from_this { std::string debug_string() const; - void check_and_update(const TaskGroupInfo& tg_info); + void check_and_update(const WorkloadGroupInfo& tg_info); void add_mem_tracker_limiter(std::shared_ptr mem_tracker_ptr); @@ -102,7 +100,7 @@ class TaskGroup : public std::enable_shared_from_this { Status add_query(TUniqueId query_id) { std::unique_lock wlock(_mutex); if (_is_shutdown) { - // If the task group is set shutdown, then should not run any more, + // If the workload group is set shutdown, then should not run any more, // because the scheduler pool and other pointer may be released. return Status::InternalError( "Failed add query to workload group, the workload group is shutdown. host: {}", @@ -129,7 +127,7 @@ class TaskGroup : public std::enable_shared_from_this { int64_t gc_memory(int64_t need_free_mem, RuntimeProfile* profile); - void upsert_task_scheduler(taskgroup::TaskGroupInfo* tg_info, ExecEnv* exec_env); + void upsert_task_scheduler(WorkloadGroupInfo* tg_info, ExecEnv* exec_env); void get_query_scheduler(doris::pipeline::TaskScheduler** exec_sched, vectorized::SimplifiedScanScheduler** scan_sched, @@ -146,13 +144,13 @@ class TaskGroup : public std::enable_shared_from_this { int64_t _memory_limit; // bytes bool _enable_memory_overcommit; std::atomic _cpu_share; - std::vector _mem_tracker_limiter_pool; + std::vector _mem_tracker_limiter_pool; std::atomic _cpu_hard_limit; std::atomic _scan_thread_num; std::atomic _max_remote_scan_thread_num; std::atomic _min_remote_scan_thread_num; - // means task group is mark dropped + // means workload group is mark dropped // new query can not submit // waiting running query to be cancelled or finish bool _is_shutdown = false; @@ -166,9 +164,9 @@ class TaskGroup : public std::enable_shared_from_this { std::unique_ptr _non_pipe_thread_pool = nullptr; }; -using TaskGroupPtr = std::shared_ptr; +using WorkloadGroupPtr = std::shared_ptr; -struct TaskGroupInfo { +struct WorkloadGroupInfo { uint64_t id; std::string name; uint64_t cpu_share; @@ -184,9 +182,8 @@ struct TaskGroupInfo { uint64_t cgroup_cpu_shares = 0; int cgroup_cpu_hard_limit = 0; - static Status parse_topic_info(const TWorkloadGroupInfo& topic_info, - taskgroup::TaskGroupInfo* task_group_info); + static Status parse_topic_info(const TWorkloadGroupInfo& tworkload_group_info, + WorkloadGroupInfo* workload_group_info); }; -} // namespace taskgroup } // namespace doris diff --git a/be/src/runtime/task_group/task_group_manager.cpp b/be/src/runtime/workload_group/workload_group_manager.cpp similarity index 59% rename from be/src/runtime/task_group/task_group_manager.cpp rename to be/src/runtime/workload_group/workload_group_manager.cpp index a336cccd3d21a6..7ec08387543657 100644 --- a/be/src/runtime/task_group/task_group_manager.cpp +++ b/be/src/runtime/workload_group/workload_group_manager.cpp @@ -15,75 +15,76 @@ // specific language governing permissions and limitations // under the License. -#include "task_group_manager.h" +#include "workload_group_manager.h" #include #include #include "pipeline/task_scheduler.h" #include "runtime/memory/mem_tracker_limiter.h" -#include "runtime/task_group/task_group.h" +#include "runtime/workload_group/workload_group.h" #include "util/threadpool.h" #include "util/time.h" #include "vec/exec/scan/scanner_scheduler.h" -namespace doris::taskgroup { +namespace doris { -TaskGroupPtr TaskGroupManager::get_or_create_task_group(const TaskGroupInfo& task_group_info) { +WorkloadGroupPtr WorkloadGroupMgr::get_or_create_workload_group( + const WorkloadGroupInfo& workload_group_info) { { std::shared_lock r_lock(_group_mutex); - if (LIKELY(_task_groups.count(task_group_info.id))) { - auto task_group = _task_groups[task_group_info.id]; - task_group->check_and_update(task_group_info); - return task_group; + if (LIKELY(_workload_groups.count(workload_group_info.id))) { + auto workload_group = _workload_groups[workload_group_info.id]; + workload_group->check_and_update(workload_group_info); + return workload_group; } } - auto new_task_group = std::make_shared(task_group_info); + auto new_task_group = std::make_shared(workload_group_info); std::lock_guard w_lock(_group_mutex); - if (_task_groups.count(task_group_info.id)) { - auto task_group = _task_groups[task_group_info.id]; - task_group->check_and_update(task_group_info); - return task_group; + if (_workload_groups.count(workload_group_info.id)) { + auto workload_group = _workload_groups[workload_group_info.id]; + workload_group->check_and_update(workload_group_info); + return workload_group; } - _task_groups[task_group_info.id] = new_task_group; + _workload_groups[workload_group_info.id] = new_task_group; return new_task_group; } -void TaskGroupManager::get_related_taskgroups( - const std::function& pred, - std::vector* task_groups) { +void WorkloadGroupMgr::get_related_workload_groups( + const std::function& pred, + std::vector* task_groups) { std::shared_lock r_lock(_group_mutex); - for (const auto& [id, task_group] : _task_groups) { - if (pred(task_group)) { - task_groups->push_back(task_group); + for (const auto& [id, workload_group] : _workload_groups) { + if (pred(workload_group)) { + task_groups->push_back(workload_group); } } } -TaskGroupPtr TaskGroupManager::get_task_group_by_id(uint64_t tg_id) { +WorkloadGroupPtr WorkloadGroupMgr::get_task_group_by_id(uint64_t tg_id) { std::shared_lock r_lock(_group_mutex); - if (_task_groups.find(tg_id) != _task_groups.end()) { - return _task_groups.at(tg_id); + if (_workload_groups.find(tg_id) != _workload_groups.end()) { + return _workload_groups.at(tg_id); } return nullptr; } -void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { +void WorkloadGroupMgr::delete_workload_group_by_ids(std::set used_wg_id) { int64_t begin_time = MonotonicMillis(); // 1 get delete group without running queries - std::vector deleted_task_groups; + std::vector deleted_task_groups; { std::lock_guard write_lock(_group_mutex); - for (auto iter = _task_groups.begin(); iter != _task_groups.end(); iter++) { + for (auto iter = _workload_groups.begin(); iter != _workload_groups.end(); iter++) { uint64_t tg_id = iter->first; - auto task_group_ptr = iter->second; + auto workload_group_ptr = iter->second; if (used_wg_id.find(tg_id) == used_wg_id.end()) { - task_group_ptr->shutdown(); - // only when no query running in task group, its resource can be released in BE - if (task_group_ptr->query_num() == 0) { + workload_group_ptr->shutdown(); + // only when no query running in workload group, its resource can be released in BE + if (workload_group_ptr->query_num() == 0) { LOG(INFO) << "There is no query in wg " << tg_id << ", delete it."; - deleted_task_groups.push_back(task_group_ptr); + deleted_task_groups.push_back(workload_group_ptr); } } } @@ -100,7 +101,7 @@ void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { { std::lock_guard write_lock(_group_mutex); for (auto& tg : deleted_task_groups) { - _task_groups.erase(tg->id()); + _workload_groups.erase(tg->id()); } } @@ -119,7 +120,7 @@ void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { if (ret.ok()) { _is_init_succ = true; } else { - LOG(INFO) << "init task group mgr cpu ctl failed, " << ret.to_string(); + LOG(INFO) << "init workload group mgr cpu ctl failed, " << ret.to_string(); } } if (_is_init_succ) { @@ -130,14 +131,14 @@ void TaskGroupManager::delete_task_group_by_ids(std::set used_wg_id) { } } int64_t time_cost_ms = MonotonicMillis() - begin_time; - LOG(INFO) << "finish clear unused task group, time cost: " << time_cost_ms + LOG(INFO) << "finish clear unused workload group, time cost: " << time_cost_ms << "ms, deleted group size:" << deleted_task_groups.size(); } -void TaskGroupManager::stop() { - for (auto iter = _task_groups.begin(); iter != _task_groups.end(); iter++) { +void WorkloadGroupMgr::stop() { + for (auto iter = _workload_groups.begin(); iter != _workload_groups.end(); iter++) { iter->second->try_stop_schedulers(); } } -} // namespace doris::taskgroup +} // namespace doris diff --git a/be/src/runtime/task_group/task_group_manager.h b/be/src/runtime/workload_group/workload_group_manager.h similarity index 70% rename from be/src/runtime/task_group/task_group_manager.h rename to be/src/runtime/workload_group/workload_group_manager.h index 21772bd3bca3f1..1f680eb17c848b 100644 --- a/be/src/runtime/task_group/task_group_manager.h +++ b/be/src/runtime/workload_group/workload_group_manager.h @@ -21,7 +21,7 @@ #include #include -#include "task_group.h" +#include "workload_group.h" namespace doris { @@ -32,21 +32,19 @@ class TaskScheduler; class MultiCoreTaskQueue; } // namespace pipeline -namespace taskgroup { - -class TaskGroupManager { +class WorkloadGroupMgr { public: - TaskGroupManager() = default; - ~TaskGroupManager() = default; + WorkloadGroupMgr() = default; + ~WorkloadGroupMgr() = default; - TaskGroupPtr get_or_create_task_group(const TaskGroupInfo& task_group_info); + WorkloadGroupPtr get_or_create_workload_group(const WorkloadGroupInfo& workload_group_info); - void get_related_taskgroups(const std::function& pred, - std::vector* task_groups); + void get_related_workload_groups(const std::function& pred, + std::vector* task_groups); - void delete_task_group_by_ids(std::set id_set); + void delete_workload_group_by_ids(std::set id_set); - TaskGroupPtr get_task_group_by_id(uint64_t tg_id); + WorkloadGroupPtr get_task_group_by_id(uint64_t tg_id); void stop(); @@ -58,12 +56,11 @@ class TaskGroupManager { private: std::shared_mutex _group_mutex; - std::unordered_map _task_groups; + std::unordered_map _workload_groups; std::shared_mutex _init_cg_ctl_lock; std::unique_ptr _cg_cpu_ctl; bool _is_init_succ = false; }; -} // namespace taskgroup } // namespace doris diff --git a/be/src/util/mem_info.cpp b/be/src/util/mem_info.cpp index 3d9c4c4b062a7a..a5ca3b70f1b387 100644 --- a/be/src/util/mem_info.cpp +++ b/be/src/util/mem_info.cpp @@ -42,8 +42,8 @@ #include "runtime/exec_env.h" #include "runtime/memory/cache_manager.h" #include "runtime/memory/mem_tracker_limiter.h" -#include "runtime/task_group/task_group.h" -#include "runtime/task_group/task_group_manager.h" +#include "runtime/workload_group/workload_group.h" +#include "runtime/workload_group/workload_group_manager.h" #include "util/cgroup_util.h" #include "util/defer_op.h" #include "util/parse_util.h" @@ -241,23 +241,24 @@ bool MemInfo::process_full_gc() { int64_t MemInfo::tg_not_enable_overcommit_group_gc() { MonotonicStopWatch watch; watch.start(); - std::vector task_groups; + std::vector task_groups; std::unique_ptr tg_profile = std::make_unique("WorkloadGroup"); int64_t total_free_memory = 0; - ExecEnv::GetInstance()->task_group_manager()->get_related_taskgroups( - [](const taskgroup::TaskGroupPtr& task_group) { - return task_group->is_mem_limit_valid() && !task_group->enable_memory_overcommit(); + ExecEnv::GetInstance()->workload_group_mgr()->get_related_workload_groups( + [](const WorkloadGroupPtr& workload_group) { + return workload_group->is_mem_limit_valid() && + !workload_group->enable_memory_overcommit(); }, &task_groups); if (task_groups.empty()) { return 0; } - std::vector task_groups_overcommit; - for (const auto& task_group : task_groups) { - if (task_group->memory_used() > task_group->memory_limit()) { - task_groups_overcommit.push_back(task_group); + std::vector task_groups_overcommit; + for (const auto& workload_group : task_groups) { + if (workload_group->memory_used() > workload_group->memory_limit()) { + task_groups_overcommit.push_back(workload_group); } } if (task_groups_overcommit.empty()) { @@ -283,10 +284,10 @@ int64_t MemInfo::tg_not_enable_overcommit_group_gc() { } }}; - for (const auto& task_group : task_groups_overcommit) { - auto used = task_group->memory_used(); + for (const auto& workload_group : task_groups_overcommit) { + auto used = workload_group->memory_used(); total_free_memory += - task_group->gc_memory(used - task_group->memory_limit(), tg_profile.get()); + workload_group->gc_memory(used - workload_group->memory_limit(), tg_profile.get()); } return total_free_memory; } @@ -295,10 +296,11 @@ int64_t MemInfo::tg_enable_overcommit_group_gc(int64_t request_free_memory, RuntimeProfile* profile) { MonotonicStopWatch watch; watch.start(); - std::vector task_groups; - ExecEnv::GetInstance()->task_group_manager()->get_related_taskgroups( - [](const taskgroup::TaskGroupPtr& task_group) { - return task_group->is_mem_limit_valid() && task_group->enable_memory_overcommit(); + std::vector task_groups; + ExecEnv::GetInstance()->workload_group_mgr()->get_related_workload_groups( + [](const WorkloadGroupPtr& workload_group) { + return workload_group->is_mem_limit_valid() && + workload_group->enable_memory_overcommit(); }, &task_groups); if (task_groups.empty()) { @@ -308,9 +310,9 @@ int64_t MemInfo::tg_enable_overcommit_group_gc(int64_t request_free_memory, int64_t total_exceeded_memory = 0; std::vector used_memorys; std::vector exceeded_memorys; - for (const auto& task_group : task_groups) { - int64_t used_memory = task_group->memory_used(); - int64_t exceeded = used_memory - task_group->memory_limit(); + for (const auto& workload_group : task_groups) { + int64_t used_memory = workload_group->memory_used(); + int64_t exceeded = used_memory - workload_group->memory_limit(); int64_t exceeded_memory = exceeded > 0 ? exceeded : 0; total_exceeded_memory += exceeded_memory; used_memorys.emplace_back(used_memory); @@ -356,8 +358,8 @@ int64_t MemInfo::tg_enable_overcommit_group_gc(int64_t request_free_memory, gc_all_exceeded ? exceeded_memorys[i] : static_cast(exceeded_memorys[i]) / total_exceeded_memory * request_free_memory /* exceeded memory as a weight */; - auto task_group = task_groups[i]; - total_free_memory += task_group->gc_memory(tg_need_free_memory, profile); + auto workload_group = task_groups[i]; + total_free_memory += workload_group->gc_memory(tg_need_free_memory, profile); } return total_free_memory; } diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h index 1052e77ae0ae11..24889054094ca1 100644 --- a/be/src/vec/exec/scan/scanner_context.h +++ b/be/src/vec/exec/scan/scanner_context.h @@ -39,16 +39,13 @@ namespace doris { class ThreadPoolToken; class RuntimeState; class TupleDescriptor; +class WorkloadGroup; namespace pipeline { class ScanLocalStateBase; class Dependency; } // namespace pipeline -namespace taskgroup { -class TaskGroup; -} // namespace taskgroup - namespace vectorized { class VScanner; From ace77925aa9cd782e84ebfc939114600037f4637 Mon Sep 17 00:00:00 2001 From: wangbo Date: Thu, 14 Mar 2024 14:57:13 +0800 Subject: [PATCH 02/10] [Refactor](executor)Add information_schema.workload_groups (#32195) --- be/src/exec/schema_scanner.cpp | 3 + .../schema_active_queries_scanner.cpp | 17 +- .../schema_workload_groups_scanner.cpp | 166 ++++++++++++++++++ .../schema_workload_groups_scanner.h | 52 ++++++ be/src/vec/exec/scan/vmeta_scanner.cpp | 42 ----- .../doris/analysis/SchemaTableType.java | 3 +- .../catalog/BuiltinTableValuedFunctions.java | 4 +- .../org/apache/doris/catalog/SchemaTable.java | 14 ++ .../functions/table/WorkloadGroups.java | 56 ------ .../visitor/TableValuedFunctionVisitor.java | 5 - .../doris/service/FrontendServiceImpl.java | 18 +- .../tablefunction/MetadataGenerator.java | 88 ++++++---- .../MetadataTableValuedFunction.java | 2 - .../tablefunction/TableValuedFunctionIf.java | 2 - .../WorkloadGroupsTableValuedFunction.java | 99 ----------- .../doris/datasource/RefreshCatalogTest.java | 4 +- gensrc/thrift/Descriptors.thrift | 3 +- gensrc/thrift/FrontendService.thrift | 12 +- gensrc/thrift/Types.thrift | 2 - .../jdbc/test_mariadb_jdbc_catalog.out | 1 + .../jdbc/test_mysql_jdbc_catalog.out | 1 + .../jdbc/test_mysql_jdbc_catalog_nereids.out | 1 + .../jdbc/test_mysql_jdbc_driver5_catalog.out | 1 + .../workload_manager_p0/test_curd_wlg.groovy | 12 +- 24 files changed, 330 insertions(+), 278 deletions(-) create mode 100644 be/src/exec/schema_scanner/schema_workload_groups_scanner.cpp create mode 100644 be/src/exec/schema_scanner/schema_workload_groups_scanner.h delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/WorkloadGroups.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/tablefunction/WorkloadGroupsTableValuedFunction.java diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp index 4f4273412c1a7d..375252501edab4 100644 --- a/be/src/exec/schema_scanner.cpp +++ b/be/src/exec/schema_scanner.cpp @@ -44,6 +44,7 @@ #include "exec/schema_scanner/schema_user_privileges_scanner.h" #include "exec/schema_scanner/schema_variables_scanner.h" #include "exec/schema_scanner/schema_views_scanner.h" +#include "exec/schema_scanner/schema_workload_groups_scanner.h" #include "olap/hll.h" #include "runtime/define_primitive_type.h" #include "util/string_util.h" @@ -155,6 +156,8 @@ std::unique_ptr SchemaScanner::create(TSchemaTableType::type type return SchemaBackendActiveTasksScanner::create_unique(); case TSchemaTableType::SCH_ACTIVE_QUERIES: return SchemaActiveQueriesScanner::create_unique(); + case TSchemaTableType::SCH_WORKLOAD_GROUPS: + return SchemaWorkloadGroupsScanner::create_unique(); default: return SchemaDummyScanner::create_unique(); break; diff --git a/be/src/exec/schema_scanner/schema_active_queries_scanner.cpp b/be/src/exec/schema_scanner/schema_active_queries_scanner.cpp index 02dcd4d4a3c3af..f16326dc8f5f42 100644 --- a/be/src/exec/schema_scanner/schema_active_queries_scanner.cpp +++ b/be/src/exec/schema_scanner/schema_active_queries_scanner.cpp @@ -50,20 +50,17 @@ Status SchemaActiveQueriesScanner::start(RuntimeState* state) { Status SchemaActiveQueriesScanner::_get_active_queries_block_from_fe() { TNetworkAddress master_addr = ExecEnv::GetInstance()->master_info()->network_address; - TQueriesMetadataParams tqueries_meta_params; - tqueries_meta_params.__set_relay_to_other_fe(true); - - TMetadataTableRequestParams metadata_table_params; - metadata_table_params.__set_metadata_type(TMetadataType::QUERIES); - metadata_table_params.__set_queries_metadata_params(tqueries_meta_params); + TSchemaTableRequestParams schema_table_params; for (int i = 0; i < _s_tbls_columns.size(); i++) { - metadata_table_params.__isset.columns_name = true; - metadata_table_params.columns_name.emplace_back(_s_tbls_columns[i].name); + schema_table_params.__isset.columns_name = true; + schema_table_params.columns_name.emplace_back(_s_tbls_columns[i].name); } + schema_table_params.replay_to_other_fe = true; + schema_table_params.__isset.replay_to_other_fe = true; TFetchSchemaTableDataRequest request; - request.__set_schema_table_name(TSchemaTableName::SCHEMA_TABLE); - request.__set_metada_table_params(metadata_table_params); + request.__set_schema_table_name(TSchemaTableName::ACTIVE_QUERIES); + request.__set_schema_table_params(schema_table_params); TFetchSchemaTableDataResult result; diff --git a/be/src/exec/schema_scanner/schema_workload_groups_scanner.cpp b/be/src/exec/schema_scanner/schema_workload_groups_scanner.cpp new file mode 100644 index 00000000000000..b3fb9adcbebe21 --- /dev/null +++ b/be/src/exec/schema_scanner/schema_workload_groups_scanner.cpp @@ -0,0 +1,166 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "exec/schema_scanner/schema_workload_groups_scanner.h" + +#include "runtime/client_cache.h" +#include "runtime/exec_env.h" +#include "runtime/runtime_state.h" +#include "util/thrift_rpc_helper.h" +#include "vec/common/string_ref.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type_factory.hpp" + +namespace doris { +std::vector SchemaWorkloadGroupsScanner::_s_tbls_columns = { + {"ID", TYPE_BIGINT, sizeof(int64_t), true}, + {"NAME", TYPE_VARCHAR, sizeof(StringRef), true}, + {"CPU_SHARE", TYPE_BIGINT, sizeof(int64_t), true}, + {"MEMORY_LIMIT", TYPE_VARCHAR, sizeof(StringRef), true}, + {"ENABLE_MEMORY_OVERCOMMIT", TYPE_VARCHAR, sizeof(StringRef), true}, + {"MAX_CONCURRENCY", TYPE_BIGINT, sizeof(int64_t), true}, + {"MAX_QUEUE_SIZE", TYPE_BIGINT, sizeof(int64_t), true}, + {"QUEUE_TIMEOUT", TYPE_BIGINT, sizeof(int64_t), true}, + {"CPU_HARD_LIMIT", TYPE_STRING, sizeof(StringRef), true}, + {"SCAN_THREAD_NUM", TYPE_BIGINT, sizeof(int64_t), true}, + {"MAX_REMOTE_SCAN_THREAD_NUM", TYPE_BIGINT, sizeof(int64_t), true}, + {"MIN_REMOTE_SCAN_THREAD_NUM", TYPE_BIGINT, sizeof(int64_t), true}}; + +SchemaWorkloadGroupsScanner::SchemaWorkloadGroupsScanner() + : SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_WORKLOAD_GROUPS) {} + +SchemaWorkloadGroupsScanner::~SchemaWorkloadGroupsScanner() {} + +Status SchemaWorkloadGroupsScanner::start(RuntimeState* state) { + _block_rows_limit = state->batch_size(); + _rpc_timeout = state->execution_timeout() * 1000; + return Status::OK(); +} + +Status SchemaWorkloadGroupsScanner::_get_workload_groups_block_from_fe() { + TNetworkAddress master_addr = ExecEnv::GetInstance()->master_info()->network_address; + + TSchemaTableRequestParams schema_table_request_params; + for (int i = 0; i < _s_tbls_columns.size(); i++) { + schema_table_request_params.__isset.columns_name = true; + schema_table_request_params.columns_name.emplace_back(_s_tbls_columns[i].name); + } + schema_table_request_params.__set_current_user_ident(*_param->common_param->current_user_ident); + + TFetchSchemaTableDataRequest request; + request.__set_schema_table_name(TSchemaTableName::WORKLOAD_GROUPS); + request.__set_schema_table_params(schema_table_request_params); + + TFetchSchemaTableDataResult result; + + RETURN_IF_ERROR(ThriftRpcHelper::rpc( + master_addr.hostname, master_addr.port, + [&request, &result](FrontendServiceConnection& client) { + client->fetchSchemaTableData(result, request); + }, + _rpc_timeout)); + + Status status(Status::create(result.status)); + if (!status.ok()) { + LOG(WARNING) << "fetch workload groups from FE failed, errmsg=" << status; + return status; + } + std::vector result_data = result.data_batch; + + _workload_groups_block = vectorized::Block::create_unique(); + for (int i = 0; i < _s_tbls_columns.size(); ++i) { + TypeDescriptor descriptor(_s_tbls_columns[i].type); + auto data_type = vectorized::DataTypeFactory::instance().create_data_type(descriptor, true); + _workload_groups_block->insert(vectorized::ColumnWithTypeAndName( + data_type->create_column(), data_type, _s_tbls_columns[i].name)); + } + + _workload_groups_block->reserve(_block_rows_limit); + + if (result_data.size() > 0) { + int col_size = result_data[0].column_value.size(); + if (col_size != _s_tbls_columns.size()) { + return Status::InternalError( + "workload groups schema is not match for FE and BE"); + } + } + + // todo(wb) reuse this callback function + auto insert_string_value = [&](int col_index, std::string str_val, vectorized::Block* block) { + vectorized::MutableColumnPtr mutable_col_ptr; + mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable(); + auto* nullable_column = + reinterpret_cast(mutable_col_ptr.get()); + vectorized::IColumn* col_ptr = &nullable_column->get_nested_column(); + reinterpret_cast(col_ptr)->insert_data(str_val.data(), + str_val.size()); + nullable_column->get_null_map_data().emplace_back(0); + }; + auto insert_int_value = [&](int col_index, int64_t int_val, vectorized::Block* block) { + vectorized::MutableColumnPtr mutable_col_ptr; + mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable(); + auto* nullable_column = + reinterpret_cast(mutable_col_ptr.get()); + vectorized::IColumn* col_ptr = &nullable_column->get_nested_column(); + reinterpret_cast*>(col_ptr)->insert_value( + int_val); + nullable_column->get_null_map_data().emplace_back(0); + }; + + for (int i = 0; i < result_data.size(); i++) { + TRow row = result_data[i]; + + for (int j = 0; j < _s_tbls_columns.size(); j++) { + if (_s_tbls_columns[j].type == TYPE_BIGINT) { + insert_int_value(j, row.column_value[j].longVal, _workload_groups_block.get()); + } else { + insert_string_value(j, row.column_value[j].stringVal, _workload_groups_block.get()); + } + } + } + return Status::OK(); +} + +Status SchemaWorkloadGroupsScanner::get_next_block(vectorized::Block* block, bool* eos) { + if (!_is_init) { + return Status::InternalError("Used before initialized."); + } + + if (nullptr == block || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); + } + + if (_workload_groups_block == nullptr) { + RETURN_IF_ERROR(_get_workload_groups_block_from_fe()); + _total_rows = _workload_groups_block->rows(); + } + + if (_row_idx == _total_rows) { + *eos = true; + return Status::OK(); + } + + int current_batch_rows = std::min(_block_rows_limit, _total_rows - _row_idx); + vectorized::MutableBlock mblock = vectorized::MutableBlock::build_mutable_block(block); + mblock.add_rows(_workload_groups_block.get(), _row_idx, current_batch_rows); + _row_idx += current_batch_rows; + + *eos = _row_idx == _total_rows; + return Status::OK(); +} + +} // namespace doris \ No newline at end of file diff --git a/be/src/exec/schema_scanner/schema_workload_groups_scanner.h b/be/src/exec/schema_scanner/schema_workload_groups_scanner.h new file mode 100644 index 00000000000000..bf7a103526dc80 --- /dev/null +++ b/be/src/exec/schema_scanner/schema_workload_groups_scanner.h @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include "common/status.h" +#include "exec/schema_scanner.h" + +namespace doris { +class RuntimeState; +namespace vectorized { +class Block; +} // namespace vectorized + +class SchemaWorkloadGroupsScanner : public SchemaScanner { + ENABLE_FACTORY_CREATOR(SchemaWorkloadGroupsScanner); + +public: + SchemaWorkloadGroupsScanner(); + ~SchemaWorkloadGroupsScanner() override; + + Status start(RuntimeState* state) override; + Status get_next_block(vectorized::Block* block, bool* eos) override; + + static std::vector _s_tbls_columns; + +private: + Status _get_workload_groups_block_from_fe(); + + int _block_rows_limit = 4096; + int _row_idx = 0; + int _total_rows = 0; + std::unique_ptr _workload_groups_block = nullptr; + int _rpc_timeout = 3000; +}; +}; // namespace doris \ No newline at end of file diff --git a/be/src/vec/exec/scan/vmeta_scanner.cpp b/be/src/vec/exec/scan/vmeta_scanner.cpp index 22545fa4dcefc5..e31846c21be9b8 100644 --- a/be/src/vec/exec/scan/vmeta_scanner.cpp +++ b/be/src/vec/exec/scan/vmeta_scanner.cpp @@ -235,9 +235,6 @@ Status VMetaScanner::_fetch_metadata(const TMetaScanRange& meta_scan_range) { case TMetadataType::FRONTENDS_DISKS: RETURN_IF_ERROR(_build_frontends_disks_metadata_request(meta_scan_range, &request)); break; - case TMetadataType::WORKLOAD_GROUPS: - RETURN_IF_ERROR(_build_workload_groups_metadata_request(meta_scan_range, &request)); - break; case TMetadataType::WORKLOAD_SCHED_POLICY: RETURN_IF_ERROR(_build_workload_sched_policy_metadata_request(meta_scan_range, &request)); break; @@ -253,9 +250,6 @@ Status VMetaScanner::_fetch_metadata(const TMetaScanRange& meta_scan_range) { case TMetadataType::TASKS: RETURN_IF_ERROR(_build_tasks_metadata_request(meta_scan_range, &request)); break; - case TMetadataType::QUERIES: - RETURN_IF_ERROR(_build_queries_metadata_request(meta_scan_range, &request)); - break; default: _meta_eos = true; return Status::OK(); @@ -365,23 +359,6 @@ Status VMetaScanner::_build_frontends_disks_metadata_request( return Status::OK(); } -Status VMetaScanner::_build_workload_groups_metadata_request( - const TMetaScanRange& meta_scan_range, TFetchSchemaTableDataRequest* request) { - VLOG_CRITICAL << "VMetaScanner::_build_workload_groups_metadata_request"; - - // create request - request->__set_cluster_name(""); - request->__set_schema_table_name(TSchemaTableName::METADATA_TABLE); - - // create TMetadataTableRequestParams - TMetadataTableRequestParams metadata_table_params; - metadata_table_params.__set_metadata_type(TMetadataType::WORKLOAD_GROUPS); - metadata_table_params.__set_current_user_ident(_user_identity); - - request->__set_metada_table_params(metadata_table_params); - return Status::OK(); -} - Status VMetaScanner::_build_workload_sched_policy_metadata_request( const TMetaScanRange& meta_scan_range, TFetchSchemaTableDataRequest* request) { VLOG_CRITICAL << "VMetaScanner::_build_workload_sched_policy_metadata_request"; @@ -473,25 +450,6 @@ Status VMetaScanner::_build_tasks_metadata_request(const TMetaScanRange& meta_sc return Status::OK(); } -Status VMetaScanner::_build_queries_metadata_request(const TMetaScanRange& meta_scan_range, - TFetchSchemaTableDataRequest* request) { - VLOG_CRITICAL << "VMetaScanner::_build_queries_metadata_request"; - if (!meta_scan_range.__isset.queries_params) { - return Status::InternalError("Can not find TQueriesMetadataParams from meta_scan_range."); - } - // create request - request->__set_cluster_name(""); - request->__set_schema_table_name(TSchemaTableName::METADATA_TABLE); - - // create TMetadataTableRequestParams - TMetadataTableRequestParams metadata_table_params; - metadata_table_params.__set_metadata_type(TMetadataType::QUERIES); - metadata_table_params.__set_queries_metadata_params(meta_scan_range.queries_params); - - request->__set_metada_table_params(metadata_table_params); - return Status::OK(); -} - Status VMetaScanner::close(RuntimeState* state) { VLOG_CRITICAL << "VMetaScanner::close"; RETURN_IF_ERROR(VScanner::close(state)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java index f3965a2f86512a..bb1aa5e795295c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java @@ -71,7 +71,8 @@ public enum SchemaTableType { SCH_METADATA_NAME_IDS("METADATA_NAME_IDS", "METADATA_NAME_IDS", TSchemaTableType.SCH_METADATA_NAME_IDS), SCH_PROFILING("PROFILING", "PROFILING", TSchemaTableType.SCH_PROFILING), SCH_BACKEND_ACTIVE_TASKS("BACKEND_ACTIVE_TASKS", "BACKEND_ACTIVE_TASKS", TSchemaTableType.SCH_BACKEND_ACTIVE_TASKS), - SCH_ACTIVE_QUERIES("ACTIVE_QUERIES", "ACTIVE_QUERIES", TSchemaTableType.SCH_ACTIVE_QUERIES); + SCH_ACTIVE_QUERIES("ACTIVE_QUERIES", "ACTIVE_QUERIES", TSchemaTableType.SCH_ACTIVE_QUERIES), + SCH_WORKLOAD_GROUPS("WORKLOAD_GROUPS", "WORKLOAD_GROUPS", TSchemaTableType.SCH_WORKLOAD_GROUPS); private static final String dbName = "INFORMATION_SCHEMA"; private static SelectList fullSelectLists; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java index 049e1b301e517b..9986ce71885974 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java @@ -31,7 +31,6 @@ import org.apache.doris.nereids.trees.expressions.functions.table.Numbers; import org.apache.doris.nereids.trees.expressions.functions.table.S3; import org.apache.doris.nereids.trees.expressions.functions.table.Tasks; -import org.apache.doris.nereids.trees.expressions.functions.table.WorkloadGroups; import com.google.common.collect.ImmutableList; @@ -56,8 +55,7 @@ public class BuiltinTableValuedFunctions implements FunctionHelper { tableValued(S3.class, "s3"), tableValued(MvInfos.class, "mv_infos"), tableValued(Jobs.class, "jobs"), - tableValued(Tasks.class, "tasks"), - tableValued(WorkloadGroups.class, "workload_groups") + tableValued(Tasks.class, "tasks") ); public static final BuiltinTableValuedFunctions INSTANCE = new BuiltinTableValuedFunctions(); 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 2ce7f5bb794986..736df74f1e6a2a 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 @@ -467,6 +467,20 @@ public class SchemaTable extends Table { .column("FRONTEND_INSTANCE", ScalarType.createVarchar(256)) .column("SQL", ScalarType.createStringType()) .build())) + .put("workload_groups", new SchemaTable(SystemIdGenerator.getNextId(), "workload_groups", TableType.SCHEMA, + builder().column("ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("NAME", ScalarType.createVarchar(256)) + .column("CPU_SHARE", ScalarType.createType(PrimitiveType.BIGINT)) + .column("MEMORY_LIMIT", ScalarType.createVarchar(256)) + .column("ENABLE_MEMORY_OVERCOMMIT", ScalarType.createVarchar(256)) + .column("MAX_CONCURRENCY", ScalarType.createType(PrimitiveType.BIGINT)) + .column("MAX_QUEUE_SIZE", ScalarType.createType(PrimitiveType.BIGINT)) + .column("QUEUE_TIMEOUT", ScalarType.createType(PrimitiveType.BIGINT)) + .column("CPU_HARD_LIMIT", ScalarType.createStringType()) + .column("SCAN_THREAD_NUM", ScalarType.createType(PrimitiveType.BIGINT)) + .column("MAX_REMOTE_SCAN_THREAD_NUM", ScalarType.createType(PrimitiveType.BIGINT)) + .column("MIN_REMOTE_SCAN_THREAD_NUM", ScalarType.createType(PrimitiveType.BIGINT)) + .build())) .build(); protected SchemaTable(long id, String name, TableType type, List baseSchema) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/WorkloadGroups.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/WorkloadGroups.java deleted file mode 100644 index 84aa4de697f036..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/WorkloadGroups.java +++ /dev/null @@ -1,56 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.expressions.functions.table; - -import org.apache.doris.catalog.FunctionSignature; -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.Properties; -import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; -import org.apache.doris.nereids.types.coercion.AnyDataType; -import org.apache.doris.tablefunction.TableValuedFunctionIf; -import org.apache.doris.tablefunction.WorkloadGroupsTableValuedFunction; - -import java.util.Map; - -/** workload_groups */ -public class WorkloadGroups extends TableValuedFunction { - public WorkloadGroups(Properties properties) { - super("workload_groups", properties); - } - - @Override - public FunctionSignature customSignature() { - return FunctionSignature.of(AnyDataType.INSTANCE_WITHOUT_INDEX, getArgumentsTypes()); - } - - @Override - protected TableValuedFunctionIf toCatalogFunction() { - try { - Map arguments = getTVFProperties().getMap(); - return new WorkloadGroupsTableValuedFunction(arguments); - } catch (Throwable t) { - throw new AnalysisException("Can not build WorkloadGroupsTableValuedFunction by " - + this + ": " + t.getMessage(), t); - } - } - - @Override - public R accept(ExpressionVisitor visitor, C context) { - return visitor.visitWorkloadGroups(this, context); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java index 36e8ac365f79a7..d0c76d143a2953 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java @@ -32,7 +32,6 @@ import org.apache.doris.nereids.trees.expressions.functions.table.S3; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; import org.apache.doris.nereids.trees.expressions.functions.table.Tasks; -import org.apache.doris.nereids.trees.expressions.functions.table.WorkloadGroups; /** TableValuedFunctionVisitor */ public interface TableValuedFunctionVisitor { @@ -93,8 +92,4 @@ default R visitNumbers(Numbers numbers, C context) { default R visitS3(S3 s3, C context) { return visitTableValuedFunction(s3, context); } - - default R visitWorkloadGroups(WorkloadGroups workloadGroups, C context) { - return visitTableValuedFunction(workloadGroups, context); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index ef70faa2322e9d..36d24ddfc4df6a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -208,6 +208,7 @@ import org.apache.doris.thrift.TRestoreSnapshotResult; import org.apache.doris.thrift.TRollbackTxnRequest; import org.apache.doris.thrift.TRollbackTxnResult; +import org.apache.doris.thrift.TSchemaTableName; import org.apache.doris.thrift.TShowProcessListRequest; import org.apache.doris.thrift.TShowProcessListResult; import org.apache.doris.thrift.TShowVariableRequest; @@ -2296,15 +2297,16 @@ public TFrontendPingFrontendResult ping(TFrontendPingFrontendRequest request) th @Override public TFetchSchemaTableDataResult fetchSchemaTableData(TFetchSchemaTableDataRequest request) throws TException { - switch (request.getSchemaTableName()) { - case METADATA_TABLE: - return MetadataGenerator.getMetadataTable(request); - case SCHEMA_TABLE: - return MetadataGenerator.getSchemaTableData(request); - default: - break; + if (!request.isSetSchemaTableName()) { + return MetadataGenerator.errorResult("Fetch schema table name is not set"); + } + // tvf queries + if (request.getSchemaTableName() == TSchemaTableName.METADATA_TABLE) { + return MetadataGenerator.getMetadataTable(request); + } else { + // database information_schema's tables + return MetadataGenerator.getSchemaTableData(request); } - return MetadataGenerator.errorResult("Fetch schema table name is not set"); } private TNetworkAddress getClientAddr() { 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 24cb4a365a99af..8f4dbf95616dcc 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 @@ -57,9 +57,9 @@ import org.apache.doris.thrift.TMetadataType; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TPipelineWorkloadGroup; -import org.apache.doris.thrift.TQueriesMetadataParams; import org.apache.doris.thrift.TQueryStatistics; import org.apache.doris.thrift.TRow; +import org.apache.doris.thrift.TSchemaTableRequestParams; import org.apache.doris.thrift.TStatus; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TTasksMetadataParams; @@ -100,12 +100,35 @@ public class MetadataGenerator { private static final ImmutableMap ACTIVE_QUERIES_COLUMN_TO_INDEX; + + private static final ImmutableList WORKLOAD_GROUPS_SCHEMA = ImmutableList.of( + new Column("ID", ScalarType.BIGINT), + new Column("NAME", ScalarType.createStringType()), + new Column("CPU_SHARE", PrimitiveType.BIGINT), + new Column("MEMORY_LIMIT", ScalarType.createStringType()), + new Column("ENABLE_MEMORY_OVERCOMMIT", ScalarType.createStringType()), + new Column("MAX_CONCURRENCY", PrimitiveType.BIGINT), + new Column("MAX_QUEUE_SIZE", PrimitiveType.BIGINT), + new Column("QUEUE_TIMEOUT", PrimitiveType.BIGINT), + new Column("CPU_HARD_LIMIT", PrimitiveType.BIGINT), + new Column("SCAN_THREAD_NUM", PrimitiveType.BIGINT), + new Column("MAX_REMOTE_SCAN_THREAD_NUM", PrimitiveType.BIGINT), + new Column("MIN_REMOTE_SCAN_THREAD_NUM", PrimitiveType.BIGINT)); + + private static final ImmutableMap WORKLOAD_GROUPS_COLUMN_TO_INDEX; + static { - ImmutableMap.Builder builder = new ImmutableMap.Builder(); + ImmutableMap.Builder activeQueriesbuilder = new ImmutableMap.Builder(); for (int i = 0; i < ACTIVE_QUERIES_SCHEMA.size(); i++) { - builder.put(ACTIVE_QUERIES_SCHEMA.get(i).getName().toLowerCase(), i); + activeQueriesbuilder.put(ACTIVE_QUERIES_SCHEMA.get(i).getName().toLowerCase(), i); + } + ACTIVE_QUERIES_COLUMN_TO_INDEX = activeQueriesbuilder.build(); + + ImmutableMap.Builder workloadGroupsBuilder = new ImmutableMap.Builder(); + for (int i = 0; i < WORKLOAD_GROUPS_SCHEMA.size(); i++) { + workloadGroupsBuilder.put(WORKLOAD_GROUPS_SCHEMA.get(i).getName().toLowerCase(), i); } - ACTIVE_QUERIES_COLUMN_TO_INDEX = builder.build(); + WORKLOAD_GROUPS_COLUMN_TO_INDEX = workloadGroupsBuilder.build(); } public static TFetchSchemaTableDataResult getMetadataTable(TFetchSchemaTableDataRequest request) throws TException { @@ -127,9 +150,6 @@ public static TFetchSchemaTableDataResult getMetadataTable(TFetchSchemaTableData case FRONTENDS_DISKS: result = frontendsDisksMetadataResult(params); break; - case WORKLOAD_GROUPS: - result = workloadGroupsMetadataResult(params); - break; case CATALOGS: result = catalogsMetadataResult(params); break; @@ -156,23 +176,26 @@ public static TFetchSchemaTableDataResult getMetadataTable(TFetchSchemaTableData public static TFetchSchemaTableDataResult getSchemaTableData(TFetchSchemaTableDataRequest request) throws TException { - if (!request.isSetMetadaTableParams() || !request.getMetadaTableParams().isSetMetadataType()) { - return errorResult("Metadata table params is not set. "); + if (!request.isSetSchemaTableParams()) { + return errorResult("schema table params is not set."); } TFetchSchemaTableDataResult result; - TMetadataTableRequestParams params = request.getMetadaTableParams(); + TSchemaTableRequestParams schemaTableParams = request.getSchemaTableParams(); ImmutableMap columnIndex; - // todo(wb) move workload group/workload scheduler policy here - switch (request.getMetadaTableParams().getMetadataType()) { - case QUERIES: - result = queriesMetadataResult(params, request); + switch (request.getSchemaTableName()) { + case ACTIVE_QUERIES: + result = queriesMetadataResult(schemaTableParams, request); columnIndex = ACTIVE_QUERIES_COLUMN_TO_INDEX; break; + case WORKLOAD_GROUPS: + result = workloadGroupsMetadataResult(schemaTableParams); + columnIndex = WORKLOAD_GROUPS_COLUMN_TO_INDEX; + break; default: - return errorResult("schema table params is not set."); + return errorResult("invalid schema table name."); } - if (result.getStatus().getStatusCode() == TStatusCode.OK) { - filterColumns(result, params.getColumnsName(), columnIndex); + if (schemaTableParams.isSetColumnsName() && result.getStatus().getStatusCode() == TStatusCode.OK) { + filterColumns(result, schemaTableParams.getColumnsName(), columnIndex); } return result; } @@ -406,7 +429,7 @@ private static TFetchSchemaTableDataResult catalogsMetadataResult(TMetadataTable return result; } - private static TFetchSchemaTableDataResult workloadGroupsMetadataResult(TMetadataTableRequestParams params) { + private static TFetchSchemaTableDataResult workloadGroupsMetadataResult(TSchemaTableRequestParams params) { if (!params.isSetCurrentUserIdent()) { return errorResult("current user ident is not set."); } @@ -427,13 +450,11 @@ private static TFetchSchemaTableDataResult workloadGroupsMetadataResult(TMetadat 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().setIntVal(Integer.parseInt(rGroupsInfo.get(9)))); // scan thread num + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(9)))); // scan thread num // max remote scan thread num - trow.addToColumnValue(new TCell().setIntVal(Integer.parseInt(rGroupsInfo.get(10)))); + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(10)))); // min remote scan thread num - trow.addToColumnValue(new TCell().setIntVal(Integer.parseInt(rGroupsInfo.get(11)))); - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(12)))); // running query num - trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(13)))); // waiting query num + trow.addToColumnValue(new TCell().setLongVal(Long.valueOf(rGroupsInfo.get(11)))); dataBatch.add(trow); } @@ -516,13 +537,8 @@ private static TRow makeQueryStatisticsTRow(SimpleDateFormat sdf, String queryId return trow; } - private static TFetchSchemaTableDataResult queriesMetadataResult(TMetadataTableRequestParams params, + private static TFetchSchemaTableDataResult queriesMetadataResult(TSchemaTableRequestParams tSchemaTableParams, TFetchSchemaTableDataRequest parentRequest) { - if (!params.isSetQueriesMetadataParams()) { - return errorResult("queries metadata param is not set."); - } - - TQueriesMetadataParams queriesMetadataParams = params.getQueriesMetadataParams(); TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult(); String selfNode = Env.getCurrentEnv().getSelfNode().getHost(); @@ -563,16 +579,14 @@ private static TFetchSchemaTableDataResult queriesMetadataResult(TMetadataTableR } /* Get the query results from other FE also */ - if (queriesMetadataParams.isRelayToOtherFe()) { - TFetchSchemaTableDataRequest relayRequest = new TFetchSchemaTableDataRequest(parentRequest); - TMetadataTableRequestParams relayParams = new TMetadataTableRequestParams(params); - TQueriesMetadataParams relayQueryParams = new TQueriesMetadataParams(queriesMetadataParams); + if (tSchemaTableParams.isReplayToOtherFe()) { + TSchemaTableRequestParams replaySchemaTableParams = new TSchemaTableRequestParams(tSchemaTableParams); + replaySchemaTableParams.setReplayToOtherFe(false); - relayQueryParams.setRelayToOtherFe(false); - relayParams.setQueriesMetadataParams(relayQueryParams); - relayRequest.setMetadaTableParams(relayParams); + TFetchSchemaTableDataRequest replayFetchSchemaTableReq = new TFetchSchemaTableDataRequest(parentRequest); + replayFetchSchemaTableReq.setSchemaTableParams(replaySchemaTableParams); - List relayResults = forwardToOtherFrontends(relayRequest); + List relayResults = forwardToOtherFrontends(replayFetchSchemaTableReq); relayResults .forEach(rs -> rs.getDataBatch() .forEach(row -> dataBatch.add(row))); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java index 1947c241ef7160..56e769cc8b3a21 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataTableValuedFunction.java @@ -40,8 +40,6 @@ public static Integer getColumnIndexFromColumnName(TMetadataType type, String co return FrontendsDisksTableValuedFunction.getColumnIndexFromColumnName(columnName); case ICEBERG: return IcebergTableValuedFunction.getColumnIndexFromColumnName(columnName); - case WORKLOAD_GROUPS: - return WorkloadGroupsTableValuedFunction.getColumnIndexFromColumnName(columnName); case CATALOGS: return CatalogsTableValuedFunction.getColumnIndexFromColumnName(columnName); case MATERIALIZED_VIEWS: diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java index 2c63cdca1c6237..64e794757d10b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java @@ -62,8 +62,6 @@ public static TableValuedFunctionIf getTableFunction(String funcName, Map SCHEMA = ImmutableList.of( - new Column("Id", ScalarType.createType(PrimitiveType.BIGINT)), - new Column("Name", ScalarType.createStringType()), - new Column(WorkloadGroup.CPU_SHARE, ScalarType.createType(PrimitiveType.BIGINT)), - new Column(WorkloadGroup.MEMORY_LIMIT, ScalarType.createStringType()), - new Column(WorkloadGroup.ENABLE_MEMORY_OVERCOMMIT, ScalarType.createStringType()), - new Column(WorkloadGroup.MAX_CONCURRENCY, ScalarType.createType(PrimitiveType.BIGINT)), - new Column(WorkloadGroup.MAX_QUEUE_SIZE, ScalarType.createType(PrimitiveType.BIGINT)), - new Column(WorkloadGroup.QUEUE_TIMEOUT, ScalarType.createType(PrimitiveType.BIGINT)), - new Column(WorkloadGroup.CPU_HARD_LIMIT, ScalarType.createStringType()), - new Column(WorkloadGroup.SCAN_THREAD_NUM, ScalarType.createType(PrimitiveType.INT)), - new Column(WorkloadGroup.MAX_REMOTE_SCAN_THREAD_NUM, ScalarType.createType(PrimitiveType.INT)), - new Column(WorkloadGroup.MIN_REMOTE_SCAN_THREAD_NUM, ScalarType.createType(PrimitiveType.INT)), - new Column(QueryQueue.RUNNING_QUERY_NUM, ScalarType.createType(PrimitiveType.BIGINT)), - new Column(QueryQueue.WAITING_QUERY_NUM, ScalarType.createType(PrimitiveType.BIGINT))); - - private static final ImmutableMap COLUMN_TO_INDEX; - - static { - ImmutableMap.Builder builder = new ImmutableMap.Builder(); - for (int i = 0; i < SCHEMA.size(); i++) { - builder.put(SCHEMA.get(i).getName().toLowerCase(), i); - } - COLUMN_TO_INDEX = builder.build(); - } - - public static Integer getColumnIndexFromColumnName(String columnName) { - return COLUMN_TO_INDEX.get(columnName.toLowerCase()); - } - - public WorkloadGroupsTableValuedFunction(Map params) throws AnalysisException { - if (params.size() != 0) { - throw new AnalysisException("workload groups table-valued-function does not support any params"); - } - } - - @Override - public TMetadataType getMetadataType() { - return TMetadataType.WORKLOAD_GROUPS; - } - - @Override - public TMetaScanRange getMetaScanRange() { - TMetaScanRange metaScanRange = new TMetaScanRange(); - metaScanRange.setMetadataType(TMetadataType.WORKLOAD_GROUPS); - return metaScanRange; - } - - @Override - public String getTableName() { - return "WorkloadGroupsTableValuedFunction"; - } - - @Override - public List getTableColumns() throws AnalysisException { - return SCHEMA; - } -} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/RefreshCatalogTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/RefreshCatalogTest.java index 4e327644bd23bd..7e8e8c3322484d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/RefreshCatalogTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/RefreshCatalogTest.java @@ -87,7 +87,7 @@ public void testRefreshCatalog() throws Exception { List dbNames2 = test1.getDbNames(); Assertions.assertEquals(4, dbNames2.size()); ExternalInfoSchemaDatabase infoDb = (ExternalInfoSchemaDatabase) test1.getDb(InfoSchemaDb.DATABASE_NAME).get(); - Assertions.assertEquals(29, infoDb.getTables().size()); + Assertions.assertEquals(30, infoDb.getTables().size()); TestExternalDatabase testDb = (TestExternalDatabase) test1.getDb("db1").get(); Assertions.assertEquals(2, testDb.getTables().size()); @@ -96,7 +96,7 @@ public void testRefreshCatalog() throws Exception { CatalogMgr mgr2 = GsonUtils.GSON.fromJson(json, CatalogMgr.class); test1 = mgr2.getCatalog("test1"); infoDb = (ExternalInfoSchemaDatabase) test1.getDb(InfoSchemaDb.DATABASE_NAME).get(); - Assertions.assertEquals(29, infoDb.getTables().size()); + Assertions.assertEquals(30, infoDb.getTables().size()); testDb = (TestExternalDatabase) test1.getDb("db1").get(); Assertions.assertEquals(2, testDb.getTables().size()); } diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index 50848140674ecc..76c8b1b7b068ec 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -127,7 +127,8 @@ enum TSchemaTableType { SCH_METADATA_NAME_IDS, SCH_PROFILING, SCH_BACKEND_ACTIVE_TASKS, - SCH_ACTIVE_QUERIES; + SCH_ACTIVE_QUERIES, + SCH_WORKLOAD_GROUPS; } enum THdfsCompression { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 952fd4ffbbb803..0bac9904cc8d86 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -919,7 +919,8 @@ struct TInitExternalCtlMetaResult { enum TSchemaTableName { // BACKENDS = 0, METADATA_TABLE = 1, // tvf - SCHEMA_TABLE = 2, // db information_schema's table + ACTIVE_QUERIES = 2, // db information_schema's table + WORKLOAD_GROUPS = 3, // db information_schema's table } struct TMetadataTableRequestParams { @@ -935,10 +936,17 @@ struct TMetadataTableRequestParams { 10: optional PlanNodes.TTasksMetadataParams tasks_metadata_params } +struct TSchemaTableRequestParams { + 1: optional list columns_name + 2: optional Types.TUserIdentity current_user_ident + 3: optional bool replay_to_other_fe +} + struct TFetchSchemaTableDataRequest { 1: optional string cluster_name 2: optional TSchemaTableName schema_table_name - 3: optional TMetadataTableRequestParams metada_table_params + 3: optional TMetadataTableRequestParams metada_table_params // used for tvf + 4: optional TSchemaTableRequestParams schema_table_params // used for request db information_schema's table } struct TFetchSchemaTableDataResult { diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 2cd6af9f050130..66694645d74f66 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -707,14 +707,12 @@ enum TSortType { enum TMetadataType { ICEBERG, BACKENDS, - WORKLOAD_GROUPS, FRONTENDS, CATALOGS, FRONTENDS_DISKS, MATERIALIZED_VIEWS, JOBS, TASKS, - QUERIES, WORKLOAD_SCHED_POLICY } diff --git a/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out index 8023cf076c31ab..a83c87b49c2b67 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out @@ -52,6 +52,7 @@ tables triggers user_privileges views +workload_groups -- !auto_default_t -- 0 diff --git a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out index f03a7591cf1814..e2e853d7cf461a 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out @@ -221,6 +221,7 @@ tables triggers user_privileges views +workload_groups -- !dt -- 2023-06-17T10:00 2023-06-17T10:00:01.100 2023-06-17T10:00:02.220 2023-06-17T10:00:03.333 2023-06-17T10:00:04.444400 2023-06-17T10:00:05.555550 2023-06-17T10:00:06.666666 diff --git a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out index 8cf9d01b6e5199..6b895c059f1e99 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out +++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out @@ -189,6 +189,7 @@ tables triggers user_privileges views +workload_groups -- !test_insert1 -- doris1 18 diff --git a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out index 5e63f90319d8d0..2665f803a658cb 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out @@ -221,6 +221,7 @@ tables triggers user_privileges views +workload_groups -- !dt -- 2023-06-17T10:00 2023-06-17T10:00:01 2023-06-17T10:00:02 2023-06-17T10:00:03 2023-06-17T10:00:04 2023-06-17T10:00:05 2023-06-17T10:00:06 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 864d1ab5b21c24..92b4836ea7f2ba 100644 --- a/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy +++ b/regression-test/suites/workload_manager_p0/test_curd_wlg.groovy @@ -117,7 +117,7 @@ suite("test_crud_wlg") { ");" sql "set workload_group=test_group;" - qt_show_1 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from workload_groups() where name in ('normal','test_group') order by name;" + qt_show_1 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from information_schema.workload_groups where name in ('normal','test_group') order by name;" // test memory_limit test { @@ -128,7 +128,7 @@ suite("test_crud_wlg") { sql "alter workload group test_group properties ( 'memory_limit'='11%' );" qt_mem_limit_1 """ select count(1) from ${table_name} """ - qt_mem_limit_2 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from workload_groups() where name in ('normal','test_group') order by name;" + qt_mem_limit_2 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from information_schema.workload_groups where name in ('normal','test_group') order by name;" // test enable_memory_overcommit test { @@ -141,7 +141,7 @@ suite("test_crud_wlg") { qt_mem_overcommit_1 """ select count(1) from ${table_name} """ sql "alter workload group test_group properties ( 'enable_memory_overcommit'='false' );" qt_mem_overcommit_2 """ select count(1) from ${table_name} """ - qt_mem_overcommit_3 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from workload_groups() where name in ('normal','test_group') order by name;" + qt_mem_overcommit_3 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from information_schema.workload_groups where name in ('normal','test_group') order by name;" // test cpu_hard_limit test { @@ -160,7 +160,7 @@ suite("test_crud_wlg") { sql "alter workload group test_group properties ( 'cpu_hard_limit'='20%' );" qt_cpu_hard_limit_1 """ select count(1) from ${table_name} """ - qt_cpu_hard_limit_2 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from workload_groups() where name in ('normal','test_group') order by name;" + qt_cpu_hard_limit_2 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from information_schema.workload_groups where name in ('normal','test_group') order by name;" // test query queue test { @@ -183,7 +183,7 @@ suite("test_crud_wlg") { sql "alter workload group test_group properties ( 'max_concurrency'='100' );" qt_queue_1 """ select count(1) from ${table_name} """ - qt_show_queue "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from workload_groups() where name in ('normal','test_group') order by name;" + qt_show_queue "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from information_schema.workload_groups where name in ('normal','test_group') order by name;" // test create group failed // failed for cpu_share @@ -261,7 +261,7 @@ suite("test_crud_wlg") { } // test show workload groups - qt_select_tvf_1 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from workload_groups() where name in ('normal','test_group') order by name;" + qt_select_tvf_1 "select name,cpu_share,memory_limit,enable_memory_overcommit,max_concurrency,max_queue_size,queue_timeout,cpu_hard_limit,scan_thread_num from information_schema.workload_groups where name in ('normal','test_group') order by name;" // test auth sql """drop user if exists test_wlg_user""" From 2636930600971763d1d9e5c9a923e5d8847c68f2 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Thu, 14 Mar 2024 14:57:32 +0800 Subject: [PATCH 03/10] [feature](Nereids) support agg state type in create table (#32171) this PR introduce a behavior change, syntax of create table with agg_state type is changed. --- .../sql-reference/Data-Types/AGG_STATE.md | 4 +- .../sql-reference/Data-Types/AGG_STATE.md | 4 +- .../apache/doris/catalog/AggStateType.java | 4 +- .../org/apache/doris/nereids/DorisLexer.g4 | 1 + .../org/apache/doris/nereids/DorisParser.g4 | 15 ++++- fe/fe-core/src/main/cup/sql_parser.cup | 33 ++++------ .../org/apache/doris/analysis/ColumnDef.java | 4 +- .../analysis/CreateMaterializedViewStmt.java | 2 +- .../apache/doris/catalog/AggregateType.java | 2 +- .../java/org/apache/doris/catalog/Column.java | 35 ++-------- .../nereids/parser/LogicalPlanBuilder.java | 34 +++++++++- .../SelectMaterializedIndexWithAggregate.java | 2 +- .../plans/commands/info/ColumnDefinition.java | 64 ++++++++++--------- .../doris/nereids/types/AggStateType.java | 7 -- fe/fe-core/src/main/jflex/sql_scanner.flex | 1 + .../test_vertical_compaction_agg_state.groovy | 2 +- .../agg_state/avg/test_agg_state_avg.groovy | 2 +- .../test_agg_state_group_concat.groovy | 2 +- .../agg_state/max/test_agg_state_max.groovy | 4 +- .../nereids/test_agg_state_nereids.groovy | 2 +- .../test_agg_state_quantile_union.groovy | 2 +- .../agg_state/test_agg_state.groovy | 2 +- .../suites/mv_p0/dis_26495/dis_26495.groovy | 2 +- .../statistics/test_agg_complex_type.groovy | 2 +- 24 files changed, 121 insertions(+), 111 deletions(-) diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/AGG_STATE.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/AGG_STATE.md index 41535d923b7d8b..9dea25a52a5e04 100644 --- a/docs/en/docs/sql-manual/sql-reference/Data-Types/AGG_STATE.md +++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/AGG_STATE.md @@ -41,8 +41,8 @@ Create table example: ```sql create table a_table( k1 int null, - k2 agg_state max_by(int not null,int), - k3 agg_state group_concat(string) + k2 agg_state generic, + k3 agg_state generic ) aggregate key (k1) distributed BY hash(k1) buckets 3 diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/AGG_STATE.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/AGG_STATE.md index 18dc95c324915e..aa459dd7483761 100644 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/AGG_STATE.md +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/AGG_STATE.md @@ -41,8 +41,8 @@ under the License. ```sql create table a_table( k1 int null, - k2 agg_state max_by(int not null,int), - k3 agg_state group_concat(string) + k2 agg_state generic, + k3 agg_state generic ) aggregate key (k1) distributed BY hash(k1) buckets 3 diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/AggStateType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/AggStateType.java index 35301fa704e5e6..72f93a2f696805 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/AggStateType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/AggStateType.java @@ -72,7 +72,7 @@ public boolean getResultIsNullable() { @Override public String toSql(int depth) { StringBuilder stringBuilder = new StringBuilder(); - stringBuilder.append("AGG_STATE("); + stringBuilder.append("AGG_STATE<").append(functionName).append("("); for (int i = 0; i < subTypes.size(); i++) { if (i > 0) { stringBuilder.append(", "); @@ -82,7 +82,7 @@ public String toSql(int depth) { stringBuilder.append(" NULL"); } } - stringBuilder.append(")"); + stringBuilder.append(")>"); return stringBuilder.toString(); } diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 index 02304ee30c6ac1..256888c2fc1409 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 @@ -271,6 +271,7 @@ FRONTENDS: 'FRONTENDS'; FULL: 'FULL'; FUNCTION: 'FUNCTION'; FUNCTIONS: 'FUNCTIONS'; +GENERIC: 'GENERIC'; GLOBAL: 'GLOBAL'; GRANT: 'GRANT'; GRANTS: 'GRANTS'; diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 133eea3ed7efad..e0a306196713e2 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -530,7 +530,10 @@ columnDefs columnDef : colName=identifier type=dataType - KEY? (aggType=aggTypeDef)? ((NOT NULL) | NULL)? (AUTO_INCREMENT (LEFT_PAREN autoIncInitValue=number RIGHT_PAREN)?)? + KEY? + (aggType=aggTypeDef)? + ((NOT)? NULL)? + (AUTO_INCREMENT (LEFT_PAREN autoIncInitValue=number RIGHT_PAREN)?)? (DEFAULT (nullValue=NULL | INTEGER_VALUE | stringValue=STRING_LITERAL | CURRENT_TIMESTAMP (LEFT_PAREN defaultValuePrecision=number RIGHT_PAREN)?))? (ON UPDATE CURRENT_TIMESTAMP (LEFT_PAREN onUpdateValuePrecision=number RIGHT_PAREN)?)? @@ -587,7 +590,7 @@ rollupDef ; aggTypeDef - : MAX | MIN | SUM | REPLACE | REPLACE_IF_NOT_NULL | HLL_UNION | BITMAP_UNION | QUANTILE_UNION + : MAX | MIN | SUM | REPLACE | REPLACE_IF_NOT_NULL | HLL_UNION | BITMAP_UNION | QUANTILE_UNION | GENERIC ; tabletList @@ -846,10 +849,17 @@ unitIdentifier : YEAR | MONTH | WEEK | DAY | HOUR | MINUTE | SECOND ; +dataTypeWithNullable + : dataType ((NOT)? NULL)? + ; + dataType : complex=ARRAY LT dataType GT #complexDataType | complex=MAP LT dataType COMMA dataType GT #complexDataType | complex=STRUCT LT complexColTypeList GT #complexDataType + | AGG_STATE LT functionNameIdentifier + LEFT_PAREN dataTypes+=dataTypeWithNullable + (COMMA dataTypes+=dataTypeWithNullable)* RIGHT_PAREN GT #aggStateDataType | primitiveColType (LEFT_PAREN (INTEGER_VALUE | ASTERISK) (COMMA INTEGER_VALUE)* RIGHT_PAREN)? #primitiveDataType ; @@ -1061,6 +1071,7 @@ nonReserved | FREE | FRONTENDS | FUNCTION + | GENERIC | GLOBAL | GRAPH | GROUPING diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 23efc77cc05fa7..0770ac73da7675 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -402,6 +402,7 @@ terminal String KW_FULL, KW_FUNCTION, KW_FUNCTIONS, + KW_GENERIC, KW_GLOBAL, KW_GRANT, KW_GRANTS, @@ -3157,6 +3158,10 @@ opt_agg_type ::= {: RESULT = AggregateType.QUANTILE_UNION; :} + | KW_GENERIC + {: + RESULT = AggregateType.GENERIC; + :} ; opt_partition ::= @@ -3756,31 +3761,11 @@ column_definition ::= ColumnDef columnDef = new ColumnDef(columnName, typeDef, isKey, null, isAllowNull, autoIncInitValue, defaultValue, comment); RESULT = columnDef; :} - | ident:columnName KW_AGG_STATE IDENT:fnName LPAREN type_def_nullable_list:list RPAREN opt_auto_inc_init_value:autoIncInitValue opt_default_value:defaultValue opt_comment:comment - {: - for (TypeDef def : list) { - def.analyze(null); - } - ColumnDef columnDef = new ColumnDef(columnName, new TypeDef(Expr.createAggStateType(fnName, - list.stream().map(TypeDef::getType).collect(Collectors.toList()), - list.stream().map(TypeDef::getNullable).collect(Collectors.toList()))), false, AggregateType.GENERIC_AGGREGATION, false, defaultValue, comment); - RESULT = columnDef; - :} | ident:columnName type_def:typeDef opt_is_key:isKey opt_agg_type:aggType opt_is_allow_null:isAllowNull opt_auto_inc_init_value:autoIncInitValue opt_default_value:defaultValue opt_comment:comment {: ColumnDef columnDef = new ColumnDef(columnName, typeDef, isKey, aggType, isAllowNull, autoIncInitValue, defaultValue, comment); RESULT = columnDef; :} - | ident:columnName KW_AGG_STATE opt_is_key:isKey opt_agg_type:aggType LPAREN type_def_nullable_list:list RPAREN opt_default_value:defaultValue opt_comment:comment - {: - for (TypeDef def : list) { - def.analyze(null); - } - ColumnDef columnDef = new ColumnDef(columnName, new TypeDef(Expr.createAggStateType(aggType.name().toLowerCase(), - list.stream().map(TypeDef::getType).collect(Collectors.toList()), - list.stream().map(TypeDef::getNullable).collect(Collectors.toList()))), isKey, AggregateType.GENERIC_AGGREGATION, false, defaultValue, comment); - RESULT = columnDef; - :} ; index_definition ::= @@ -6599,6 +6584,12 @@ type ::= {: ScalarType type = ScalarType.createHllType(); RESULT = type; :} + | KW_AGG_STATE LESSTHAN IDENT:fnName LPAREN type_def_nullable_list:list RPAREN GREATERTHAN + {: + RESULT = Expr.createAggStateType(fnName, + list.stream().map(TypeDef::getType).collect(Collectors.toList()), + list.stream().map(TypeDef::getNullable).collect(Collectors.toList())); + :} | KW_ALL {: RESULT = Type.ALL; :} ; @@ -7828,6 +7819,8 @@ keyword ::= {: RESULT = id; :} | KW_GLOBAL:id {: RESULT = id; :} + | KW_GENERIC:id + {: RESULT = id; :} | KW_GRAPH:id {: RESULT = id; :} | KW_HASH:id diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java index 59e5be4725b5fb..d202467dfb37ba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java @@ -332,12 +332,12 @@ public void analyze(boolean isOlap) throws AnalysisException { } // check if aggregate type is valid - if (aggregateType != AggregateType.GENERIC_AGGREGATION + if (aggregateType != AggregateType.GENERIC && !aggregateType.checkCompatibility(type.getPrimitiveType())) { throw new AnalysisException(String.format("Aggregate type %s is not compatible with primitive type %s", toString(), type.toSql())); } - if (aggregateType == AggregateType.GENERIC_AGGREGATION) { + if (aggregateType == AggregateType.GENERIC) { if (!SessionVariable.enableAggState()) { throw new AnalysisException("agg state not enable, need set enable_agg_state=true"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateMaterializedViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateMaterializedViewStmt.java index 5686877a751d1e..f95b83ae404695 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateMaterializedViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateMaterializedViewStmt.java @@ -549,7 +549,7 @@ private MVColumnItem buildMVColumnItem(Analyzer analyzer, FunctionCallExpr funct type = Type.BIGINT; break; default: - mvAggregateType = AggregateType.GENERIC_AGGREGATION; + mvAggregateType = AggregateType.GENERIC; if (functionCallExpr.getParams().isDistinct() || functionCallExpr.getParams().isStar()) { throw new AnalysisException( "The Materialized-View's generic aggregation not support star or distinct"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateType.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateType.java index 47ee91c0b5ca02..94a788f0b9afe0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateType.java @@ -37,7 +37,7 @@ public enum AggregateType { NONE("NONE"), BITMAP_UNION("BITMAP_UNION"), QUANTILE_UNION("QUANTILE_UNION"), - GENERIC_AGGREGATION("GENERIC_AGGREGATION"); + GENERIC("GENERIC"); private static EnumMap> compatibilityMap; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 748f6f7b1beefa..82bac846d7d9d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -133,9 +133,6 @@ public class Column implements Writable, GsonPostProcessable { @SerializedName(value = "uniqueId") private int uniqueId; - @SerializedName(value = "genericAggregationName") - private String genericAggregationName; - @SerializedName(value = "clusterKeyId") private int clusterKeyId = -1; @@ -249,8 +246,8 @@ public Column(String name, Type type, boolean isKey, AggregateType aggregateType c.setIsAllowNull(aggState.getSubTypeNullables().get(i)); addChildrenColumn(c); } - this.genericAggregationName = aggState.getFunctionName(); - this.aggregationType = AggregateType.GENERIC_AGGREGATION; + this.isAllowNull = false; + this.aggregationType = AggregateType.GENERIC; } } @@ -454,11 +451,7 @@ public AggregateType getAggregationType() { } public String getAggregationString() { - if (getAggregationType() == AggregateType.GENERIC_AGGREGATION) { - return getGenericAggregationString(); - } else { - return getAggregationType().name(); - } + return getAggregationType().name(); } public boolean isAggregated() { @@ -915,22 +908,6 @@ public String toSql(boolean isUniqueTable) { return toSql(isUniqueTable, false); } - public String getGenericAggregationString() { - StringBuilder sb = new StringBuilder(); - sb.append(genericAggregationName).append("("); - for (int i = 0; i < children.size(); i++) { - if (i != 0) { - sb.append(", "); - } - sb.append(children.get(i).getType().toSql()); - if (children.get(i).isAllowNull()) { - sb.append(" NULL"); - } - } - sb.append(")"); - return sb.toString(); - } - public String toSql(boolean isUniqueTable, boolean isCompatible) { StringBuilder sb = new StringBuilder(); sb.append("`").append(name).append("` "); @@ -942,11 +919,9 @@ public String toSql(boolean isUniqueTable, boolean isCompatible) { } else { sb.append(typeStr); } - if (aggregationType == AggregateType.GENERIC_AGGREGATION) { - sb.append(" ").append(getGenericAggregationString()); - } else if (aggregationType != null && aggregationType != AggregateType.NONE && !isUniqueTable + if (aggregationType != null && aggregationType != AggregateType.NONE && !isUniqueTable && !isAggregationTypeImplicit) { - sb.append(" ").append(aggregationType.name()); + sb.append(" ").append(aggregationType.toSql()); } if (isAllowNull) { sb.append(" NULL"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 90a9b63f4f533f..1e2f8d2f23089b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -23,6 +23,7 @@ import org.apache.doris.analysis.TableName; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.AggregateType; +import org.apache.doris.catalog.BuiltinAggregateFunctions; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.ScalarType; @@ -42,6 +43,7 @@ import org.apache.doris.nereids.DorisParser; import org.apache.doris.nereids.DorisParser.AddConstraintContext; import org.apache.doris.nereids.DorisParser.AggClauseContext; +import org.apache.doris.nereids.DorisParser.AggStateDataTypeContext; import org.apache.doris.nereids.DorisParser.AliasQueryContext; import org.apache.doris.nereids.DorisParser.AliasedQueryContext; import org.apache.doris.nereids.DorisParser.AlterMTMVContext; @@ -75,6 +77,7 @@ import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext; import org.apache.doris.nereids.DorisParser.CreateTableContext; import org.apache.doris.nereids.DorisParser.CteContext; +import org.apache.doris.nereids.DorisParser.DataTypeWithNullableContext; import org.apache.doris.nereids.DorisParser.DateCeilContext; import org.apache.doris.nereids.DorisParser.DateFloorContext; import org.apache.doris.nereids.DorisParser.Date_addContext; @@ -422,6 +425,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.trees.plans.logical.UsingJoin; +import org.apache.doris.nereids.types.AggStateType; import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.MapType; @@ -2523,7 +2527,9 @@ public ColumnDefinition visitColumnDef(ColumnDefContext ctx) { String colName = ctx.colName.getText(); DataType colType = ctx.type instanceof PrimitiveDataTypeContext ? visitPrimitiveDataType(((PrimitiveDataTypeContext) ctx.type)) - : visitComplexDataType(((ComplexDataTypeContext) ctx.type)); + : ctx.type instanceof ComplexDataTypeContext + ? visitComplexDataType((ComplexDataTypeContext) ctx.type) + : visitAggStateDataType((AggStateDataTypeContext) ctx.type); colType = colType.conversion(); boolean isKey = ctx.KEY() != null; boolean isNotNull = ctx.NOT() != null; @@ -3256,6 +3262,32 @@ private ExplainLevel parseExplainPlanType(PlanTypeContext planTypeContext) { return ExplainLevel.ALL_PLAN; } + @Override + public Pair visitDataTypeWithNullable(DataTypeWithNullableContext ctx) { + return ParserUtils.withOrigin(ctx, () -> Pair.of(typedVisit(ctx.dataType()), ctx.NOT() == null)); + } + + @Override + public DataType visitAggStateDataType(AggStateDataTypeContext ctx) { + return ParserUtils.withOrigin(ctx, () -> { + List> dataTypeWithNullables = ctx.dataTypes.stream() + .map(this::visitDataTypeWithNullable) + .collect(Collectors.toList()); + List dataTypes = dataTypeWithNullables.stream() + .map(dt -> dt.first) + .collect(ImmutableList.toImmutableList()); + List nullables = dataTypeWithNullables.stream() + .map(dt -> dt.second) + .collect(ImmutableList.toImmutableList()); + String functionName = ctx.functionNameIdentifier().getText(); + if (!BuiltinAggregateFunctions.INSTANCE.aggFuncNames.contains(functionName)) { + // TODO use function binder to check function exists + throw new ParseException("Can not found function '" + functionName + "'", ctx); + } + return new AggStateType(functionName, dataTypes, nullables); + }); + } + @Override public DataType visitPrimitiveDataType(PrimitiveDataTypeContext ctx) { return ParserUtils.withOrigin(ctx, () -> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java index 3857a04ba1c7a9..ea09b25ba6f493 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/SelectMaterializedIndexWithAggregate.java @@ -1487,7 +1487,7 @@ public Expression visitSum(Sum sum, RewriteContext context) { @Override public Expression visitAggregateFunction(AggregateFunction aggregateFunction, RewriteContext context) { String aggStateName = normalizeName(CreateMaterializedViewStmt.mvColumnBuilder( - AggregateType.GENERIC_AGGREGATION, StateCombinator.create(aggregateFunction).toSql())); + AggregateType.GENERIC, StateCombinator.create(aggregateFunction).toSql())); Column mvColumn = context.checkContext.getColumn(aggStateName); if (mvColumn != null && context.checkContext.valueNameToColumn.containsValue(mvColumn)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java index eac17f05d370f6..9360f2d608a482 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ColumnDefinition.java @@ -246,6 +246,20 @@ public void validate(boolean isOlap, Set keysSet, boolean isEnableMergeO } } + if (aggType != null) { + // check if aggregate type is valid + if (aggType != AggregateType.GENERIC + && !aggType.checkCompatibility(type.toCatalogDataType().getPrimitiveType())) { + throw new AnalysisException(String.format("Aggregate type %s is not compatible with primitive type %s", + aggType, type.toSql())); + } + if (aggType == AggregateType.GENERIC) { + if (!SessionVariable.enableAggState()) { + throw new AnalysisException("agg state not enable, need set enable_agg_state=true"); + } + } + } + if (isOlap) { if (!isKey && keysType.equals(KeysType.UNIQUE_KEYS)) { aggTypeImplicit = true; @@ -334,14 +348,28 @@ public void validate(boolean isOlap, Set keysSet, boolean isEnableMergeO // from old planner CreateTableStmt's analyze method, after call columnDef.analyze(engineName.equals("olap")); if (isOlap && type.isComplexType()) { - if (aggType != null && aggType != AggregateType.NONE - && aggType != AggregateType.REPLACE) { - throw new AnalysisException(type.toCatalogDataType().getPrimitiveType() - + " column can't support aggregation " + aggType); - } if (isKey) { throw new AnalysisException(type.toCatalogDataType().getPrimitiveType() - + " can only be used in the non-key column of the duplicate table at present."); + + " can only be used in the non-key column at present."); + } + if (type.isAggStateType()) { + if (aggType == null) { + throw new AnalysisException(type.toCatalogDataType().getPrimitiveType() + + " column must have aggregation type"); + } else { + if (aggType != AggregateType.GENERIC + && aggType != AggregateType.NONE + && aggType != AggregateType.REPLACE) { + throw new AnalysisException(type.toCatalogDataType().getPrimitiveType() + + " column can't support aggregation " + aggType); + } + } + isNullable = false; + } else { + if (aggType != null && aggType != AggregateType.NONE && aggType != AggregateType.REPLACE) { + throw new AnalysisException(type.toCatalogDataType().getPrimitiveType() + + " column can't support aggregation " + aggType); + } } } @@ -350,30 +378,6 @@ public void validate(boolean isOlap, Set keysSet, boolean isEnableMergeO } } - /** - * check if is nested complex type. - */ - private boolean isNestedComplexType(DataType dataType) { - if (!dataType.isComplexType()) { - return false; - } - if (dataType instanceof ArrayType) { - if (((ArrayType) dataType).getItemType() instanceof ArrayType) { - return isNestedComplexType(((ArrayType) dataType).getItemType()); - } else { - return ((ArrayType) dataType).getItemType().isComplexType(); - } - } - if (dataType instanceof MapType) { - return ((MapType) dataType).getKeyType().isComplexType() - || ((MapType) dataType).getValueType().isComplexType(); - } - if (dataType instanceof StructType) { - return ((StructType) dataType).getFields().stream().anyMatch(f -> f.getDataType().isComplexType()); - } - return false; - } - // from TypeDef.java analyze() private void validateDataType(Type catalogType) { if (catalogType.exceedsMaxNestingDepth()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java index 6680a6ccee0827..e7d8dbdfe3dad4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/AggStateType.java @@ -35,8 +35,6 @@ */ public class AggStateType extends DataType { - public static final AggStateType SYSTEM_DEFAULT = new AggStateType(null, ImmutableList.of(), ImmutableList.of()); - public static final int WIDTH = 16; private final List subTypes; @@ -94,11 +92,6 @@ public String simpleString() { return "agg_state"; } - @Override - public DataType defaultConcreteType() { - return SYSTEM_DEFAULT; - } - @Override public boolean equals(Object o) { if (!(o instanceof AggStateType)) { diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex index 175744532c033f..43dd60a4ce1fb9 100644 --- a/fe/fe-core/src/main/jflex/sql_scanner.flex +++ b/fe/fe-core/src/main/jflex/sql_scanner.flex @@ -251,6 +251,7 @@ import org.apache.doris.qe.SqlModeHelper; keywordMap.put("function", new Integer(SqlParserSymbols.KW_FUNCTION)); keywordMap.put("functions", new Integer(SqlParserSymbols.KW_FUNCTIONS)); keywordMap.put("type_cast", new Integer(SqlParserSymbols.KW_TYPECAST)); + keywordMap.put("generic", new Integer(SqlParserSymbols.KW_GENERIC)); keywordMap.put("global", new Integer(SqlParserSymbols.KW_GLOBAL)); keywordMap.put("grant", new Integer(SqlParserSymbols.KW_GRANT)); keywordMap.put("grants", new Integer(SqlParserSymbols.KW_GRANTS)); diff --git a/regression-test/suites/compaction/test_vertical_compaction_agg_state.groovy b/regression-test/suites/compaction/test_vertical_compaction_agg_state.groovy index 9e0f99dd0c35b5..209f816785ca0a 100644 --- a/regression-test/suites/compaction/test_vertical_compaction_agg_state.groovy +++ b/regression-test/suites/compaction/test_vertical_compaction_agg_state.groovy @@ -46,7 +46,7 @@ suite("test_vertical_compaction_agg_state") { sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( user_id VARCHAR, - agg_user_id agg_state collect_set(string) + agg_user_id agg_state generic )ENGINE=OLAP AGGREGATE KEY(`user_id`) COMMENT 'OLAP' diff --git a/regression-test/suites/datatype_p0/agg_state/avg/test_agg_state_avg.groovy b/regression-test/suites/datatype_p0/agg_state/avg/test_agg_state_avg.groovy index 977f855f52da55..fb86b4428f0c7d 100644 --- a/regression-test/suites/datatype_p0/agg_state/avg/test_agg_state_avg.groovy +++ b/regression-test/suites/datatype_p0/agg_state/avg/test_agg_state_avg.groovy @@ -21,7 +21,7 @@ suite("test_agg_state_avg") { sql """ create table a_table( k1 int not null, - k2 agg_state avg(int not null) + k2 agg_state generic ) aggregate key (k1) distributed BY hash(k1) diff --git a/regression-test/suites/datatype_p0/agg_state/group_concat/test_agg_state_group_concat.groovy b/regression-test/suites/datatype_p0/agg_state/group_concat/test_agg_state_group_concat.groovy index fed7d075af5fcf..f17e44a1596114 100644 --- a/regression-test/suites/datatype_p0/agg_state/group_concat/test_agg_state_group_concat.groovy +++ b/regression-test/suites/datatype_p0/agg_state/group_concat/test_agg_state_group_concat.groovy @@ -21,7 +21,7 @@ suite("test_agg_state_group_concat") { sql """ create table a_table( k1 int null, - k2 agg_state group_concat(string) + k2 agg_state generic ) aggregate key (k1) distributed BY hash(k1) buckets 3 diff --git a/regression-test/suites/datatype_p0/agg_state/max/test_agg_state_max.groovy b/regression-test/suites/datatype_p0/agg_state/max/test_agg_state_max.groovy index 45b6e5463e9192..4bcc98a8b22baf 100644 --- a/regression-test/suites/datatype_p0/agg_state/max/test_agg_state_max.groovy +++ b/regression-test/suites/datatype_p0/agg_state/max/test_agg_state_max.groovy @@ -21,7 +21,7 @@ suite("test_agg_state_max") { sql """ create table a_table( k1 int not null, - k2 agg_state max(int not null) + k2 agg_state generic ) aggregate key (k1) distributed BY hash(k1) @@ -60,7 +60,7 @@ suite("test_agg_state_max") { sql """ create table a_table2( k1 int not null, - k2 agg_state max(int null) + k2 agg_state generic ) aggregate key (k1) distributed BY hash(k1) diff --git a/regression-test/suites/datatype_p0/agg_state/nereids/test_agg_state_nereids.groovy b/regression-test/suites/datatype_p0/agg_state/nereids/test_agg_state_nereids.groovy index 3adbfe9e43e968..0d9115664a53c1 100644 --- a/regression-test/suites/datatype_p0/agg_state/nereids/test_agg_state_nereids.groovy +++ b/regression-test/suites/datatype_p0/agg_state/nereids/test_agg_state_nereids.groovy @@ -48,7 +48,7 @@ suite("test_agg_state_nereids") { sql """ create table a_table( k1 int null, - k2 agg_state max_by(int not null,int) + k2 agg_state generic ) aggregate key (k1) distributed BY hash(k1) buckets 3 diff --git a/regression-test/suites/datatype_p0/agg_state/quantile_union/test_agg_state_quantile_union.groovy b/regression-test/suites/datatype_p0/agg_state/quantile_union/test_agg_state_quantile_union.groovy index 92f9f80a38dbd5..41d40ce8f49b03 100644 --- a/regression-test/suites/datatype_p0/agg_state/quantile_union/test_agg_state_quantile_union.groovy +++ b/regression-test/suites/datatype_p0/agg_state/quantile_union/test_agg_state_quantile_union.groovy @@ -21,7 +21,7 @@ suite("test_agg_state_quantile_union") { sql """ create table a_table( k1 int not null, - k2 agg_state quantile_union(quantile_state not null) + k2 agg_state generic ) aggregate key (k1) distributed BY hash(k1) diff --git a/regression-test/suites/datatype_p0/agg_state/test_agg_state.groovy b/regression-test/suites/datatype_p0/agg_state/test_agg_state.groovy index 19443191588b1e..78edeaaa340909 100644 --- a/regression-test/suites/datatype_p0/agg_state/test_agg_state.groovy +++ b/regression-test/suites/datatype_p0/agg_state/test_agg_state.groovy @@ -45,7 +45,7 @@ suite("test_agg_state") { sql """ create table a_table( k1 int null, - k2 agg_state max_by(int not null,int) + k2 agg_state generic ) aggregate key (k1) distributed BY hash(k1) buckets 3 diff --git a/regression-test/suites/mv_p0/dis_26495/dis_26495.groovy b/regression-test/suites/mv_p0/dis_26495/dis_26495.groovy index 4cc3842b9914f4..e30af70b0041b3 100644 --- a/regression-test/suites/mv_p0/dis_26495/dis_26495.groovy +++ b/regression-test/suites/mv_p0/dis_26495/dis_26495.groovy @@ -22,7 +22,7 @@ suite ("dis_26495") { sql """ DROP TABLE IF EXISTS doris_test; """ sql """ - create table doris_test (a int,b int, agg_st_1 agg_state max_by(int ,int)) + create table doris_test (a int,b int, agg_st_1 agg_state generic) DISTRIBUTED BY HASH(a) BUCKETS 1 properties("replication_num" = "1"); """ diff --git a/regression-test/suites/statistics/test_agg_complex_type.groovy b/regression-test/suites/statistics/test_agg_complex_type.groovy index 0f10234ae6ac0a..43cdeff4776533 100644 --- a/regression-test/suites/statistics/test_agg_complex_type.groovy +++ b/regression-test/suites/statistics/test_agg_complex_type.groovy @@ -24,7 +24,7 @@ suite("test_analyze_with_agg_complex_type") { device_id bitmap BITMAP_UNION NULL, hll_test hll hll_union, qs QUANTILE_STATE QUANTILE_UNION, - agg_st_1 agg_state max_by(int ,int) + agg_st_1 agg_state GENERIC ) aggregate key (datekey) distributed by hash(datekey) buckets 1 From ee0a461f96609ad013670284886caeb2704de502 Mon Sep 17 00:00:00 2001 From: Rohit Satardekar Date: Thu, 14 Mar 2024 12:31:25 +0530 Subject: [PATCH 04/10] [bugfix]json_length() BE crash fix (#32145) Co-authored-by: Rohit Satardekar --- be/src/vec/functions/function_jsonb.cpp | 5 +++-- .../data/jsonb_p0/test_jsonb_load_and_function.out | 3 +++ .../suites/jsonb_p0/test_jsonb_load_and_function.groovy | 1 + 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/be/src/vec/functions/function_jsonb.cpp b/be/src/vec/functions/function_jsonb.cpp index 4048e5c7799f02..317ba846832725 100644 --- a/be/src/vec/functions/function_jsonb.cpp +++ b/be/src/vec/functions/function_jsonb.cpp @@ -1080,7 +1080,8 @@ struct JsonbLengthUtil { MutableColumnPtr res = return_type->create_column(); for (size_t i = 0; i < input_rows_count; ++i) { - if (jsonb_data_column->is_null_at(i) || path_column->is_null_at(i)) { + if (jsonb_data_column->is_null_at(i) || path_column->is_null_at(i) || + (jsonb_data_column->get_data_at(i).size == 0)) { null_map->get_data()[i] = 1; res->insert_data(nullptr, 0); continue; @@ -1100,7 +1101,7 @@ struct JsonbLengthUtil { // doc is NOT necessary to be deleted since JsonbDocument will not allocate memory JsonbDocument* doc = JsonbDocument::createDocument(jsonb_value.data, jsonb_value.size); JsonbValue* value = doc->getValue()->findValue(path, nullptr); - if (UNLIKELY(jsonb_value.size == 0 || !value)) { + if (UNLIKELY(!value)) { null_map->get_data()[i] = 1; res->insert_data(nullptr, 0); continue; diff --git a/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out b/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out index 215f798df9b68f..8a3b573cebdba2 100644 --- a/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out +++ b/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out @@ -7494,6 +7494,9 @@ false -- !sql_json_length -- 2 +-- !sql_json_length -- +\N + -- !select_length -- 1 \N \N 2 null 1 diff --git a/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy b/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy index e6a9ca62dc94fe..bc63071f66a115 100644 --- a/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy +++ b/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy @@ -552,6 +552,7 @@ suite("test_jsonb_load_and_function", "p0") { qt_sql_json_length """SELECT json_length('{"k1":"v31","k2":300}')""" qt_sql_json_length """SELECT json_length('{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')""" qt_sql_json_length """SELECT json_length('{"a":{"k1.a1":"v31", "k2": 300},"b":"niu"}','\$.a')""" + qt_sql_json_length """SELECT json_length('abc','\$.k1')""" qt_select_length """SELECT id, j, json_length(j) FROM ${testTable} ORDER BY id""" qt_select_length """SELECT id, j, json_length(j, '\$[1]') FROM ${testTable} ORDER BY id""" From 60f97abb1f954c36bd8431c701e86a38b1422260 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Thu, 14 Mar 2024 16:04:06 +0800 Subject: [PATCH 05/10] [fix](jdbc catalog) fix jdbc-connector coredump as get env return nullptr (#32217) --- be/src/util/jni-util.cpp | 4 ++-- be/src/util/jni-util.h | 11 +++++++++-- .../aggregate_function_java_udaf.h | 2 +- be/src/vec/exec/vjdbc_connector.cpp | 2 +- 4 files changed, 13 insertions(+), 6 deletions(-) diff --git a/be/src/util/jni-util.cpp b/be/src/util/jni-util.cpp index 764fc812cba010..6e66f97ddae6fb 100644 --- a/be/src/util/jni-util.cpp +++ b/be/src/util/jni-util.cpp @@ -141,7 +141,7 @@ const std::string GetDorisJNIClasspathOption() { jvm_options[i] = {const_cast(options[i].c_str()), nullptr}; } - JNIEnv* env; + JNIEnv* env = nullptr; JavaVMInitArgs vm_args; vm_args.version = JNI_VERSION_1_8; vm_args.options = jvm_options.get(); @@ -407,7 +407,7 @@ Status JniUtil::Init() { RETURN_IF_ERROR(LibJVMLoader::instance().load()); // Get the JNIEnv* corresponding to current thread. - JNIEnv* env; + JNIEnv* env = nullptr; RETURN_IF_ERROR(JniUtil::GetJNIEnv(&env)); if (env == NULL) return Status::InternalError("Failed to get/create JVM"); diff --git a/be/src/util/jni-util.h b/be/src/util/jni-util.h index 317ba48d3d19e7..ca305c32bf1416 100644 --- a/be/src/util/jni-util.h +++ b/be/src/util/jni-util.h @@ -52,9 +52,16 @@ class JniUtil { static Status GetJNIEnv(JNIEnv** env) { if (tls_env_) { *env = tls_env_; - return Status::OK(); + } else { + Status status = GetJNIEnvSlowPath(env); + if (!status.ok()) { + return status; + } } - return GetJNIEnvSlowPath(env); + if (*env == nullptr) { + return Status::RuntimeError("Failed to get JNIEnv: it is nullptr."); + } + return Status::OK(); } static Status GetGlobalClassRef(JNIEnv* env, const char* class_str, diff --git a/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h b/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h index 4ef64aae558c3b..7a367623eaf197 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h +++ b/be/src/vec/aggregate_functions/aggregate_function_java_udaf.h @@ -60,7 +60,7 @@ struct AggregateJavaUdafData { AggregateJavaUdafData(int64_t num_args) { argument_size = num_args; } ~AggregateJavaUdafData() { - JNIEnv* env; + JNIEnv* env = nullptr; if (!JniUtil::GetJNIEnv(&env).ok()) { LOG(WARNING) << "Failed to get JNIEnv"; } diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp index 41acaf00fb88d3..6a5aa0fdc38769 100644 --- a/be/src/vec/exec/vjdbc_connector.cpp +++ b/be/src/vec/exec/vjdbc_connector.cpp @@ -83,7 +83,7 @@ Status JdbcConnector::close(Status /*unused*/) { if (_is_in_transaction) { RETURN_IF_ERROR(abort_trans()); } - JNIEnv* env; + JNIEnv* env = nullptr; RETURN_IF_ERROR(JniUtil::GetJNIEnv(&env)); env->DeleteGlobalRef(_executor_factory_clazz); env->DeleteGlobalRef(_executor_clazz); From 2815f977d3bc5d3ccbac68da308e61c6ea83c886 Mon Sep 17 00:00:00 2001 From: HappenLee Date: Thu, 14 Mar 2024 17:59:15 +0800 Subject: [PATCH 06/10] [exec](runtimefilter) support null aware in runtime filter (#32152) null aware in runtime filter --- be/src/exprs/bloom_filter_func.h | 5 +-- be/src/exprs/hybrid_set.h | 30 +++++++++++-- be/src/exprs/runtime_filter.cpp | 45 ++++++++++++++----- be/src/pipeline/exec/scan_operator.cpp | 2 +- be/src/runtime/types.cpp | 4 +- be/src/vec/exec/scan/vscan_node.cpp | 2 +- be/src/vec/exprs/vdirect_in_predicate.h | 9 ++-- be/src/vec/functions/in.h | 16 +++---- .../plans/physical/AbstractPhysicalPlan.java | 6 +-- gensrc/thrift/Exprs.thrift | 3 ++ 10 files changed, 81 insertions(+), 41 deletions(-) diff --git a/be/src/exprs/bloom_filter_func.h b/be/src/exprs/bloom_filter_func.h index f5e822e2572f91..397f86a3693e65 100644 --- a/be/src/exprs/bloom_filter_func.h +++ b/be/src/exprs/bloom_filter_func.h @@ -185,10 +185,7 @@ class BloomFilterFuncBase : public RuntimeFilterFuncBase { return _bloom_filter->contain_null(); } - void set_contain_null() { - DCHECK(_bloom_filter); - _bloom_filter->set_contain_null(); - } + void set_contain_null() { _bloom_filter->set_contain_null(); } size_t get_size() const { return _bloom_filter ? _bloom_filter->size() : 0; } diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h index 96e0c3f879af08..ba5fabe509be62 100644 --- a/be/src/exprs/hybrid_set.h +++ b/be/src/exprs/hybrid_set.h @@ -192,6 +192,7 @@ class HybridSetBase : public RuntimeFilterFuncBase { insert(value); iter->next(); } + _contains_null |= set->_contains_null; } virtual int size() = 0; @@ -231,6 +232,9 @@ class HybridSetBase : public RuntimeFilterFuncBase { }; virtual IteratorBase* begin() = 0; + + bool contain_null() const { return _contains_null && _null_aware; } + bool _contains_null = false; }; template @@ -268,10 +272,12 @@ class HybridSet : public HybridSetBase { void insert(const void* data) override { if (data == nullptr) { + _contains_null = true; return; } _set.insert(*reinterpret_cast(data)); } + void insert(void* data, size_t /*unused*/) override { insert(data); } void insert_fixed_len(const vectorized::ColumnPtr& column, size_t start) override { @@ -288,6 +294,8 @@ class HybridSet : public HybridSetBase { for (size_t i = start; i < size; i++) { if (!nullmap[i]) { _set.insert(*(data + i)); + } else { + _contains_null = true; } } } else { @@ -392,6 +400,7 @@ class StringSet : public HybridSetBase { void insert(const void* data) override { if (data == nullptr) { + _contains_null = true; return; } @@ -401,8 +410,12 @@ class StringSet : public HybridSetBase { } void insert(void* data, size_t size) override { - std::string str_value(reinterpret_cast(data), size); - _set.insert(str_value); + if (data == nullptr) { + insert(nullptr); + } else { + std::string str_value(reinterpret_cast(data), size); + _set.insert(str_value); + } } void insert_fixed_len(const vectorized::ColumnPtr& column, size_t start) override { @@ -417,6 +430,8 @@ class StringSet : public HybridSetBase { for (size_t i = start; i < nullable->size(); i++) { if (!nullmap[i]) { _set.insert(col.get_data_at(i).to_string()); + } else { + _contains_null = true; } } } else { @@ -534,6 +549,7 @@ class StringValueSet : public HybridSetBase { void insert(const void* data) override { if (data == nullptr) { + _contains_null = true; return; } @@ -543,8 +559,12 @@ class StringValueSet : public HybridSetBase { } void insert(void* data, size_t size) override { - StringRef sv(reinterpret_cast(data), size); - _set.insert(sv); + if (data == nullptr) { + insert(nullptr); + } else { + StringRef sv(reinterpret_cast(data), size); + _set.insert(sv); + } } void insert_fixed_len(const vectorized::ColumnPtr& column, size_t start) override { @@ -559,6 +579,8 @@ class StringValueSet : public HybridSetBase { for (size_t i = start; i < nullable->size(); i++) { if (!nullmap[i]) { _set.insert(col.get_data_at(i)); + } else { + _contains_null = true; } } } else { diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index e8e169e8b9e5c4..963bcae7b0bc30 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -280,15 +280,16 @@ class RuntimePredicateWrapper { public: RuntimePredicateWrapper(ObjectPool* pool, const RuntimeFilterParams* params) : RuntimePredicateWrapper(pool, params->column_return_type, params->filter_type, - params->filter_id) {}; + params->filter_id, params->build_bf_exactly) {}; // for a 'tmp' runtime predicate wrapper // only could called assign method or as a param for merge RuntimePredicateWrapper(ObjectPool* pool, PrimitiveType column_type, RuntimeFilterType type, - uint32_t filter_id) + uint32_t filter_id, bool build_bf_exactly = false) : _pool(pool), _column_return_type(column_type), _filter_type(type), - _filter_id(filter_id) {} + _filter_id(filter_id), + _build_bf_exactly(build_bf_exactly) {} // init runtime filter wrapper // alloc memory to init runtime filter function @@ -297,6 +298,7 @@ class RuntimePredicateWrapper { switch (_filter_type) { case RuntimeFilterType::IN_FILTER: { _context.hybrid_set.reset(create_set(_column_return_type)); + _context.hybrid_set->set_null_aware(params->null_aware); break; } case RuntimeFilterType::MIN_FILTER: @@ -315,6 +317,7 @@ class RuntimePredicateWrapper { } case RuntimeFilterType::IN_OR_BLOOM_FILTER: { _context.hybrid_set.reset(create_set(_column_return_type)); + _context.hybrid_set->set_null_aware(params->null_aware); _context.bloom_filter_func.reset(create_bloom_filter(_column_return_type)); _context.bloom_filter_func->set_length(params->bloom_filter_size); _context.bloom_filter_func->set_build_bf_exactly(params->build_bf_exactly); @@ -362,6 +365,9 @@ class RuntimePredicateWrapper { it->next(); } } + if (_context.hybrid_set->contain_null()) { + bloom_filter->set_contain_null(); + } } BloomFilterFuncBase* get_bloomfilter() const { return _context.bloom_filter_func.get(); } @@ -428,7 +434,7 @@ class RuntimePredicateWrapper { _context.bitmap_filter_func->insert_many(bitmaps); } - RuntimeFilterType get_real_type() { + RuntimeFilterType get_real_type() const { auto real_filter_type = _filter_type; if (real_filter_type == RuntimeFilterType::IN_OR_BLOOM_FILTER) { real_filter_type = _is_bloomfilter ? RuntimeFilterType::BLOOM_FILTER @@ -437,7 +443,7 @@ class RuntimePredicateWrapper { return real_filter_type; } - size_t get_bloom_filter_size() { + size_t get_bloom_filter_size() const { if (_is_bloomfilter) { return _context.bloom_filter_func->get_size(); } @@ -516,7 +522,7 @@ class RuntimePredicateWrapper { } else { VLOG_DEBUG << " change runtime filter to bloom filter(id=" << _filter_id << ") because: already exist a bloom filter"; - RETURN_IF_ERROR(change_to_bloom_filter()); + RETURN_IF_ERROR(change_to_bloom_filter(!_build_bf_exactly)); RETURN_IF_ERROR(_context.bloom_filter_func->merge( wrapper->_context.bloom_filter_func.get())); } @@ -541,7 +547,7 @@ class RuntimePredicateWrapper { return Status::OK(); } - Status assign(const PInFilter* in_filter) { + Status assign(const PInFilter* in_filter, bool contain_null) { if (in_filter->has_ignored_msg()) { VLOG_DEBUG << "Ignore in filter(id=" << _filter_id << ") because: " << in_filter->ignored_msg(); @@ -552,6 +558,11 @@ class RuntimePredicateWrapper { PrimitiveType type = to_primitive_type(in_filter->column_type()); _context.hybrid_set.reset(create_set(type)); + if (contain_null) { + _context.hybrid_set->set_null_aware(true); + _context.hybrid_set->insert((const void*)nullptr); + } + switch (type) { case TYPE_BOOLEAN: { batch_assign(in_filter, [](std::shared_ptr& set, PColumnValue& column, @@ -882,7 +893,14 @@ class RuntimePredicateWrapper { bool is_bloomfilter() const { return _is_bloomfilter; } bool contain_null() const { - return _is_bloomfilter && _context.bloom_filter_func->contain_null(); + if (_is_bloomfilter) { + return _context.bloom_filter_func->contain_null(); + } + if (_context.hybrid_set) { + DCHECK(get_real_type() == RuntimeFilterType::IN_FILTER); + return _context.hybrid_set->contain_null(); + } + return false; } bool is_ignored() const { return _ignored; } @@ -931,6 +949,7 @@ class RuntimePredicateWrapper { bool _ignored = false; std::string _ignored_msg; uint32_t _filter_id; + bool _build_bf_exactly; }; Status IRuntimeFilter::create(RuntimeFilterParamsContext* state, ObjectPool* pool, @@ -1315,7 +1334,7 @@ Status IRuntimeFilter::create_wrapper(const UpdateRuntimeFilterParamsV2* param, switch (filter_type) { case PFilterType::IN_FILTER: { DCHECK(param->request->has_in_filter()); - return (*wrapper)->assign(¶m->request->in_filter()); + return (*wrapper)->assign(¶m->request->in_filter(), param->request->contain_null()); } case PFilterType::BLOOM_FILTER: { DCHECK(param->request->has_bloom_filter()); @@ -1358,7 +1377,7 @@ Status IRuntimeFilter::_create_wrapper(const T* param, ObjectPool* pool, switch (filter_type) { case PFilterType::IN_FILTER: { DCHECK(param->request->has_in_filter()); - return (*wrapper)->assign(¶m->request->in_filter()); + return (*wrapper)->assign(¶m->request->in_filter(), param->request->contain_null()); } case PFilterType::BLOOM_FILTER: { DCHECK(param->request->has_bloom_filter()); @@ -1742,17 +1761,19 @@ Status RuntimePredicateWrapper::get_push_exprs( << " _filter_type: " << IRuntimeFilter::to_string(_filter_type); auto real_filter_type = get_real_type(); + bool null_aware = contain_null(); switch (real_filter_type) { case RuntimeFilterType::IN_FILTER: { TTypeDesc type_desc = create_type_desc(PrimitiveType::TYPE_BOOLEAN); type_desc.__set_is_nullable(false); TExprNode node; node.__set_type(type_desc); - node.__set_node_type(TExprNodeType::IN_PRED); + node.__set_node_type(null_aware ? TExprNodeType::NULL_AWARE_IN_PRED + : TExprNodeType::IN_PRED); node.in_predicate.__set_is_not_in(false); node.__set_opcode(TExprOpcode::FILTER_IN); node.__set_is_nullable(false); - auto in_pred = vectorized::VDirectInPredicate::create_shared(node); + auto in_pred = vectorized::VDirectInPredicate::create_shared(node, null_aware); in_pred->set_filter(_context.hybrid_set); in_pred->add_child(probe_ctx->root()); auto wrapper = vectorized::VRuntimeFilterWrapper::create_shared(node, in_pred); diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index 987034a76fb4dd..c10e7777bdb649 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -804,7 +804,7 @@ Status ScanLocalState::_normalize_not_in_and_not_eq_predicate( HybridSetBase::IteratorBase* iter = state->hybrid_set->begin(); auto fn_name = std::string(""); - if (!is_fixed_range && state->null_in_set) { + if (!is_fixed_range && state->hybrid_set->contain_null()) { _eos = true; _scan_dependency->set_ready(); } diff --git a/be/src/runtime/types.cpp b/be/src/runtime/types.cpp index 10a6b47f84c690..14ba4b2cebdece 100644 --- a/be/src/runtime/types.cpp +++ b/be/src/runtime/types.cpp @@ -66,7 +66,7 @@ TypeDescriptor::TypeDescriptor(const std::vector& types, int* idx) DCHECK_LT(*idx, types.size() - 1); type = TYPE_ARRAY; contains_nulls.reserve(1); - // here should compatible with fe 1.2, because use contains_null in contains_nulls + // here should compatible with fe 1.2, because use contain_null in contains_nulls if (node.__isset.contains_nulls) { DCHECK_EQ(node.contains_nulls.size(), 1); contains_nulls.push_back(node.contains_nulls[0]); @@ -94,7 +94,7 @@ TypeDescriptor::TypeDescriptor(const std::vector& types, int* idx) break; } case TTypeNodeType::MAP: { - //TODO(xy): handle contains_null[0] for key and [1] for value + //TODO(xy): handle contain_null[0] for key and [1] for value DCHECK(!node.__isset.scalar_type); DCHECK_LT(*idx, types.size() - 2); DCHECK_EQ(node.contains_nulls.size(), 2); diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp index 0f83fac59f60cc..372b8e6bca6a7a 100644 --- a/be/src/vec/exec/scan/vscan_node.cpp +++ b/be/src/vec/exec/scan/vscan_node.cpp @@ -864,7 +864,7 @@ Status VScanNode::_normalize_not_in_and_not_eq_predicate(VExpr* expr, VExprConte HybridSetBase::IteratorBase* iter = state->hybrid_set->begin(); auto fn_name = std::string(""); - if (!is_fixed_range && state->null_in_set) { + if (!is_fixed_range && state->hybrid_set->contain_null()) { _eos = true; } while (iter->has_next()) { diff --git a/be/src/vec/exprs/vdirect_in_predicate.h b/be/src/vec/exprs/vdirect_in_predicate.h index fbba76de61ddeb..9b3d861b3b9b05 100644 --- a/be/src/vec/exprs/vdirect_in_predicate.h +++ b/be/src/vec/exprs/vdirect_in_predicate.h @@ -26,8 +26,11 @@ class VDirectInPredicate final : public VExpr { ENABLE_FACTORY_CREATOR(VDirectInPredicate); public: - VDirectInPredicate(const TExprNode& node) - : VExpr(node), _filter(nullptr), _expr_name("direct_in_predicate") {} + VDirectInPredicate(const TExprNode& node, bool null_aware = false) + : VExpr(node), + _filter(nullptr), + _expr_name("direct_in_predicate"), + _null_aware(null_aware) {} ~VDirectInPredicate() override = default; Status prepare(RuntimeState* state, const RowDescriptor& row_desc, @@ -93,7 +96,7 @@ class VDirectInPredicate final : public VExpr { private: std::shared_ptr _filter; - bool _null_aware = false; std::string _expr_name; + bool _null_aware; }; } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/functions/in.h b/be/src/vec/functions/in.h index 9fa182caf0309e..1bfbf7eb2d5b38 100644 --- a/be/src/vec/functions/in.h +++ b/be/src/vec/functions/in.h @@ -59,9 +59,6 @@ namespace doris::vectorized { struct InState { bool use_set = true; - - // only use in null in set - bool null_in_set = false; std::unique_ptr hybrid_set; }; @@ -125,17 +122,16 @@ class FunctionIn : public IFunction { state->hybrid_set.reset( create_set(context->get_arg_type(0)->type, get_size_with_out_null(context))); } + state->hybrid_set->set_null_aware(true); + for (int i = 1; i < context->get_num_args(); ++i) { const auto& const_column_ptr = context->get_constant_col(i); if (const_column_ptr != nullptr) { auto const_data = const_column_ptr->column_ptr->get_data_at(0); - if (const_data.data == nullptr) { - state->null_in_set = true; - } else { - state->hybrid_set->insert((void*)const_data.data, const_data.size); - } + state->hybrid_set->insert((void*)const_data.data, const_data.size); } else { state->use_set = false; + state->hybrid_set.reset(); break; } } @@ -181,7 +177,7 @@ class FunctionIn : public IFunction { nested_col_ptr); } - if (!in_state->null_in_set) { + if (!in_state->hybrid_set->contain_null()) { for (size_t i = 0; i < input_rows_count; ++i) { vec_null_map_to[i] = null_map[i]; } @@ -200,7 +196,7 @@ class FunctionIn : public IFunction { search_hash_set(in_state, input_rows_count, vec_res, materialized_column.get()); } - if (in_state->null_in_set) { + if (in_state->hybrid_set->contain_null()) { for (size_t i = 0; i < input_rows_count; ++i) { vec_null_map_to[i] = negative == vec_res[i]; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java index 9f71bda0b4061f..03dc70e653bf3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java @@ -135,10 +135,8 @@ public boolean pushDownRuntimeFilter(CascadesContext context, IdGenerator Date: Thu, 14 Mar 2024 18:30:18 +0800 Subject: [PATCH 07/10] [feature](hudi) support hudi incremental read (#32052) * [feature](hudi) support incremental read for hudi table * fix jdk17 java options --- conf/be.conf | 2 +- docs/en/docs/lakehouse/multi-catalog/hudi.md | 33 ++- .../docs/lakehouse/multi-catalog/hudi.md | 37 ++- .../org/apache/doris/hudi/HudiJniScanner.java | 12 +- .../apache/doris/hudi/BaseSplitReader.scala | 8 +- .../doris/hudi/HoodieRecordIterator.scala | 67 +++-- .../hudi/MORIncrementalSplitReader.scala | 86 ++++++ .../doris/hudi/MORSnapshotSplitReader.scala | 2 +- .../org/apache/doris/nereids/DorisParser.g4 | 6 +- fe/fe-core/src/main/cup/sql_parser.cup | 16 +- .../org/apache/doris/analysis/TableRef.java | 14 + .../doris/analysis/TableScanParams.java | 46 ++++ .../datasource/hive/HMSExternalTable.java | 100 ++++++- .../hudi/source/COWIncrementalRelation.java | 254 ++++++++++++++++++ .../datasource/hudi/source/HudiScanNode.java | 175 ++++++++---- .../hudi/source/IncrementalRelation.java | 44 +++ .../hudi/source/MORIncrementalRelation.java | 217 +++++++++++++++ .../nereids/analyzer/UnboundRelation.java | 34 ++- .../nereids/parser/LogicalPlanBuilder.java | 9 +- .../nereids/rules/analysis/BindRelation.java | 8 +- .../nereids/rules/analysis/CheckPolicy.java | 25 +- .../external_table_p2/hive/test_hive_hudi.out | 12 + .../hive/test_hive_hudi.groovy | 6 + 23 files changed, 1095 insertions(+), 118 deletions(-) create mode 100644 fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/MORIncrementalSplitReader.scala create mode 100644 fe/fe-core/src/main/java/org/apache/doris/analysis/TableScanParams.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/IncrementalRelation.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/MORIncrementalRelation.java diff --git a/conf/be.conf b/conf/be.conf index 13ad50ba60ab76..e23e68e42a0228 100644 --- a/conf/be.conf +++ b/conf/be.conf @@ -25,7 +25,7 @@ JAVA_OPTS="-Xmx1024m -DlogPath=$DORIS_HOME/log/jni.log -Xloggc:$DORIS_HOME/log/b JAVA_OPTS_FOR_JDK_9="-Xmx1024m -DlogPath=$DORIS_HOME/log/jni.log -Xlog:gc:$DORIS_HOME/log/be.gc.log.$CUR_DATE -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives" # For jdk 17+, this JAVA_OPTS will be used as default JVM options -JAVA_OPTS_FOR_JDK_17="-Xmx1024m -DlogPath=$DORIS_HOME/log/jni.log -Xlog:gc:$DORIS_HOME/log/be.gc.log.$CUR_DATE -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED" +JAVA_OPTS_FOR_JDK_17="-Xmx1024m -DlogPath=$DORIS_HOME/log/jni.log -Xlog:gc:$DORIS_HOME/log/be.gc.log.$CUR_DATE -Djavax.security.auth.useSubjectCredsOnly=false -Dsun.security.krb5.debug=true -Dsun.java.command=DorisBE -XX:-CriticalJNINatives -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED --add-opens=java.management/sun.management=ALL-UNNAMED" # since 1.2, the JAVA_HOME need to be set to run BE process. # JAVA_HOME=/path/to/jdk/ diff --git a/docs/en/docs/lakehouse/multi-catalog/hudi.md b/docs/en/docs/lakehouse/multi-catalog/hudi.md index a52c2370ced161..3a5420319b05e2 100644 --- a/docs/en/docs/lakehouse/multi-catalog/hudi.md +++ b/docs/en/docs/lakehouse/multi-catalog/hudi.md @@ -33,8 +33,8 @@ under the License. | Table Type | Supported Query types | | ---- | ---- | -| Copy On Write | Snapshot Query + Time Travel | -| Merge On Read | Snapshot Queries + Read Optimized Queries + Time Travel | +| Copy On Write | Snapshot Query, Time Travel, Icremental Read | +| Merge On Read | Snapshot Queries, Read Optimized Queries, Time Travel, Icremental Read | 2. Doris supports Hive Metastore(Including catalogs compatible with Hive MetaStore, like [AWS Glue](./hive.md)/[Alibaba DLF](./dlf.md)) Catalogs. @@ -94,16 +94,29 @@ Users can view the perfomace of Java SDK through [profile](../../admin-manual/ht ## Time Travel -Supports reading snapshots specified in Hudi table. - -Every write operation to the Hudi table will generate a new snapshot. - -By default, query requests will only read the latest version of the snapshot. +Every write operation to the Hudi table will generate a new snapshot. [Time Travel](https://hudi.apache.org/docs/0.14.0/quick-start-guide/#timetravel) supports reading snapshots specified in Hudi table. By default, query requests will only read the latest version of the snapshot. You can use the `FOR TIME AS OF` statement, based on the time of the snapshot to read historical version data. Examples are as follows: +``` +SELECT * FROM hudi_tbl FOR TIME AS OF "2022-10-07 17:20:37"; +SELECT * FROM hudi_tbl FOR TIME AS OF "20221007172037"; +SELECT * FROM hudi_tbl FOR TIME AS OF "2022-10-07"; +``` +Hudi table does not support the `FOR VERSION AS OF` statement. Using this syntax to query the Hudi table will throw an error. -`SELECT * FROM hudi_tbl FOR TIME AS OF "2022-10-07 17:20:37";` +## Incremental Read +Incremental Read obtains a set of records that changed between a start and end commit time, providing you with the "latest state" for each such record as of the end commit time. -`SELECT * FROM hudi_tbl FOR TIME AS OF "20221007172037";` +Doris uses `@incr` syntax to support Incremental Read: +``` +SELECT * from hudi_table@incr('beginTime'='xxx', ['endTime'='xxx'], ['hoodie.read.timeline.holes.resolution.policy'='FAIL'], ...); +``` +`beginTime` is required, the time format is consistent with [hudi_table_changes](https://hudi.apache.org/docs/0.14.0/quick-start-guide/#incremental-query), and also supports "earliest". `endTime` is optional, default to latest commit time. The remaining optional parameters can be [Spark Read Options](https://hudi.apache.org/docs/0.14.0/configurations#Read-Options). -Hudi table does not support the `FOR VERSION AS OF` statement. Using this syntax to query the Hudi table will throw an error. +Incremental Read should turn on Nereids Planner. Doris translates `@incr` as `predicates` and pushdown to `VHUDI_SCAN_NODE`: +``` +| 0:VHUDI_SCAN_NODE(113) | +| table: lineitem_mor | +| predicates: (_hoodie_commit_time[#0] >= '20240311151019723'), (_hoodie_commit_time[#0] <= '20240311151606605') | +| inputSplitNum=1, totalFileSize=13099711, scanRanges=1 | +``` diff --git a/docs/zh-CN/docs/lakehouse/multi-catalog/hudi.md b/docs/zh-CN/docs/lakehouse/multi-catalog/hudi.md index 38bb26d3bc79ed..b7f2776e38d303 100644 --- a/docs/zh-CN/docs/lakehouse/multi-catalog/hudi.md +++ b/docs/zh-CN/docs/lakehouse/multi-catalog/hudi.md @@ -29,12 +29,12 @@ under the License. ## 使用限制 -1. Hudi 表支持的查询类型如下,后续将支持 Incremental Query。 +1. Hudi 表支持的查询类型如下,后续将支持 CDC。 | 表类型 | 支持的查询类型 | | ---- | ---- | -| Copy On Write | Snapshot Query + Time Travel | -| Merge On Read | Snapshot Queries + Read Optimized Queries + Time Travel | +| Copy On Write | Snapshot Query, Time Travel, Icremental Read | +| Merge On Read | Snapshot Queries, Read Optimized Queries, Time Travel, Icremental Read | 2. 目前支持 Hive Metastore 和兼容 Hive Metastore 类型(例如[AWS Glue](./hive.md)/[Alibaba DLF](./dlf.md))的 Catalog。 @@ -96,16 +96,29 @@ Doris 使用 parquet native reader 读取 COW 表的数据文件,使用 Java S ## Time Travel -支持读取 Hudi 表指定的 Snapshot。 +每一次对 Hudi 表的写操作都会产生一个新的快照,Time Travel 支持读取 Hudi 表指定的 Snapshot。默认情况下,查询请求只会读取最新版本的快照。 -每一次对 Hudi 表的写操作都会产生一个新的快照。 - -默认情况下,查询请求只会读取最新版本的快照。 - -可以使用 `FOR TIME AS OF` 语句,根据快照的时间([时间格式](https://hudi.apache.org/docs/quick-start-guide#time-travel-query)和Hudi官网保持一致)读取历史版本的数据。示例如下: +可以使用 `FOR TIME AS OF` 语句,根据快照的时间([时间格式](https://hudi.apache.org/docs/0.14.0/quick-start-guide/#timetravel)和Hudi官网保持一致)读取历史版本的数据。示例如下: +``` +SELECT * FROM hudi_tbl FOR TIME AS OF "2022-10-07 17:20:37"; +SELECT * FROM hudi_tbl FOR TIME AS OF "20221007172037"; +SELECT * FROM hudi_tbl FOR TIME AS OF "2022-10-07"; +``` +Hudi 表不支持 `FOR VERSION AS OF` 语句,使用该语法查询 Hudi 表将抛错。 -`SELECT * FROM hudi_tbl FOR TIME AS OF "2022-10-07 17:20:37";` +## Incremental Read +Incremental Read 可以查询在 startTime 和 endTime 之间变化的数据,返回的结果集是数据在 endTime 的最终状态。 -`SELECT * FROM hudi_tbl FOR TIME AS OF "20221007172037";` +Doris 提供了 `@incr` 语法支持 Incremental Read: +``` +SELECT * from hudi_table@incr('beginTime'='xxx', ['endTime'='xxx'], ['hoodie.read.timeline.holes.resolution.policy'='FAIL'], ...); +``` +`beginTime` 是必须的,时间格式和 hudi 官网 [hudi_table_changes](https://hudi.apache.org/docs/0.14.0/quick-start-guide/#incremental-query) 保持一致,支持 "earliest"。`endTime` 选填,默认最新commitTime。兼容 [Spark Read Options](https://hudi.apache.org/docs/0.14.0/configurations#Read-Options)。 -Hudi 表不支持 `FOR VERSION AS OF` 语句,使用该语法查询 Hudi 表将抛错。 +支持 Incremental Read 需要开启[新优化器](../../query-acceleration/nereids.md),新优化器默认打开。通过 `desc` 查看执行计划,可以发现 Doris 将 `@incr` 转化为 `predicates` 下推给 `VHUDI_SCAN_NODE`: +``` +| 0:VHUDI_SCAN_NODE(113) | +| table: lineitem_mor | +| predicates: (_hoodie_commit_time[#0] >= '20240311151019723'), (_hoodie_commit_time[#0] <= '20240311151606605') | +| inputSplitNum=1, totalFileSize=13099711, scanRanges=1 | +``` diff --git a/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/HudiJniScanner.java b/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/HudiJniScanner.java index 932b53b9a7cf07..8da006d66bd197 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/HudiJniScanner.java +++ b/fe/be-java-extensions/hudi-scanner/src/main/java/org/apache/doris/hudi/HudiJniScanner.java @@ -174,9 +174,15 @@ public void open() throws IOException { cleanResolverLock.readLock().lock(); try { lastUpdateTime.set(System.currentTimeMillis()); - recordIterator = HadoopUGI.ugiDoAs( - AuthenticationConfig.getKerberosConfig(split.hadoopConf()), () -> new MORSnapshotSplitReader( - split).buildScanIterator(new Filter[0])); + if (split.incrementalRead()) { + recordIterator = HadoopUGI.ugiDoAs(AuthenticationConfig.getKerberosConfig( + split.hadoopConf()), + () -> new MORIncrementalSplitReader(split).buildScanIterator(new Filter[0])); + } else { + recordIterator = HadoopUGI.ugiDoAs(AuthenticationConfig.getKerberosConfig( + split.hadoopConf()), + () -> new MORSnapshotSplitReader(split).buildScanIterator(new Filter[0])); + } if (AVRO_RESOLVER_CACHE != null && AVRO_RESOLVER_CACHE.get() != null) { cachedResolvers.computeIfAbsent(Thread.currentThread().getId(), threadId -> AVRO_RESOLVER_CACHE.get()); diff --git a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/BaseSplitReader.scala b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/BaseSplitReader.scala index a730f2cd1b298f..8229064163d7ca 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/BaseSplitReader.scala +++ b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/BaseSplitReader.scala @@ -125,6 +125,10 @@ class HoodieSplit(private val params: jutil.Map[String, String]) { conf } + def incrementalRead: Boolean = { + "true".equalsIgnoreCase(optParams.getOrElse("hoodie.datasource.read.incr.operation", "false")) + } + // NOTE: In cases when Hive Metastore is used as catalog and the table is partitioned, schema in the HMS might contain // Hive-specific partitioning columns created specifically for HMS to handle partitioning appropriately. In that // case we opt in to not be providing catalog's schema, and instead force Hudi relations to fetch the schema @@ -169,6 +173,8 @@ abstract class BaseSplitReader(val split: HoodieSplit) { protected val tableInformation: HoodieTableInformation = cache.get(split) + protected val timeline: HoodieTimeline = tableInformation.timeline + protected val sparkSession: SparkSession = tableInformation.sparkSession protected val sqlContext: SQLContext = sparkSession.sqlContext imbueConfigs(sqlContext) @@ -578,8 +584,6 @@ abstract class BaseSplitReader(val split: HoodieSplit) { ) } - protected val timeline: HoodieTimeline = tableInformation.timeline - protected def embedInternalSchema(conf: Configuration, internalSchemaOpt: Option[InternalSchema]): Configuration = { val internalSchema = internalSchemaOpt.getOrElse(InternalSchema.getEmptyInternalSchema) val querySchemaString = SerDeHelper.toJson(internalSchema) diff --git a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/HoodieRecordIterator.scala b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/HoodieRecordIterator.scala index 6e2b7b31e547bc..f393e9e12469c7 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/HoodieRecordIterator.scala +++ b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/HoodieRecordIterator.scala @@ -20,11 +20,14 @@ package org.apache.doris.hudi import org.apache.hadoop.conf.Configuration import org.apache.hudi.HoodieBaseRelation.{BaseFileReader, projectReader} import org.apache.hudi.MergeOnReadSnapshotRelation.isProjectionCompatible +import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.util.StringUtils import org.apache.hudi.{DataSourceReadOptions, HoodieMergeOnReadFileSplit, HoodieTableSchema, HoodieTableState, LogFileIterator, RecordMergingFileIterator} import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection import org.apache.spark.sql.catalyst.InternalRow import java.io.Closeable +import java.util.function.Predicate /** * Class holding base-file readers for 3 different use-cases: @@ -84,29 +87,61 @@ class HoodieMORRecordIterator(config: Configuration, requiredSchema: HoodieTableSchema, tableState: HoodieTableState, mergeType: String, - fileSplit: HoodieMergeOnReadFileSplit) extends Iterator[InternalRow] with Closeable { + fileSplit: HoodieMergeOnReadFileSplit, + includeStartTime: Boolean = false, + startTimestamp: String = null, + endTimestamp: String = null) extends Iterator[InternalRow] with Closeable { protected val maxCompactionMemoryInBytes: Long = config.getLongBytes( "hoodie.compaction.memory", 512 * 1024 * 1024) - protected val recordIterator: Iterator[InternalRow] = fileSplit match { - case dataFileOnlySplit if dataFileOnlySplit.logFiles.isEmpty => - val projectedReader = projectReader(fileReaders.requiredSchemaReaderSkipMerging, requiredSchema.structTypeSchema) - projectedReader(dataFileOnlySplit.dataFile.get) + protected val recordIterator: Iterator[InternalRow] = { + val iter = fileSplit match { + case dataFileOnlySplit if dataFileOnlySplit.logFiles.isEmpty => + val projectedReader = projectReader(fileReaders.requiredSchemaReaderSkipMerging, requiredSchema.structTypeSchema) + projectedReader(dataFileOnlySplit.dataFile.get) - case logFileOnlySplit if logFileOnlySplit.dataFile.isEmpty => - new LogFileIterator(logFileOnlySplit, tableSchema, requiredSchema, tableState, config) + case logFileOnlySplit if logFileOnlySplit.dataFile.isEmpty => + new LogFileIterator(logFileOnlySplit, tableSchema, requiredSchema, tableState, config) - case split => mergeType match { - case DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL => - // val reader = fileReaders.requiredSchemaReaderSkipMerging - // new SkipMergeIterator(split, reader, tableSchema, requiredSchema, tableState, config) - throw new UnsupportedOperationException("Skip merge is optimized by native read") + case split => mergeType match { + case DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL => + val reader = fileReaders.requiredSchemaReaderSkipMerging + new SkipMergeIterator(split, reader, tableSchema, requiredSchema, tableState, config) - case DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL => - val reader = pickBaseFileReader() - new RecordMergingFileIterator(split, reader, tableSchema, requiredSchema, tableState, config) + case DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL => + val reader = pickBaseFileReader() + new RecordMergingFileIterator(split, reader, tableSchema, requiredSchema, tableState, config) - case _ => throw new UnsupportedOperationException(s"Not supported merge type ($mergeType)") + case _ => throw new UnsupportedOperationException(s"Not supported merge type ($mergeType)") + } + } + + val commitTimeMetadataFieldIdx = requiredSchema.structTypeSchema.fieldNames.indexOf(HoodieRecord.COMMIT_TIME_METADATA_FIELD) + val needsFiltering = commitTimeMetadataFieldIdx >= 0 && !StringUtils.isNullOrEmpty(startTimestamp) && !StringUtils.isNullOrEmpty(endTimestamp) + if (needsFiltering) { + val filterT: Predicate[InternalRow] = getCommitTimeFilter(includeStartTime, commitTimeMetadataFieldIdx) + iter.filter(filterT.test) + } + else { + iter + } + } + + private def getCommitTimeFilter(includeStartTime: Boolean, commitTimeMetadataFieldIdx: Int): Predicate[InternalRow] = { + if (includeStartTime) { + new Predicate[InternalRow] { + override def test(row: InternalRow): Boolean = { + val commitTime = row.getString(commitTimeMetadataFieldIdx) + commitTime >= startTimestamp && commitTime <= endTimestamp + } + } + } else { + new Predicate[InternalRow] { + override def test(row: InternalRow): Boolean = { + val commitTime = row.getString(commitTimeMetadataFieldIdx) + commitTime > startTimestamp && commitTime <= endTimestamp + } + } } } diff --git a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/MORIncrementalSplitReader.scala b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/MORIncrementalSplitReader.scala new file mode 100644 index 00000000000000..73c87e29034086 --- /dev/null +++ b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/MORIncrementalSplitReader.scala @@ -0,0 +1,86 @@ +// 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. + +package org.apache.doris.hudi + +import org.apache.hudi.HoodieTableSchema +import org.apache.hudi.common.model.HoodieRecord +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources._ + +/** + * Reference to Apache Hudi + * see MergeOnReadIncrementalRelation + */ +class MORIncrementalSplitReader(override val split: HoodieSplit) extends MORSnapshotSplitReader(split) with IncrementalSplitReaderTrait { + + override protected def composeIterator(tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + requestedColumns: Array[String], + filters: Array[Filter]): Iterator[InternalRow] = { + // The only required filters are ones that make sure we're only fetching records that + // fall into incremental span of the timeline being queried + val requiredFilters = incrementalSpanRecordFilters + val optionalFilters = filters + val readers = createBaseFileReaders(tableSchema, requiredSchema, requestedColumns, requiredFilters, optionalFilters) + + new HoodieMORRecordIterator(split.hadoopConf, + readers, + tableSchema, + requiredSchema, + tableState, + mergeType, + getFileSplit(), + includeStartTime = includeStartTime, + startTimestamp = startTs, + endTimestamp = endTs) + } + +} + +/** + * Reference to Apache Hudi + * see HoodieIncrementalRelationTrait + */ +trait IncrementalSplitReaderTrait extends BaseSplitReader { + protected val includeStartTime: Boolean = "true".equalsIgnoreCase(optParams("hoodie.datasource.read.incr.includeStartTime")) + protected val startTs: String = optParams("hoodie.datasource.read.begin.instanttime") + protected val endTs: String = optParams("hoodie.datasource.read.end.instanttime") + + // Record filters making sure that only records w/in the requested bounds are being fetched as part of the + // scan collected by this relation + protected lazy val incrementalSpanRecordFilters: Seq[Filter] = { + val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD) + + val largerThanFilter = if (includeStartTime) { + GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, startTs) + } else { + GreaterThan(HoodieRecord.COMMIT_TIME_METADATA_FIELD, startTs) + } + + val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, endTs) + + Seq(isNotNullFilter, largerThanFilter, lessThanFilter) + } + + override lazy val mandatoryFields: Seq[String] = { + // NOTE: This columns are required for Incremental flow to be able to handle the rows properly, even in + // cases when no columns are requested to be fetched (for ex, when using {@code count()} API) + Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++ + preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) + } +} diff --git a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/MORSnapshotSplitReader.scala b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/MORSnapshotSplitReader.scala index e9958b231e7a1a..07e236082ce99a 100644 --- a/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/MORSnapshotSplitReader.scala +++ b/fe/be-java-extensions/hudi-scanner/src/main/scala/org/apache/doris/hudi/MORSnapshotSplitReader.scala @@ -74,7 +74,7 @@ class MORSnapshotSplitReader(override val split: HoodieSplit) extends BaseSplitR getFileSplit()) } - private def getFileSplit(): HoodieMergeOnReadFileSplit = { + protected def getFileSplit(): HoodieMergeOnReadFileSplit = { val logFiles = split.deltaFilePaths.map(new HoodieLogFile(_)) .sorted(Ordering.comparatorToOrdering(HoodieLogFile.getLogFileComparator)).toList val partitionedBaseFile = if (split.dataFilePath.isEmpty) { diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index e0a306196713e2..ebb5984e862840 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -477,8 +477,12 @@ identifierSeq : ident+=errorCapturingIdentifier (COMMA ident+=errorCapturingIdentifier)* ; +optScanParams + : ATSIGN funcName=identifier LEFT_PAREN (properties=propertyItemList)? RIGHT_PAREN + ; + relationPrimary - : multipartIdentifier materializedViewName? specifiedPartition? + : multipartIdentifier optScanParams? materializedViewName? specifiedPartition? tabletList? tableAlias sample? relationHint? lateralView* #tableName | LEFT_PAREN query RIGHT_PAREN tableAlias lateralView* #aliasedQuery | tvfName=identifier LEFT_PAREN diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index 0770ac73da7675..22dd017b68a41e 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -841,6 +841,7 @@ nonterminal String opt_job_starts; nonterminal String opt_job_ends; nonterminal String job_at_time; nonterminal ColocateGroupName colocate_group_name; +nonterminal TableScanParams opt_scan_params_ref; nonterminal LoadTask.MergeType opt_merge_type, opt_with_merge_type; @@ -5808,6 +5809,17 @@ colocate_group_name ::= :} ; +opt_scan_params_ref ::= + /* empty */ + {: + RESULT = null; + :} + | AT ident:func_name LPAREN opt_key_value_map_in_paren:properties RPAREN + {: + RESULT = new TableScanParams(func_name, properties); + :} + ; + encryptkey_name ::= ident:name {: @@ -5928,9 +5940,9 @@ base_table_ref_list ::= ; base_table_ref ::= - table_name:name opt_table_snapshot:tableSnapshot opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints + table_name:name opt_scan_params_ref:scanParams opt_table_snapshot:tableSnapshot opt_partition_names:partitionNames opt_tablet_list:tabletIds opt_table_alias:alias opt_table_sample:tableSample opt_common_hints:commonHints {: - RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints, tableSnapshot); + RESULT = new TableRef(name, alias, partitionNames, tabletIds, tableSample, commonHints, tableSnapshot, scanParams); :} ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java index 0ade6730c0d115..13821a510c6811 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TableRef.java @@ -140,6 +140,8 @@ public class TableRef implements ParseNode, Writable { private TableSnapshot tableSnapshot; + private TableScanParams scanParams; + // END: Members that need to be reset() // /////////////////////////////////////// @@ -169,6 +171,12 @@ public TableRef(TableName name, String alias, PartitionNames partitionNames, Arr public TableRef(TableName name, String alias, PartitionNames partitionNames, ArrayList sampleTabletIds, TableSample tableSample, ArrayList commonHints, TableSnapshot tableSnapshot) { + this(name, alias, partitionNames, sampleTabletIds, tableSample, commonHints, tableSnapshot, null); + } + + public TableRef(TableName name, String alias, PartitionNames partitionNames, + ArrayList sampleTabletIds, TableSample tableSample, ArrayList commonHints, + TableSnapshot tableSnapshot, TableScanParams scanParams) { this.name = name; if (alias != null) { if (Env.isStoredTableNamesLowerCase()) { @@ -184,6 +192,7 @@ public TableRef(TableName name, String alias, PartitionNames partitionNames, Arr this.tableSample = tableSample; this.commonHints = commonHints; this.tableSnapshot = tableSnapshot; + this.scanParams = scanParams; isAnalyzed = false; } @@ -204,6 +213,7 @@ protected TableRef(TableRef other) { onClause = (other.onClause != null) ? other.onClause.clone().reset() : null; partitionNames = (other.partitionNames != null) ? new PartitionNames(other.partitionNames) : null; tableSnapshot = (other.tableSnapshot != null) ? new TableSnapshot(other.tableSnapshot) : null; + scanParams = other.scanParams; tableSample = (other.tableSample != null) ? new TableSample(other.tableSample) : null; commonHints = other.commonHints; @@ -333,6 +343,10 @@ public Boolean haveDesc() { return desc != null; } + public TableScanParams getScanParams() { + return scanParams; + } + /** * This method should only be called after the TableRef has been analyzed. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/TableScanParams.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/TableScanParams.java new file mode 100644 index 00000000000000..ab1491ccc3ac80 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TableScanParams.java @@ -0,0 +1,46 @@ +// 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. + +package org.apache.doris.analysis; + +import com.google.common.collect.ImmutableMap; + +import java.util.Map; + +public class TableScanParams { + public static String INCREMENTAL_READ = "incr"; + + private final String paramType; + private final Map params; + + public TableScanParams(String paramType, Map params) { + this.paramType = paramType; + this.params = params == null ? ImmutableMap.of() : ImmutableMap.copyOf(params); + } + + public String getParamType() { + return paramType; + } + + public Map getParams() { + return params; + } + + public boolean incrementalRead() { + return INCREMENTAL_READ.equals(paramType); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index 50eac67deff9f3..a788d9e57bca5a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -17,6 +17,7 @@ package org.apache.doris.datasource.hive; +import org.apache.doris.analysis.TableScanParams; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ListPartitionItem; @@ -28,12 +29,23 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.hudi.HudiUtils; +import org.apache.doris.datasource.hudi.source.COWIncrementalRelation; +import org.apache.doris.datasource.hudi.source.IncrementalRelation; +import org.apache.doris.datasource.hudi.source.MORIncrementalRelation; import org.apache.doris.datasource.iceberg.IcebergUtils; import org.apache.doris.mtmv.MTMVMaxTimestampSnapshot; import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVTimestampSnapshot; import org.apache.doris.nereids.exceptions.NotSupportedException; +import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.GreaterThan; +import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; +import org.apache.doris.nereids.trees.expressions.LessThanEqual; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; import org.apache.doris.statistics.AnalysisInfo; import org.apache.doris.statistics.BaseAnalysisTask; import org.apache.doris.statistics.ColumnStatistic; @@ -45,6 +57,7 @@ import org.apache.doris.thrift.TTableDescriptor; import org.apache.doris.thrift.TTableType; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -61,6 +74,7 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -68,6 +82,7 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.time.LocalDate; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -147,6 +162,10 @@ public class HMSExternalTable extends ExternalTable implements MTMVRelatedTableI // record the event update time when enable hms event listener protected volatile long eventUpdateTime; + // for hudi incremental read + private TableScanParams scanParams = null; + private IncrementalRelation incrementalRelation = null; + public enum DLAType { UNKNOWN, HIVE, HUDI, ICEBERG } @@ -224,7 +243,10 @@ public boolean isHoodieCowTable() { return false; } String inputFormatName = remoteTable.getSd().getInputFormat(); - return "org.apache.hudi.hadoop.HoodieParquetInputFormat".equals(inputFormatName); + Map params = remoteTable.getParameters(); + return "org.apache.hudi.hadoop.HoodieParquetInputFormat".equals(inputFormatName) + || "skip_merge".equals(getCatalogProperties().get("hoodie.datasource.merge.type")) + || (params != null && "COPY_ON_WRITE".equalsIgnoreCase(params.get("flink.table.type"))); } /** @@ -281,6 +303,82 @@ public List getPartitionColumns() { return partitionColumns; } + public TableScanParams getScanParams() { + return scanParams; + } + + public void setScanParams(TableScanParams scanParams) { + if (scanParams != null && scanParams.incrementalRead()) { + Map optParams = getHadoopProperties(); + if (scanParams.getParams().containsKey("beginTime")) { + optParams.put("hoodie.datasource.read.begin.instanttime", scanParams.getParams().get("beginTime")); + } + if (scanParams.getParams().containsKey("endTime")) { + optParams.put("hoodie.datasource.read.end.instanttime", scanParams.getParams().get("endTime")); + } + scanParams.getParams().forEach((k, v) -> { + if (k.startsWith("hoodie.")) { + optParams.put(k, v); + } + }); + HoodieTableMetaClient hudiClient = HiveMetaStoreClientHelper.getHudiClient(this); + try { + boolean isCowOrRoTable = isHoodieCowTable(); + if (isCowOrRoTable) { + Map serd = remoteTable.getSd().getSerdeInfo().getParameters(); + if ("true".equals(serd.get("hoodie.query.as.ro.table")) + && remoteTable.getTableName().endsWith("_ro")) { + // Incremental read RO table as RT table, I don't know why? + isCowOrRoTable = false; + LOG.warn("Execute incremental read on RO table"); + } + } + if (isCowOrRoTable) { + incrementalRelation = new COWIncrementalRelation( + optParams, HiveMetaStoreClientHelper.getConfiguration(this), hudiClient); + } else { + incrementalRelation = new MORIncrementalRelation( + optParams, HiveMetaStoreClientHelper.getConfiguration(this), hudiClient); + } + } catch (Exception e) { + LOG.warn("Failed to create incremental relation", e); + } + } + this.scanParams = scanParams; + } + + public IncrementalRelation getIncrementalRelation() { + return incrementalRelation; + } + + /** + * replace incremental params as AND expression + * incr('beginTime'='20240308110257169', 'endTime'='20240308110677278') => + * _hoodie_commit_time >= 20240308110257169 and _hoodie_commit_time <= '20240308110677278' + */ + public Set generateIncrementalExpression(List slots) { + if (incrementalRelation == null) { + return Collections.emptySet(); + } + SlotReference timeField = null; + for (Slot slot : slots) { + if ("_hoodie_commit_time".equals(slot.getName())) { + timeField = (SlotReference) slot; + break; + } + } + if (timeField == null) { + return Collections.emptySet(); + } + StringLiteral upperValue = new StringLiteral(incrementalRelation.getEndTs()); + StringLiteral lowerValue = new StringLiteral(incrementalRelation.getStartTs()); + ComparisonPredicate less = new LessThanEqual(timeField, upperValue); + ComparisonPredicate great = incrementalRelation.isIncludeStartTime() + ? new GreaterThanEqual(timeField, lowerValue) + : new GreaterThan(timeField, lowerValue); + return ImmutableSet.of(great, less); + } + public boolean isHiveTransactionalTable() { return dlaType == DLAType.HIVE && AcidUtils.isTransactionalTable(remoteTable) && isSupportedTransactionalFileFormat(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java new file mode 100644 index 00000000000000..fa24dc53e56781 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/COWIncrementalRelation.java @@ -0,0 +1,254 @@ +// 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. + +package org.apache.doris.datasource.hudi.source; + +import org.apache.doris.datasource.FileSplit; +import org.apache.doris.spi.Split; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.GlobPattern; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class COWIncrementalRelation implements IncrementalRelation { + private final Map optParams; + private final HoodieTableMetaClient metaClient; + private final HollowCommitHandling hollowCommitHandling; + private final boolean startInstantArchived; + private final boolean endInstantArchived; + private final boolean fullTableScan; + private final FileSystem fs; + private final Map fileToWriteStat; + private final Collection filteredRegularFullPaths; + private final Collection filteredMetaBootstrapFullPaths; + + private final boolean includeStartTime; + private final String startTs; + private final String endTs; + + public COWIncrementalRelation(Map optParams, Configuration configuration, + HoodieTableMetaClient metaClient) + throws HoodieException, IOException { + this.optParams = optParams; + this.metaClient = metaClient; + hollowCommitHandling = HollowCommitHandling.valueOf( + optParams.getOrDefault("hoodie.read.timeline.holes.resolution.policy", "FAIL")); + HoodieTimeline commitTimeline = TimelineUtils.handleHollowCommitIfNeeded( + metaClient.getCommitTimeline().filterCompletedInstants(), metaClient, hollowCommitHandling); + if (commitTimeline.empty()) { + throw new HoodieException("No instants to incrementally pull"); + } + if (!metaClient.getTableConfig().populateMetaFields()) { + throw new HoodieException("Incremental queries are not supported when meta fields are disabled"); + } + HoodieInstant lastInstant = commitTimeline.lastInstant().get(); + String startInstantTime = optParams.get("hoodie.datasource.read.begin.instanttime"); + if (startInstantTime == null) { + throw new HoodieException("Specify the begin instant time to pull from using " + + "option hoodie.datasource.read.begin.instanttime"); + } + if (EARLIEST_TIME.equals(startInstantTime)) { + startInstantTime = "000"; + } + String endInstantTime = optParams.getOrDefault("hoodie.datasource.read.end.instanttime", + lastInstant.getTimestamp()); + startInstantArchived = commitTimeline.isBeforeTimelineStarts(startInstantTime); + endInstantArchived = commitTimeline.isBeforeTimelineStarts(endInstantTime); + + HoodieTimeline commitsTimelineToReturn; + if (hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME) { + commitsTimelineToReturn = commitTimeline.findInstantsInRangeByStateTransitionTime(startInstantTime, + lastInstant.getStateTransitionTime()); + } else { + commitsTimelineToReturn = commitTimeline.findInstantsInRange(startInstantTime, lastInstant.getTimestamp()); + } + List commitsToReturn = commitsTimelineToReturn.getInstants(); + + // todo: support configuration hoodie.datasource.read.incr.filters + Path basePath = metaClient.getBasePathV2(); + Map regularFileIdToFullPath = new HashMap<>(); + Map metaBootstrapFileIdToFullPath = new HashMap<>(); + HoodieTimeline replacedTimeline = commitsTimelineToReturn.getCompletedReplaceTimeline(); + Map replacedFile = new HashMap<>(); + for (HoodieInstant instant : replacedTimeline.getInstants()) { + HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), + HoodieReplaceCommitMetadata.class).getPartitionToReplaceFileIds().forEach( + (key, value) -> value.forEach( + e -> replacedFile.put(e, FSUtils.getPartitionPath(basePath, key).toString()))); + } + + fileToWriteStat = new HashMap<>(); + for (HoodieInstant commit : commitsToReturn) { + HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes( + commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); + metadata.getPartitionToWriteStats().forEach((partition, stats) -> { + for (HoodieWriteStat stat : stats) { + fileToWriteStat.put(FSUtils.getPartitionPath(basePath, stat.getPath()).toString(), stat); + } + }); + if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS.equals(commit.getTimestamp())) { + metadata.getFileIdAndFullPaths(basePath).forEach((k, v) -> { + if (!(replacedFile.containsKey(k) && v.startsWith(replacedFile.get(k)))) { + metaBootstrapFileIdToFullPath.put(k, v); + } + }); + } else { + metadata.getFileIdAndFullPaths(basePath).forEach((k, v) -> { + if (!(replacedFile.containsKey(k) && v.startsWith(replacedFile.get(k)))) { + regularFileIdToFullPath.put(k, v); + } + }); + } + } + + if (!metaBootstrapFileIdToFullPath.isEmpty()) { + // filer out meta bootstrap files that have had more commits since metadata bootstrap + metaBootstrapFileIdToFullPath.entrySet().removeIf(e -> regularFileIdToFullPath.containsKey(e.getKey())); + } + String pathGlobPattern = optParams.getOrDefault("hoodie.datasource.read.incr.path.glob", ""); + if ("".equals(pathGlobPattern)) { + filteredRegularFullPaths = regularFileIdToFullPath.values(); + filteredMetaBootstrapFullPaths = metaBootstrapFileIdToFullPath.values(); + } else { + GlobPattern globMatcher = new GlobPattern("*" + pathGlobPattern); + filteredRegularFullPaths = regularFileIdToFullPath.values().stream().filter(globMatcher::matches) + .collect(Collectors.toList()); + filteredMetaBootstrapFullPaths = metaBootstrapFileIdToFullPath.values().stream() + .filter(globMatcher::matches).collect(Collectors.toList()); + + } + + fs = basePath.getFileSystem(configuration); + fullTableScan = shouldFullTableScan(); + includeStartTime = !fullTableScan; + if (fullTableScan || commitsToReturn.isEmpty()) { + startTs = startInstantTime; + endTs = endInstantTime; + } else { + startTs = commitsToReturn.get(0).getTimestamp(); + endTs = commitsToReturn.get(commitsToReturn.size() - 1).getTimestamp(); + } + } + + private boolean shouldFullTableScan() throws HoodieException, IOException { + boolean fallbackToFullTableScan = Boolean.parseBoolean( + optParams.getOrDefault("hoodie.datasource.read.incr.fallback.fulltablescan.enable", "false")); + if (fallbackToFullTableScan && (startInstantArchived || endInstantArchived)) { + if (hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME) { + throw new HoodieException("Cannot use stateTransitionTime while enables full table scan"); + } + return true; + } + if (fallbackToFullTableScan) { + for (String path : filteredMetaBootstrapFullPaths) { + if (!fs.exists(new Path(path))) { + return true; + } + } + for (String path : filteredRegularFullPaths) { + if (!fs.exists(new Path(path))) { + return true; + } + } + } + return false; + } + + @Override + public List collectFileSlices() throws HoodieException { + throw new UnsupportedOperationException(); + } + + @Override + public List collectSplits() throws HoodieException { + if (fullTableScan) { + throw new HoodieException("Fallback to full table scan"); + } + if (filteredRegularFullPaths.isEmpty() && filteredMetaBootstrapFullPaths.isEmpty()) { + return Collections.emptyList(); + } + List splits = new ArrayList<>(); + Option partitionColumns = metaClient.getTableConfig().getPartitionFields(); + List partitionNames = partitionColumns.isPresent() ? Arrays.asList(partitionColumns.get()) + : Collections.emptyList(); + for (String baseFile : filteredMetaBootstrapFullPaths) { + HoodieWriteStat stat = fileToWriteStat.get(baseFile); + splits.add(new FileSplit(new Path(baseFile), 0, stat.getFileSizeInBytes(), stat.getFileSizeInBytes(), + new String[0], + HudiPartitionProcessor.parsePartitionValues(partitionNames, stat.getPartitionPath()))); + } + for (String baseFile : filteredRegularFullPaths) { + HoodieWriteStat stat = fileToWriteStat.get(baseFile); + splits.add(new FileSplit(new Path(baseFile), 0, stat.getFileSizeInBytes(), stat.getFileSizeInBytes(), + new String[0], + HudiPartitionProcessor.parsePartitionValues(partitionNames, stat.getPartitionPath()))); + } + return splits; + } + + @Override + public Map getHoodieParams() { + optParams.put("hoodie.datasource.read.incr.operation", "true"); + optParams.put("hoodie.datasource.read.begin.instanttime", startTs); + optParams.put("hoodie.datasource.read.end.instanttime", endTs); + optParams.put("hoodie.datasource.read.incr.includeStartTime", includeStartTime ? "true" : "false"); + return optParams; + } + + @Override + public boolean fallbackFullTableScan() { + return fullTableScan; + } + + @Override + public boolean isIncludeStartTime() { + return includeStartTime; + } + + @Override + public String getStartTs() { + return startTs; + } + + @Override + public String getEndTs() { + return endTs; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java index dfbb12e8584703..58068c575d7365 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/HudiScanNode.java @@ -17,6 +17,7 @@ package org.apache.doris.datasource.hudi.source; +import org.apache.doris.analysis.TableScanParams; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.PartitionItem; @@ -50,6 +51,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -63,6 +65,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -78,12 +81,22 @@ public class HudiScanNode extends HiveScanNode { private static final Logger LOG = LogManager.getLogger(HudiScanNode.class); - private final boolean isCowOrRoTable; + private boolean isCowOrRoTable; private final AtomicLong noLogsSplitNum = new AtomicLong(0); private final boolean useHiveSyncPartition; + private HoodieTableMetaClient hudiClient; + private String basePath; + private String inputFormat; + private String serdeLib; + private List columnNames; + private List columnTypes; + + private boolean incrementalRead = false; + private IncrementalRelation incrementalRelation; + /** * External file scan node for Query Hudi table * needCheckColumnPriv: Some of ExternalFileScanNode do not need to check column priv @@ -92,10 +105,7 @@ public class HudiScanNode extends HiveScanNode { */ public HudiScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv) { super(id, desc, "HUDI_SCAN_NODE", StatisticalType.HUDI_SCAN_NODE, needCheckColumnPriv); - Map paras = hmsTable.getRemoteTable().getParameters(); - isCowOrRoTable = hmsTable.isHoodieCowTable() - || "skip_merge".equals(hmsTable.getCatalogProperties().get("hoodie.datasource.merge.type")) - || (paras != null && "COPY_ON_WRITE".equalsIgnoreCase(paras.get("flink.table.type"))); + isCowOrRoTable = hmsTable.isHoodieCowTable(); if (isCowOrRoTable) { if (LOG.isDebugEnabled()) { LOG.debug("Hudi table {} can read as cow/read optimize table", hmsTable.getName()); @@ -129,12 +139,66 @@ protected void doInitialize() throws UserException { computeColumnsFilter(); initBackendPolicy(); initSchemaParams(); + + hudiClient = HiveMetaStoreClientHelper.getHudiClient(hmsTable); + hudiClient.reloadActiveTimeline(); + basePath = hmsTable.getRemoteTable().getSd().getLocation(); + inputFormat = hmsTable.getRemoteTable().getSd().getInputFormat(); + serdeLib = hmsTable.getRemoteTable().getSd().getSerdeInfo().getSerializationLib(); + columnNames = new ArrayList<>(); + columnTypes = new ArrayList<>(); + TableSchemaResolver schemaUtil = new TableSchemaResolver(hudiClient); + Schema hudiSchema; + try { + hudiSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchema()); + } catch (Exception e) { + throw new UserException("Cannot get hudi table schema."); + } + for (Schema.Field hudiField : hudiSchema.getFields()) { + columnNames.add(hudiField.name().toLowerCase(Locale.ROOT)); + String columnType = HudiUtils.fromAvroHudiTypeToHiveTypeString(hudiField.schema()); + columnTypes.add(columnType); + } + + TableScanParams scanParams = desc.getRef().getScanParams(); + if (scanParams != null) { + throw new UserException("Incremental read should turn on nereids planner"); + } + scanParams = hmsTable.getScanParams(); + if (scanParams != null) { + if (scanParams.incrementalRead()) { + incrementalRead = true; + } else { + throw new UserException("Not support function '" + scanParams.getParamType() + "' in hudi table"); + } + } + if (incrementalRead) { + if (isCowOrRoTable) { + try { + Map serd = hmsTable.getRemoteTable().getSd().getSerdeInfo().getParameters(); + if ("true".equals(serd.get("hoodie.query.as.ro.table")) + && hmsTable.getRemoteTable().getTableName().endsWith("_ro")) { + // Incremental read RO table as RT table, I don't know why? + isCowOrRoTable = false; + LOG.warn("Execute incremental read on RO table"); + } + } catch (Exception e) { + // ignore + } + } + incrementalRelation = hmsTable.getIncrementalRelation(); + if (incrementalRelation == null) { + throw new UserException("Failed to create incremental relation"); + } + } else { + incrementalRelation = null; + } } @Override protected Map getLocationProperties() throws UserException { - if (isCowOrRoTable) { - return super.getLocationProperties(); + if (incrementalRead) { + return incrementalRelation.getHoodieParams(); } else { // HudiJniScanner uses hadoop client to read data. return hmsTable.getHadoopProperties(); @@ -176,7 +240,7 @@ private List getPrunedPartitions( TablePartitionValues partitionValues; if (snapshotTimestamp.isPresent()) { partitionValues = processor.getSnapshotPartitionValues( - hmsTable, metaClient, snapshotTimestamp.get(), useHiveSyncPartition); + hmsTable, metaClient, snapshotTimestamp.get(), useHiveSyncPartition); } else { partitionValues = processor.getPartitionValues(hmsTable, metaClient, useHiveSyncPartition); } @@ -222,28 +286,24 @@ private List getPrunedPartitions( return Lists.newArrayList(dummyPartition); } - @Override - public List getSplits() throws UserException { - HoodieTableMetaClient hudiClient = HiveMetaStoreClientHelper.getHudiClient(hmsTable); - hudiClient.reloadActiveTimeline(); - String basePath = hmsTable.getRemoteTable().getSd().getLocation(); - String inputFormat = hmsTable.getRemoteTable().getSd().getInputFormat(); - String serdeLib = hmsTable.getRemoteTable().getSd().getSerdeInfo().getSerializationLib(); - - TableSchemaResolver schemaUtil = new TableSchemaResolver(hudiClient); - Schema hudiSchema; - try { - hudiSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchema()); - } catch (Exception e) { - throw new RuntimeException("Cannot get hudi table schema."); + private List getIncrementalSplits() { + if (isCowOrRoTable) { + List splits = incrementalRelation.collectSplits(); + noLogsSplitNum.addAndGet(splits.size()); + return splits; } + Option partitionColumns = hudiClient.getTableConfig().getPartitionFields(); + List partitionNames = partitionColumns.isPresent() ? Arrays.asList(partitionColumns.get()) + : Collections.emptyList(); + return incrementalRelation.collectFileSlices().stream().map(fileSlice -> generateHudiSplit(fileSlice, + HudiPartitionProcessor.parsePartitionValues(partitionNames, fileSlice.getPartitionPath()), + incrementalRelation.getEndTs())).collect(Collectors.toList()); + } - List columnNames = new ArrayList<>(); - List columnTypes = new ArrayList<>(); - for (Schema.Field hudiField : hudiSchema.getFields()) { - columnNames.add(hudiField.name().toLowerCase(Locale.ROOT)); - String columnType = HudiUtils.fromAvroHudiTypeToHiveTypeString(hudiField.schema()); - columnTypes.add(columnType); + @Override + public List getSplits() throws UserException { + if (incrementalRead && !incrementalRelation.fallbackFullTableScan()) { + return getIncrementalSplits(); } HoodieTimeline timeline = hudiClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); @@ -300,33 +360,9 @@ public List getSplits() throws UserException { new String[0], partition.getPartitionValues())); }); } else { - fileSystemView.getLatestMergedFileSlicesBeforeOrOn(partitionName, queryInstant).forEach(fileSlice -> { - Optional baseFile = fileSlice.getBaseFile().toJavaOptional(); - String filePath = baseFile.map(BaseFile::getPath).orElse(""); - long fileSize = baseFile.map(BaseFile::getFileSize).orElse(0L); - - List logs = fileSlice.getLogFiles().map(HoodieLogFile::getPath) - .map(Path::toString) - .collect(Collectors.toList()); - if (logs.isEmpty()) { - noLogsSplitNum.incrementAndGet(); - } - - // no base file, use log file to parse file type - String agencyPath = filePath.isEmpty() ? logs.get(0) : filePath; - HudiSplit split = new HudiSplit(new Path(agencyPath), 0, fileSize, fileSize, - new String[0], partition.getPartitionValues()); - split.setTableFormatType(TableFormatType.HUDI); - split.setDataFilePath(filePath); - split.setHudiDeltaLogs(logs); - split.setInputFormat(inputFormat); - split.setSerde(serdeLib); - split.setBasePath(basePath); - split.setHudiColumnNames(columnNames); - split.setHudiColumnTypes(columnTypes); - split.setInstantTime(queryInstant); - splits.add(split); - }); + fileSystemView.getLatestMergedFileSlicesBeforeOrOn(partitionName, queryInstant) + .forEach(fileSlice -> splits.add( + generateHudiSplit(fileSlice, partition.getPartitionValues(), queryInstant))); } countDownLatch.countDown(); })); @@ -338,6 +374,35 @@ public List getSplits() throws UserException { return splits; } + private HudiSplit generateHudiSplit(FileSlice fileSlice, List partitionValues, String queryInstant) { + Optional baseFile = fileSlice.getBaseFile().toJavaOptional(); + String filePath = baseFile.map(BaseFile::getPath).orElse(""); + long fileSize = baseFile.map(BaseFile::getFileSize).orElse(0L); + fileSlice.getPartitionPath(); + + List logs = fileSlice.getLogFiles().map(HoodieLogFile::getPath) + .map(Path::toString) + .collect(Collectors.toList()); + if (logs.isEmpty()) { + noLogsSplitNum.incrementAndGet(); + } + + // no base file, use log file to parse file type + String agencyPath = filePath.isEmpty() ? logs.get(0) : filePath; + HudiSplit split = new HudiSplit(new Path(agencyPath), 0, fileSize, fileSize, + new String[0], partitionValues); + split.setTableFormatType(TableFormatType.HUDI); + split.setDataFilePath(filePath); + split.setHudiDeltaLogs(logs); + split.setInputFormat(inputFormat); + split.setSerde(serdeLib); + split.setBasePath(basePath); + split.setHudiColumnNames(columnNames); + split.setHudiColumnTypes(columnTypes); + split.setInstantTime(queryInstant); + return split; + } + @Override public String getNodeExplainString(String prefix, TExplainLevel detailLevel) { return super.getNodeExplainString(prefix, detailLevel) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/IncrementalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/IncrementalRelation.java new file mode 100644 index 00000000000000..4a707064fb6f2f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/IncrementalRelation.java @@ -0,0 +1,44 @@ +// 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. + +package org.apache.doris.datasource.hudi.source; + +import org.apache.doris.spi.Split; + +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.exception.HoodieException; + +import java.util.List; +import java.util.Map; + +public interface IncrementalRelation { + public static String EARLIEST_TIME = "earliest"; + + List collectFileSlices() throws HoodieException; + + List collectSplits() throws HoodieException; + + Map getHoodieParams(); + + boolean fallbackFullTableScan(); + + boolean isIncludeStartTime(); + + String getStartTs(); + + String getEndTs(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/MORIncrementalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/MORIncrementalRelation.java new file mode 100644 index 00000000000000..c06fcc2a578d43 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hudi/source/MORIncrementalRelation.java @@ -0,0 +1,217 @@ +// 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. + +package org.apache.doris.datasource.hudi.source; + +import org.apache.doris.spi.Split; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.GlobPattern; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class MORIncrementalRelation implements IncrementalRelation { + private final Map optParams; + private final HoodieTableMetaClient metaClient; + private final HoodieTimeline timeline; + private final HollowCommitHandling hollowCommitHandling; + private String startTimestamp; + private final String endTimestamp; + private final boolean startInstantArchived; + private final boolean endInstantArchived; + private final List includedCommits; + private final List commitsMetadata; + private final FileStatus[] affectedFilesInCommits; + private final boolean fullTableScan; + private final String globPattern; + private final boolean includeStartTime; + private final String startTs; + private final String endTs; + + + public MORIncrementalRelation(Map optParams, Configuration configuration, + HoodieTableMetaClient metaClient) + throws HoodieException, IOException { + this.optParams = optParams; + this.metaClient = metaClient; + timeline = metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); + if (timeline.empty()) { + throw new HoodieException("No instants to incrementally pull"); + } + if (!metaClient.getTableConfig().populateMetaFields()) { + throw new HoodieException("Incremental queries are not supported when meta fields are disabled"); + } + hollowCommitHandling = HollowCommitHandling.valueOf( + optParams.getOrDefault("hoodie.read.timeline.holes.resolution.policy", "FAIL")); + + startTimestamp = optParams.get("hoodie.datasource.read.begin.instanttime"); + if (startTimestamp == null) { + throw new HoodieException("Specify the begin instant time to pull from using " + + "option hoodie.datasource.read.begin.instanttime"); + } + if (EARLIEST_TIME.equals(startTimestamp)) { + startTimestamp = "000"; + } + endTimestamp = optParams.getOrDefault("hoodie.datasource.read.end.instanttime", + hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME + ? timeline.lastInstant().get().getStateTransitionTime() + : timeline.lastInstant().get().getTimestamp()); + + startInstantArchived = timeline.isBeforeTimelineStarts(startTimestamp); + endInstantArchived = timeline.isBeforeTimelineStarts(endTimestamp); + + includedCommits = getIncludedCommits(); + commitsMetadata = getCommitsMetadata(); + affectedFilesInCommits = HoodieInputFormatUtils.listAffectedFilesForCommits(configuration, + new Path(metaClient.getBasePath()), commitsMetadata); + fullTableScan = shouldFullTableScan(); + if (hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME && fullTableScan) { + throw new HoodieException("Cannot use stateTransitionTime while enables full table scan"); + } + globPattern = optParams.getOrDefault("hoodie.datasource.read.incr.path.glob", ""); + + if (startInstantArchived) { + includeStartTime = false; + startTs = startTimestamp; + } else { + includeStartTime = true; + startTs = includedCommits.isEmpty() ? startTimestamp : includedCommits.get(0).getTimestamp(); + } + endTs = endInstantArchived || includedCommits.isEmpty() ? endTimestamp + : includedCommits.get(includedCommits.size() - 1).getTimestamp(); + } + + @Override + public Map getHoodieParams() { + optParams.put("hoodie.datasource.read.incr.operation", "true"); + optParams.put("hoodie.datasource.read.begin.instanttime", startTs); + optParams.put("hoodie.datasource.read.end.instanttime", endTs); + optParams.put("hoodie.datasource.read.incr.includeStartTime", includeStartTime ? "true" : "false"); + return optParams; + } + + private List getIncludedCommits() { + if (!startInstantArchived || !endInstantArchived) { + // If endTimestamp commit is not archived, will filter instants + // before endTimestamp. + if (hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME) { + return timeline.findInstantsInRangeByStateTransitionTime(startTimestamp, endTimestamp).getInstants(); + } else { + return timeline.findInstantsInRange(startTimestamp, endTimestamp).getInstants(); + } + } else { + return timeline.getInstants(); + } + } + + private List getCommitsMetadata() throws IOException { + List result = new ArrayList<>(); + for (HoodieInstant commit : includedCommits) { + result.add(TimelineUtils.getCommitMetadata(commit, timeline)); + } + return result; + } + + private boolean shouldFullTableScan() throws IOException { + boolean should = Boolean.parseBoolean( + optParams.getOrDefault("hoodie.datasource.read.incr.fallback.fulltablescan.enable", "false")) && ( + startInstantArchived || endInstantArchived); + if (should) { + return true; + } + for (FileStatus fileStatus : affectedFilesInCommits) { + if (!metaClient.getFs().exists(fileStatus.getPath())) { + return true; + } + } + return false; + } + + @Override + public boolean fallbackFullTableScan() { + return fullTableScan; + } + + @Override + public boolean isIncludeStartTime() { + return includeStartTime; + } + + @Override + public String getStartTs() { + return startTs; + } + + @Override + public String getEndTs() { + return endTs; + } + + @Override + public List collectFileSlices() throws HoodieException { + if (includedCommits.isEmpty()) { + return Collections.emptyList(); + } else if (fullTableScan) { + throw new HoodieException("Fallback to full table scan"); + } + HoodieTimeline scanTimeline; + if (hollowCommitHandling == HollowCommitHandling.USE_TRANSITION_TIME) { + scanTimeline = metaClient.getCommitsAndCompactionTimeline() + .findInstantsInRangeByStateTransitionTime(startTimestamp, endTimestamp); + } else { + scanTimeline = TimelineUtils.handleHollowCommitIfNeeded( + metaClient.getCommitsAndCompactionTimeline(), metaClient, hollowCommitHandling) + .findInstantsInRange(startTimestamp, endTimestamp); + } + String latestCommit = includedCommits.get(includedCommits.size() - 1).getTimestamp(); + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, scanTimeline, + affectedFilesInCommits); + Stream fileSlices = HoodieInputFormatUtils.getWritePartitionPaths(commitsMetadata) + .stream().flatMap(relativePartitionPath -> + fsView.getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, latestCommit)); + if ("".equals(globPattern)) { + return fileSlices.collect(Collectors.toList()); + } + GlobPattern globMatcher = new GlobPattern("*" + globPattern); + return fileSlices.filter(fileSlice -> globMatcher.matches(fileSlice.getBaseFile().map(BaseFile::getPath) + .or(fileSlice.getLatestLogFile().map(f -> f.getPath().toString())).get())).collect(Collectors.toList()); + } + + @Override + public List collectSplits() throws HoodieException { + throw new UnsupportedOperationException(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java index 74f85e316512de..4514ea05bfb9c9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.analyzer; +import org.apache.doris.analysis.TableScanParams; import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; @@ -53,21 +54,36 @@ public class UnboundRelation extends LogicalRelation implements Unbound, BlockFu private final List hints; private final Optional tableSample; private final Optional indexName; + private TableScanParams scanParams; public UnboundRelation(RelationId id, List nameParts) { this(id, nameParts, Optional.empty(), Optional.empty(), ImmutableList.of(), false, ImmutableList.of(), - ImmutableList.of(), Optional.empty(), Optional.empty()); + ImmutableList.of(), Optional.empty(), Optional.empty(), null); } public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart) { this(id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, ImmutableList.of(), - ImmutableList.of(), Optional.empty(), Optional.empty()); + ImmutableList.of(), Optional.empty(), Optional.empty(), null); } public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart, List tabletIds, List hints, Optional tableSample, Optional indexName) { this(id, nameParts, Optional.empty(), Optional.empty(), - partNames, isTempPart, tabletIds, hints, tableSample, indexName); + partNames, isTempPart, tabletIds, hints, tableSample, indexName, null); + } + + public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart, + List tabletIds, List hints, Optional tableSample, Optional indexName, + TableScanParams scanParams) { + this(id, nameParts, Optional.empty(), Optional.empty(), + partNames, isTempPart, tabletIds, hints, tableSample, indexName, scanParams); + } + + public UnboundRelation(RelationId id, List nameParts, Optional groupExpression, + Optional logicalProperties, List partNames, boolean isTempPart, + List tabletIds, List hints, Optional tableSample, Optional indexName) { + this(id, nameParts, groupExpression, logicalProperties, partNames, + isTempPart, tabletIds, hints, tableSample, indexName, null); } /** @@ -75,7 +91,8 @@ public UnboundRelation(RelationId id, List nameParts, List partN */ public UnboundRelation(RelationId id, List nameParts, Optional groupExpression, Optional logicalProperties, List partNames, boolean isTempPart, - List tabletIds, List hints, Optional tableSample, Optional indexName) { + List tabletIds, List hints, Optional tableSample, Optional indexName, + TableScanParams scanParams) { super(id, PlanType.LOGICAL_UNBOUND_RELATION, groupExpression, logicalProperties); this.nameParts = ImmutableList.copyOf(Objects.requireNonNull(nameParts, "nameParts should not null")); this.partNames = ImmutableList.copyOf(Objects.requireNonNull(partNames, "partNames should not null")); @@ -84,6 +101,7 @@ public UnboundRelation(RelationId id, List nameParts, Optional getNameParts() { @@ -104,14 +122,14 @@ public LogicalProperties computeLogicalProperties() { public Plan withGroupExpression(Optional groupExpression) { return new UnboundRelation(relationId, nameParts, groupExpression, Optional.of(getLogicalProperties()), - partNames, isTempPart, tabletIds, hints, tableSample, indexName); + partNames, isTempPart, tabletIds, hints, tableSample, indexName, null); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { return new UnboundRelation(relationId, nameParts, groupExpression, logicalProperties, partNames, - isTempPart, tabletIds, hints, tableSample, indexName); + isTempPart, tabletIds, hints, tableSample, indexName, null); } @Override @@ -165,4 +183,8 @@ public List getHints() { public Optional getTableSample() { return tableSample; } + + public TableScanParams getScanParams() { + return scanParams; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 1e2f8d2f23089b..019e95cbed55a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -21,6 +21,7 @@ import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.StorageBackend; import org.apache.doris.analysis.TableName; +import org.apache.doris.analysis.TableScanParams; import org.apache.doris.analysis.UserIdentity; import org.apache.doris.catalog.AggregateType; import org.apache.doris.catalog.BuiltinAggregateFunctions; @@ -1310,11 +1311,17 @@ public LogicalPlan visitTableName(TableNameContext ctx) { relationHints = ImmutableList.of(); } + TableScanParams scanParams = null; + if (ctx.optScanParams() != null) { + Map map = visitPropertyItemList(ctx.optScanParams().properties); + scanParams = new TableScanParams(ctx.optScanParams().funcName.getText(), map); + } + TableSample tableSample = ctx.sample() == null ? null : (TableSample) visit(ctx.sample()); LogicalPlan checkedRelation = LogicalPlanBuilderAssistant.withCheckPolicy( new UnboundRelation(StatementScopeIdGenerator.newRelationId(), tableId, partitionNames, isTempPart, tabletIdLists, relationHints, - Optional.ofNullable(tableSample), indexName)); + Optional.ofNullable(tableSample), indexName, scanParams)); LogicalPlan plan = withTableAlias(checkedRelation, ctx.tableAlias()); for (LateralViewContext lateralViewContext : ctx.lateralView()) { plan = withGenerate(plan, lateralViewContext); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index d74c487d296ae2..e41c4283d603f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -249,12 +249,14 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio LogicalView logicalView = new LogicalView<>(view, viewBody); return new LogicalSubQueryAlias<>(tableQualifier, logicalView); case HMS_EXTERNAL_TABLE: - if (Config.enable_query_hive_views && ((HMSExternalTable) table).isView()) { - String hiveCatalog = ((HMSExternalTable) table).getCatalog().getName(); - String ddlSql = ((HMSExternalTable) table).getViewText(); + HMSExternalTable hmsTable = (HMSExternalTable) table; + if (Config.enable_query_hive_views && hmsTable.isView()) { + String hiveCatalog = hmsTable.getCatalog().getName(); + String ddlSql = hmsTable.getViewText(); Plan hiveViewPlan = parseAndAnalyzeHiveView(hiveCatalog, ddlSql, cascadesContext); return new LogicalSubQueryAlias<>(tableQualifier, hiveViewPlan); } + hmsTable.setScanParams(unboundRelation.getScanParams()); return new LogicalFileScan(unboundRelation.getRelationId(), (HMSExternalTable) table, tableQualifier, unboundRelation.getTableSample()); case ICEBERG_EXTERNAL_TABLE: diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckPolicy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckPolicy.java index 24f8396cf21c14..aa2917ae1e1d9f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckPolicy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckPolicy.java @@ -17,20 +17,22 @@ package org.apache.doris.nereids.rules.analysis; +import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalCheckPolicy; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; +import java.util.HashSet; import java.util.List; -import java.util.Optional; import java.util.Set; /** @@ -58,11 +60,26 @@ public List buildRules() { return ctx.root.child(); } LogicalRelation relation = (LogicalRelation) child; - Optional filter = checkPolicy.getFilter(relation, ctx.connectContext); - if (!filter.isPresent()) { + Set combineFilter = new HashSet<>(); + + // replace incremental params as AND expression + if (relation instanceof LogicalFileScan) { + LogicalFileScan fileScan = (LogicalFileScan) relation; + if (fileScan.getTable() instanceof HMSExternalTable) { + HMSExternalTable hmsTable = (HMSExternalTable) fileScan.getTable(); + combineFilter.addAll(hmsTable.generateIncrementalExpression( + fileScan.getLogicalProperties().getOutput())); + } + } + + // row policy + checkPolicy.getFilter(relation, ctx.connectContext) + .ifPresent(expression -> combineFilter.addAll( + ExpressionUtils.extractConjunctionToSet(expression))); + + if (combineFilter.isEmpty()) { return ctx.root.child(); } - Set combineFilter = ExpressionUtils.extractConjunctionToSet(filter.get()); if (upperFilter != null) { combineFilter.addAll(upperFilter.getConjuncts()); } diff --git a/regression-test/data/external_table_p2/hive/test_hive_hudi.out b/regression-test/data/external_table_p2/hive/test_hive_hudi.out index 9202749e61745f..d217a289d8d74b 100644 --- a/regression-test/data/external_table_p2/hive/test_hive_hudi.out +++ b/regression-test/data/external_table_p2/hive/test_hive_hudi.out @@ -135,6 +135,18 @@ row_4 2021-02-01 4 v_4 20240221111000868 20240221111000868_0_8 e3cf430c-889d-4015-bc98-59bdce1e530c sao_paulo c97347e9-033a-4c19-a033-94ac1de9f892 1695516137016 e3cf430c-889d-4015-bc98-59bdce1e530c rider-F driver-P 34.15 sao_paulo 20240221111000868 20240221111000868_0_4 e96c4396-3fad-413a-a942-4cb36106d721 san_francisco 3efcaa94-3e58-436a-b489-1232731ed088 1695091554788 e96c4396-3fad-413a-a942-4cb36106d721 rider-C driver-M 27.7 san_francisco +-- !incr_mor_table -- +20240312163737521 20240312163737521_0_1 20240312163737521_1_0 pid=p2 92e677af-6487-4213-b42f-ee56c5a2acdc-0_0-86-568_20240312163737521.parquet 4 k4 4.992 p2 +20240312163737521 20240312163737521_1_0 20240312163737521_0_0 pid=p3 22af1878-d8e0-4829-b4af-c7c9693d33f3-0_1-86-569_20240312163737521.parquet 5 k5 5.5 p3 +20240312163613712 20240312163613712_0_1 20240312163541346_0_0 pid=p1 ead436a2-f99b-4d35-8b3e-9d67e4828dd5-0_0-99-636_20240312163738946.parquet 1 k1 1.134 p1 + +-- !inc_cow_table -- +20240312164834145 20240312164834145_0_0 20240312164538551_2_0 pid=p2 b8c2db38-e6b8-41f4-8132-620106e8c7e9-0_0-222-1414_20240312164938557.parquet 3 k3-3 3.77 p2 +20240312164938557 20240312164938557_0_1 20240312164938557_1_0 pid=p2 b8c2db38-e6b8-41f4-8132-620106e8c7e9-0_0-222-1414_20240312164938557.parquet 4 k4 4.992 p2 +20240312164938557 20240312164938557_1_0 20240312164938557_0_0 pid=p3 1335e747-d611-4575-8612-15e491224a0e-0_1-222-1415_20240312164938557.parquet 5 k5 5.5 p3 +20240312164613448 20240312164613448_0_0 20240312164538551_0_0 pid=p1 372f896b-10cb-4803-b109-d467189326b8-0_0-166-1042_20240312164650751.parquet 1 k1 1.37 p1 +20240312164650751 20240312164650751_0_1 20240312164538551_1_0 pid=p1 372f896b-10cb-4803-b109-d467189326b8-0_0-166-1042_20240312164650751.parquet 2 k2 2.2 p1 + -- !skip_merge -- 20230605145009209 20230605145009209_0_0 rowId:row_1 partitionId=2021-01-01/versionId=v_0 65ffc5d9-397a-456e-a735-30f3ad37466f-0_0-33-96_20230605145009209.parquet row_1 2021-01-01 0 bob v_0 toBeDel0 0 1000000 20230605145403388 20230605145403388_2_0 rowId:row_1 partitionId=2011-11-11/versionId=v_1 dbff8acb-42bc-400c-be33-47d9e0bae9b7-0_2-83-222_20230605145403388.parquet row_1 2011-11-11 1 bob v_1 toBeDel1 0 1000001 diff --git a/regression-test/suites/external_table_p2/hive/test_hive_hudi.groovy b/regression-test/suites/external_table_p2/hive/test_hive_hudi.groovy index d852e604df5eec..75c22b26cd7773 100644 --- a/regression-test/suites/external_table_p2/hive/test_hive_hudi.groovy +++ b/regression-test/suites/external_table_p2/hive/test_hive_hudi.groovy @@ -48,6 +48,12 @@ suite("test_hive_hudi", "p2,external,hive,hudi") { // hudi table created by flink hudi catalog qt_flink_hudi_catalog """select * from hudi_ctl_table order by uuid""" + // incremental read for MOR table + qt_incr_mor_table """select * from incr_mor_partition@incr('beginTime'='20240312163541346')""" + + // incremental read for COW table + qt_inc_cow_table """select * from incr_cow_partition@incr('beginTime'='20240312164538551')""" + // skip logs sql """drop catalog if exists ${catalog_name};""" sql """ From aa53eb3d42e6d1f0dbb1568a269d88faee3b024c Mon Sep 17 00:00:00 2001 From: Tiewei Fang <43782773+BePPPower@users.noreply.github.com> Date: Thu, 14 Mar 2024 18:35:01 +0800 Subject: [PATCH 08/10] [Fix](trino-connector) nereids supports trino-connector (#32166) Issue Number: #31719 1. nereids supports trino-connector catalog 2. Fix some trino-connector's bug when reading data --- .../TrinoConnectorJniScanner.java | 27 +++++++++---------- .../TrinoConnectorExternalTable.java | 2 +- .../source/TrinoConnectorScanNode.java | 15 ++++++++--- .../translator/PhysicalPlanTranslator.java | 4 +++ .../nereids/rules/analysis/BindRelation.java | 1 + 5 files changed, 30 insertions(+), 19 deletions(-) diff --git a/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorJniScanner.java b/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorJniScanner.java index dae1149d6c713b..4a8afaf7b898cd 100644 --- a/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorJniScanner.java +++ b/fe/be-java-extensions/trino-connector-scanner/src/main/java/org/apache/doris/trinoconnector/TrinoConnectorJniScanner.java @@ -58,11 +58,11 @@ import io.trino.spi.connector.ConnectorTransactionHandle; import io.trino.spi.connector.DynamicFilter; import io.trino.spi.security.Identity; +import io.trino.spi.type.TimeZoneKey; import io.trino.spi.type.Type; import io.trino.spi.type.TypeManager; import io.trino.split.RecordPageSourceProvider; import io.trino.sql.planner.OptimizerConfig; -import io.trino.testing.TestingSession; import io.trino.type.InternalTypeManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,6 +70,7 @@ import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; +import java.time.ZoneId; import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -154,6 +155,10 @@ public void open() throws IOException { this.objectMapperProvider = generateObjectMapperProvider(); initTrinoTableMetadata(); parseRequiredTypes(); + + source = pageSourceProvider.createPageSource(connectorTransactionHandle, + session.toConnectorSession(catalogHandle), + connectorSplit, connectorTableHandle, columns, dynamicFilter); } @Override @@ -163,24 +168,16 @@ public void close() throws IOException { @Override protected int getNext() throws IOException { int rows = 0; - if (connectorSplit == null) { - return 0; - } - if (source == null) { - // TODO(ftw): This line takes a lot time - source = pageSourceProvider.createPageSource(connectorTransactionHandle, - session.toConnectorSession(catalogHandle), - connectorSplit, connectorTableHandle, columns, dynamicFilter); + if (connectorSplit == null || source == null) { + return rows; } - // TODO(ftw): Page is up to 8192 rows, it is best to make 4064 rows - Page page; try { - while (true) { - page = source.getNextPage(); + while (!source.isFinished()) { + Page page = source.getNextPage(); if (page == null) { // used for RecordPageSource - // because RecordPageSource will null even if source is not isFinished. + // because RecordPageSource will be null even if source is not isFinished. if (!source.isFinished()) { continue; } else { @@ -355,7 +352,7 @@ private Session createSession(TrinoConnectorServicesProvider trinoConnectorServi .setSource("test") .setCatalog("catalog") .setSchema("schema") - .setTimeZoneKey(TestingSession.DEFAULT_TIME_ZONE_KEY) + .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(ZoneId.systemDefault().toString())) .setLocale(Locale.ENGLISH) .setClientCapabilities(Arrays.stream(ClientCapabilities.values()).map(Enum::name) .collect(ImmutableSet.toImmutableSet())) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java index 08d33fc1dd2b21..cbb3852c8588d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java @@ -136,7 +136,7 @@ public List initSchema() { null, true, columnMetadata.getComment(), - columnMetadata.isHidden(), + !columnMetadata.isHidden(), Column.COLUMN_UNIQUE_ID_INIT_VALUE); columns.add(column); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/source/TrinoConnectorScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/source/TrinoConnectorScanNode.java index 9e1282f349dd45..ffce150757b618 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/source/TrinoConnectorScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/source/TrinoConnectorScanNode.java @@ -255,12 +255,21 @@ private String encodeObjectToString(T t, ObjectMapperProvider objectMapperPr public void updateRequiredSlots(PlanTranslatorContext planTranslatorContext, Set requiredByProjectSlotIdSet) throws UserException { super.updateRequiredSlots(planTranslatorContext, requiredByProjectSlotIdSet); - String cols = desc.getSlots().stream().map(slot -> slot.getColumn().getName()) - .collect(Collectors.joining(",")); + Map columnMetadataMap = source.getTargetTable().getColumnMetadataMap(); + Map columnHandleMap = source.getTargetTable().getColumnHandleMap(); + List columnHandles = new ArrayList<>(); + for (SlotDescriptor slotDescriptor : desc.getSlots()) { + String colName = slotDescriptor.getColumn().getName(); + if (columnMetadataMap.containsKey(colName)) { + columnHandles.add(columnHandleMap.get(colName)); + } + } + for (TScanRangeLocations tScanRangeLocations : scanRangeLocations) { List ranges = tScanRangeLocations.scan_range.ext_scan_range.file_scan_range.ranges; for (TFileRangeDesc tFileRangeDesc : ranges) { - tFileRangeDesc.table_format_params.trino_connector_params.setTrinoConnectorColumnNames(cols); + tFileRangeDesc.table_format_params.trino_connector_params.setTrinoConnectorColumnHandles( + encodeObjectToString(columnHandles, objectMapperProvider)); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index e4b7eaed92450b..a482330c05f9d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -62,6 +62,8 @@ import org.apache.doris.datasource.odbc.source.OdbcScanNode; import org.apache.doris.datasource.paimon.PaimonExternalTable; import org.apache.doris.datasource.paimon.source.PaimonScanNode; +import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalTable; +import org.apache.doris.datasource.trinoconnector.source.TrinoConnectorScanNode; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.properties.DistributionSpec; import org.apache.doris.nereids.properties.DistributionSpecAny; @@ -549,6 +551,8 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla scanNode = new IcebergScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else if (table instanceof PaimonExternalTable) { scanNode = new PaimonScanNode(context.nextPlanNodeId(), tupleDescriptor, false); + } else if (table instanceof TrinoConnectorExternalTable) { + scanNode = new TrinoConnectorScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else if (table instanceof MaxComputeExternalTable) { scanNode = new MaxComputeScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index e41c4283d603f3..b1b0fe1045c299 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -262,6 +262,7 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio case ICEBERG_EXTERNAL_TABLE: case PAIMON_EXTERNAL_TABLE: case MAX_COMPUTE_EXTERNAL_TABLE: + case TRINO_CONNECTOR_EXTERNAL_TABLE: return new LogicalFileScan(unboundRelation.getRelationId(), (ExternalTable) table, tableQualifier, unboundRelation.getTableSample()); case SCHEMA: From 1a9db84c8b5f31487d957076377a4a2c02a32c6b Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Thu, 14 Mar 2024 20:08:26 +0800 Subject: [PATCH 09/10] [regression](framework) add new config caseNamePrefix (#32229) Co-authored-by: stephen add new config caseNamePrefix --- .../groovy/org/apache/doris/regression/Config.groovy | 11 +++++++++++ .../org/apache/doris/regression/ConfigOptions.groovy | 9 +++++++++ .../org/apache/doris/regression/RegressionTest.groovy | 4 ++++ .../apache/doris/regression/util/TeamcityUtils.groovy | 11 +++++++---- 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy index f678d7e2bc8f33..21c1544427d5f7 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy @@ -107,6 +107,7 @@ class Config { public Integer actionParallel public Integer times public boolean withOutLoadData + public String caseNamePrefix public boolean isSmokeTest public String multiClusterBes public String metaServiceToken @@ -133,6 +134,7 @@ class Config { Config() {} Config( + String caseNamePrefix, String defaultDb, String jdbcUrl, String jdbcUser, @@ -184,6 +186,7 @@ class Config { String clusterDir, String kafkaBrokerList, String cloudVersion) { + this.caseNamePrefix = caseNamePrefix this.defaultDb = defaultDb this.jdbcUrl = jdbcUrl this.jdbcUser = jdbcUser @@ -439,12 +442,14 @@ class Config { config.randomOrder = cmd.hasOption(randomOrderOpt) config.stopWhenFail = cmd.hasOption(stopWhenFailOpt) config.withOutLoadData = cmd.hasOption(withOutLoadDataOpt) + config.caseNamePrefix = cmd.getOptionValue(caseNamePrefixOpt, config.caseNamePrefix) config.dryRun = cmd.hasOption(dryRunOpt) config.isSmokeTest = cmd.hasOption(isSmokeTestOpt) log.info("randomOrder is ${config.randomOrder}".toString()) log.info("stopWhenFail is ${config.stopWhenFail}".toString()) log.info("withOutLoadData is ${config.withOutLoadData}".toString()) + log.info("caseNamePrefix is ${config.caseNamePrefix}".toString()) log.info("dryRun is ${config.dryRun}".toString()) Properties props = cmd.getOptionProperties("conf") @@ -458,6 +463,7 @@ class Config { static Config fromConfigObject(ConfigObject obj) { def config = new Config( + configToString(obj.caseNamePrefix), configToString(obj.defaultDb), configToString(obj.jdbcUrl), configToString(obj.jdbcUser), @@ -576,6 +582,11 @@ class Config { } static void fillDefaultConfig(Config config) { + if (config.caseNamePrefix == null) { + config.caseNamePrefix = "" + log.info("set caseNamePrefix to '' because not specify.".toString()) + } + if (config.defaultDb == null) { config.defaultDb = "regression_test" log.info("Set defaultDb to '${config.defaultDb}' because not specify.".toString()) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/ConfigOptions.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/ConfigOptions.groovy index 3c455e9ecf1644..48e8a74ffd69a8 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/ConfigOptions.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/ConfigOptions.groovy @@ -71,6 +71,7 @@ class ConfigOptions { static Option stopWhenFailOpt static Option timesOpt static Option withOutLoadDataOpt + static Option caseNamePrefixOpt static Option dryRunOpt static Option isSmokeTestOpt static Option multiClusterBesOpt @@ -450,6 +451,13 @@ class ConfigOptions { .longOpt("withOutLoadData") .desc("do not run load.groovy to reload data to Doris.") .build() + caseNamePrefixOpt = Option.builder("cnp") + .required(false) + .hasArg(true) + .type(String.class) + .longOpt("caseNamePrefix") + .desc("add prefix to each case name") + .build() dryRunOpt = Option.builder("dryRun") .required(false) .hasArg(false) @@ -596,6 +604,7 @@ class ConfigOptions { .addOption(stopWhenFailOpt) .addOption(timesOpt) .addOption(withOutLoadDataOpt) + .addOption(caseNamePrefixOpt) .addOption(dryRunOpt) .addOption(isSmokeTestOpt) .addOption(multiClusterBesOpt) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/RegressionTest.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/RegressionTest.groovy index 4f0515c382d9af..feafabaddb477e 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/RegressionTest.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/RegressionTest.groovy @@ -97,6 +97,10 @@ class RegressionTest { TeamcityUtils.postfix = i.toString() } + if (config.caseNamePrefix) { + TeamcityUtils.prefix = config.caseNamePrefix.toString() + } + Recorder recorder = runScripts(config) success = (success && printResult(config, recorder)) diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/TeamcityUtils.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/TeamcityUtils.groovy index 61e6ff2eed3140..fae3cdad846bea 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/TeamcityUtils.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/TeamcityUtils.groovy @@ -25,13 +25,16 @@ import org.apache.tools.ant.util.DateUtils @CompileStatic class TeamcityUtils { static String postfix = "" + static String prefix = "" static String getSuiteName(String name) { - if (postfix == "") { - return name - } else { - return name+"-"+postfix + if (prefix != "") { + name = prefix + "-" + name } + if (postfix != "") { + name = name + "-" + postfix + } + return name } static String formatNow() { From a18845b4bc689af35c591e97ef48d8d6ce858738 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Thu, 14 Mar 2024 21:07:34 +0800 Subject: [PATCH 10/10] [fix](test) ckbench shape unstable since topn_opt_limit_threshold in fuzzy list (#32256) fix ckbench shape unstable since topn_opt_limit_threshold in fuzzy list --- .../data/nereids_clickbench_shape_p0/query1.out | 2 +- .../data/nereids_clickbench_shape_p0/query10.out | 2 +- .../data/nereids_clickbench_shape_p0/query11.out | 2 +- .../data/nereids_clickbench_shape_p0/query12.out | 2 +- .../data/nereids_clickbench_shape_p0/query13.out | 2 +- .../data/nereids_clickbench_shape_p0/query14.out | 2 +- .../data/nereids_clickbench_shape_p0/query15.out | 2 +- .../data/nereids_clickbench_shape_p0/query16.out | 2 +- .../data/nereids_clickbench_shape_p0/query17.out | 2 +- .../data/nereids_clickbench_shape_p0/query18.out | 2 +- .../data/nereids_clickbench_shape_p0/query19.out | 2 +- .../data/nereids_clickbench_shape_p0/query2.out | 2 +- .../data/nereids_clickbench_shape_p0/query20.out | 2 +- .../data/nereids_clickbench_shape_p0/query21.out | 2 +- .../data/nereids_clickbench_shape_p0/query22.out | 2 +- .../data/nereids_clickbench_shape_p0/query23.out | 2 +- .../data/nereids_clickbench_shape_p0/query24.out | 2 +- .../data/nereids_clickbench_shape_p0/query25.out | 2 +- .../data/nereids_clickbench_shape_p0/query26.out | 2 +- .../data/nereids_clickbench_shape_p0/query27.out | 2 +- .../data/nereids_clickbench_shape_p0/query28.out | 2 +- .../data/nereids_clickbench_shape_p0/query29.out | 2 +- .../data/nereids_clickbench_shape_p0/query3.out | 2 +- .../data/nereids_clickbench_shape_p0/query30.out | 2 +- .../data/nereids_clickbench_shape_p0/query31.out | 2 +- .../data/nereids_clickbench_shape_p0/query32.out | 2 +- .../data/nereids_clickbench_shape_p0/query33.out | 2 +- .../data/nereids_clickbench_shape_p0/query34.out | 2 +- .../data/nereids_clickbench_shape_p0/query35.out | 2 +- .../data/nereids_clickbench_shape_p0/query36.out | 2 +- .../data/nereids_clickbench_shape_p0/query37.out | 2 +- .../data/nereids_clickbench_shape_p0/query38.out | 2 +- .../data/nereids_clickbench_shape_p0/query39.out | 2 +- .../data/nereids_clickbench_shape_p0/query4.out | 2 +- .../data/nereids_clickbench_shape_p0/query40.out | 2 +- .../data/nereids_clickbench_shape_p0/query41.out | 2 +- .../data/nereids_clickbench_shape_p0/query42.out | 2 +- .../data/nereids_clickbench_shape_p0/query43.out | 2 +- .../data/nereids_clickbench_shape_p0/query5.out | 2 +- .../data/nereids_clickbench_shape_p0/query6.out | 2 +- .../data/nereids_clickbench_shape_p0/query7.out | 2 +- .../data/nereids_clickbench_shape_p0/query8.out | 2 +- .../data/nereids_clickbench_shape_p0/query9.out | 2 +- .../suites/nereids_clickbench_shape_p0/query1.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query10.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query11.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query12.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query13.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query14.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query15.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query16.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query17.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query18.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query19.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query2.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query20.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query21.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query22.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query23.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query24.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query25.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query26.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query27.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query28.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query29.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query3.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query30.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query31.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query32.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query33.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query34.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query35.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query36.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query37.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query38.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query39.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query4.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query40.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query41.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query42.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query43.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query5.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query6.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query7.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query8.groovy | 7 ++++--- .../suites/nereids_clickbench_shape_p0/query9.groovy | 7 ++++--- 86 files changed, 215 insertions(+), 172 deletions(-) diff --git a/regression-test/data/nereids_clickbench_shape_p0/query1.out b/regression-test/data/nereids_clickbench_shape_p0/query1.out index 140c946b58f574..f98c53e3d5fc4e 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query1.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query1.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_1 -- +-- !ckbench_shape_1 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query10.out b/regression-test/data/nereids_clickbench_shape_p0/query10.out index 3987a5ecb5241c..c784056436912a 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query10.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query10.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_10 -- +-- !ckbench_shape_10 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query11.out b/regression-test/data/nereids_clickbench_shape_p0/query11.out index c3a969eb06d9e4..4b5e4486d3f4cc 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query11.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query11.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_11 -- +-- !ckbench_shape_11 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query12.out b/regression-test/data/nereids_clickbench_shape_p0/query12.out index 61f1f60130c5ee..10928363a83c02 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query12.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query12.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_12 -- +-- !ckbench_shape_12 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query13.out b/regression-test/data/nereids_clickbench_shape_p0/query13.out index 6f78dbb9dfc831..ce6675dc3bb26e 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query13.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query13.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_13 -- +-- !ckbench_shape_13 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query14.out b/regression-test/data/nereids_clickbench_shape_p0/query14.out index aa271b6d27f2c4..35eedce41b927a 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query14.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query14.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_14 -- +-- !ckbench_shape_14 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query15.out b/regression-test/data/nereids_clickbench_shape_p0/query15.out index 114c108ba72a11..bf7f267f0e47be 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query15.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query15.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_15 -- +-- !ckbench_shape_15 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query16.out b/regression-test/data/nereids_clickbench_shape_p0/query16.out index 8f77e55d7a4d62..a229f5310dfc2d 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query16.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query16.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_16 -- +-- !ckbench_shape_16 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query17.out b/regression-test/data/nereids_clickbench_shape_p0/query17.out index 9d58bcabe0a517..78635481d04652 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query17.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query17.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_17 -- +-- !ckbench_shape_17 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query18.out b/regression-test/data/nereids_clickbench_shape_p0/query18.out index aeb98a77114775..0a03abdf84f4d9 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query18.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query18.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_18 -- +-- !ckbench_shape_18 -- PhysicalResultSink --PhysicalLimit[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query19.out b/regression-test/data/nereids_clickbench_shape_p0/query19.out index 5e4556e203f2d8..7540225b393218 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query19.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query19.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_19 -- +-- !ckbench_shape_19 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query2.out b/regression-test/data/nereids_clickbench_shape_p0/query2.out index 5106d00298ef36..4f4565a083c67b 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query2.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query2.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_2 -- +-- !ckbench_shape_2 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query20.out b/regression-test/data/nereids_clickbench_shape_p0/query20.out index 74a11d0639b9be..51f1da68b40a5b 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query20.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query20.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_20 -- +-- !ckbench_shape_20 -- PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] ----PhysicalProject diff --git a/regression-test/data/nereids_clickbench_shape_p0/query21.out b/regression-test/data/nereids_clickbench_shape_p0/query21.out index 6695e7a4f026e4..104d1b4710532a 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query21.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query21.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_21 -- +-- !ckbench_shape_21 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query22.out b/regression-test/data/nereids_clickbench_shape_p0/query22.out index fec375258d45d9..d5274c3548eb28 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query22.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query22.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_22 -- +-- !ckbench_shape_22 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query23.out b/regression-test/data/nereids_clickbench_shape_p0/query23.out index ecb08ba6074577..76a91b3ad49968 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query23.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query23.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_23 -- +-- !ckbench_shape_23 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query24.out b/regression-test/data/nereids_clickbench_shape_p0/query24.out index 9cacd85803dd98..fd0a2f5b670727 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query24.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query24.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_24 -- +-- !ckbench_shape_24 -- PhysicalDeferMaterializeResultSink --PhysicalDeferMaterializeTopN ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query25.out b/regression-test/data/nereids_clickbench_shape_p0/query25.out index 16acace7064e59..271149db672442 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query25.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query25.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_25 -- +-- !ckbench_shape_25 -- PhysicalResultSink --PhysicalProject ----PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query26.out b/regression-test/data/nereids_clickbench_shape_p0/query26.out index 07b70c3395694b..7317f810a3bb23 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query26.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query26.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_26 -- +-- !ckbench_shape_26 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query27.out b/regression-test/data/nereids_clickbench_shape_p0/query27.out index 81fee2d8234e9c..1dbae1e0dc1a8c 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query27.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query27.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_27 -- +-- !ckbench_shape_27 -- PhysicalResultSink --PhysicalProject ----PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query28.out b/regression-test/data/nereids_clickbench_shape_p0/query28.out index 338b96ea2a962d..e5cb28eab7aa2b 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query28.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query28.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_28 -- +-- !ckbench_shape_28 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query29.out b/regression-test/data/nereids_clickbench_shape_p0/query29.out index beb53fbc0ada1f..01e642b5b4339f 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query29.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query29.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_29 -- +-- !ckbench_shape_29 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query3.out b/regression-test/data/nereids_clickbench_shape_p0/query3.out index 2a053055183cbf..d4fb562b4fdb3f 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query3.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query3.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_3 -- +-- !ckbench_shape_3 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query30.out b/regression-test/data/nereids_clickbench_shape_p0/query30.out index 256e01bf4a087d..8a3f753f3dff24 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query30.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query30.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_30 -- +-- !ckbench_shape_30 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query31.out b/regression-test/data/nereids_clickbench_shape_p0/query31.out index 4588d54889ad0e..a662fac4ef4581 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query31.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query31.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_31 -- +-- !ckbench_shape_31 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query32.out b/regression-test/data/nereids_clickbench_shape_p0/query32.out index bc994d43d43a28..29828472ccab40 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query32.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query32.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_32 -- +-- !ckbench_shape_32 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query33.out b/regression-test/data/nereids_clickbench_shape_p0/query33.out index df1e6047b44391..f47fe46b46005a 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query33.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query33.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_33 -- +-- !ckbench_shape_33 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query34.out b/regression-test/data/nereids_clickbench_shape_p0/query34.out index c0634a10984c82..c2b2ed43e72e9f 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query34.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query34.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_34 -- +-- !ckbench_shape_34 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query35.out b/regression-test/data/nereids_clickbench_shape_p0/query35.out index 90cb80f6ce2a16..7fb8594fbe3977 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query35.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query35.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_35 -- +-- !ckbench_shape_35 -- PhysicalResultSink --PhysicalProject ----PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query36.out b/regression-test/data/nereids_clickbench_shape_p0/query36.out index 82bf9c416f93a9..5000e553f81e34 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query36.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query36.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_36 -- +-- !ckbench_shape_36 -- PhysicalResultSink --PhysicalProject ----PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query37.out b/regression-test/data/nereids_clickbench_shape_p0/query37.out index c5b0ca06c85f0a..757b4f64df4e98 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query37.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query37.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_37 -- +-- !ckbench_shape_37 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query38.out b/regression-test/data/nereids_clickbench_shape_p0/query38.out index 7f3cf967a56602..37d0392a7b2490 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query38.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query38.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_38 -- +-- !ckbench_shape_38 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query39.out b/regression-test/data/nereids_clickbench_shape_p0/query39.out index cf5f1f1f0a7f2a..89222c0f0d2abe 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query39.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query39.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_39 -- +-- !ckbench_shape_39 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query4.out b/regression-test/data/nereids_clickbench_shape_p0/query4.out index 5722d13a70822e..966b6cb7ecca97 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query4.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query4.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_4 -- +-- !ckbench_shape_4 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query40.out b/regression-test/data/nereids_clickbench_shape_p0/query40.out index 1acdb05af87124..1218097c179faf 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query40.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query40.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_40 -- +-- !ckbench_shape_40 -- PhysicalResultSink --PhysicalProject ----PhysicalTopN[MERGE_SORT] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query41.out b/regression-test/data/nereids_clickbench_shape_p0/query41.out index 210ae16247f10b..8a7019e5969e79 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query41.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query41.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_41 -- +-- !ckbench_shape_41 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query42.out b/regression-test/data/nereids_clickbench_shape_p0/query42.out index adabcb74980978..b4e8bce045c9c8 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query42.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query42.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_42 -- +-- !ckbench_shape_42 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query43.out b/regression-test/data/nereids_clickbench_shape_p0/query43.out index fda2394495dcf1..80e197103fc1fb 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query43.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query43.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_43 -- +-- !ckbench_shape_43 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query5.out b/regression-test/data/nereids_clickbench_shape_p0/query5.out index b903ce81bfcc1b..94b8f2ad28f023 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query5.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query5.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_5 -- +-- !ckbench_shape_5 -- PhysicalResultSink --hashAgg[DISTINCT_GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query6.out b/regression-test/data/nereids_clickbench_shape_p0/query6.out index 302cdcb3147d8e..904169f68cb62a 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query6.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query6.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_6 -- +-- !ckbench_shape_6 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query7.out b/regression-test/data/nereids_clickbench_shape_p0/query7.out index 9a184539f8f7f0..565f0c0f71985a 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query7.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query7.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_7 -- +-- !ckbench_shape_7 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query8.out b/regression-test/data/nereids_clickbench_shape_p0/query8.out index fc35e28b0e09fb..5bebd9361a2ca6 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query8.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query8.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_8 -- +-- !ckbench_shape_8 -- PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_clickbench_shape_p0/query9.out b/regression-test/data/nereids_clickbench_shape_p0/query9.out index df1ff46add9b99..dcece9f0ce72d7 100644 --- a/regression-test/data/nereids_clickbench_shape_p0/query9.out +++ b/regression-test/data/nereids_clickbench_shape_p0/query9.out @@ -1,5 +1,5 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !ds_shape_9 -- +-- !ckbench_shape_9 -- PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query1.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query1.groovy index dd600b39164257..d6043bb51bd555 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query1.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query1.groovy @@ -20,9 +20,10 @@ suite("query1") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT COUNT(*) FROM hits""" - qt_ds_shape_1 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT COUNT(*) FROM hits""" + qt_ckbench_shape_1 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query10.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query10.groovy index da844658243f91..b967deacf78e88 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query10.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query10.groovy @@ -21,9 +21,10 @@ suite("query10") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10""" - qt_ds_shape_10 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_10 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query11.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query11.groovy index 11210c7b7a9bb7..a3c5e7c89b9487 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query11.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query11.groovy @@ -22,9 +22,10 @@ suite("query11") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10""" - qt_ds_shape_11 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10""" + qt_ckbench_shape_11 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query12.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query12.groovy index 831a894975023b..6dd7cfc7066c5f 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query12.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query12.groovy @@ -22,9 +22,10 @@ suite("query12") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10""" - qt_ds_shape_12 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10""" + qt_ckbench_shape_12 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query13.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query13.groovy index 69980bd27c5c66..02f2d3c0bcb79c 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query13.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query13.groovy @@ -22,9 +22,10 @@ suite("query13") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10""" - qt_ds_shape_13 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_13 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query14.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query14.groovy index 70ac236e3367a1..96ccb1d87b1bf2 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query14.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query14.groovy @@ -22,9 +22,10 @@ suite("query14") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10""" - qt_ds_shape_14 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10""" + qt_ckbench_shape_14 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query15.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query15.groovy index 00962fc323fb44..bd5ec710823614 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query15.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query15.groovy @@ -22,9 +22,10 @@ suite("query15") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10""" - qt_ds_shape_15 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_15 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query16.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query16.groovy index 77d36d9ce467a4..e2d2d7c3a2ac50 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query16.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query16.groovy @@ -22,9 +22,10 @@ suite("query16") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10""" - qt_ds_shape_16 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10""" + qt_ckbench_shape_16 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query17.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query17.groovy index a7649669bdc999..5951de6864c4d9 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query17.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query17.groovy @@ -22,9 +22,10 @@ suite("query17") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10""" - qt_ds_shape_17 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10""" + qt_ckbench_shape_17 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query18.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query18.groovy index e7909a972e1e02..1bd7c55fd47a47 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query18.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query18.groovy @@ -22,9 +22,10 @@ suite("query18") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10""" - qt_ds_shape_18 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10""" + qt_ckbench_shape_18 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query19.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query19.groovy index 5f5e0460fee867..fe740c7a9390a6 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query19.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query19.groovy @@ -22,9 +22,10 @@ suite("query19") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10""" - qt_ds_shape_19 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10""" + qt_ckbench_shape_19 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query2.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query2.groovy index 8d0f3e40b8a498..3482d6d89b7d6e 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query2.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query2.groovy @@ -20,11 +20,12 @@ suite("query2") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """ SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0 """ - qt_ds_shape_2 """ + qt_ckbench_shape_2 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query20.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query20.groovy index 2efe03fa3e6264..78adc179a663dc 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query20.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query20.groovy @@ -22,9 +22,10 @@ suite("query20") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT UserID FROM hits WHERE UserID = 435090932899640449""" - qt_ds_shape_20 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT UserID FROM hits WHERE UserID = 435090932899640449""" + qt_ckbench_shape_20 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query21.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query21.groovy index ad0ca58b24d473..0a221c734fc91a 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query21.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query21.groovy @@ -22,9 +22,10 @@ suite("query21") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'""" - qt_ds_shape_21 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%'""" + qt_ckbench_shape_21 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query22.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query22.groovy index 19145961d855bf..ab7fc01382b16a 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query22.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query22.groovy @@ -22,9 +22,10 @@ suite("query22") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10""" - qt_ds_shape_22 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_22 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query23.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query23.groovy index 90e1a00c38c495..54cdf1b1dacb37 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query23.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query23.groovy @@ -22,9 +22,10 @@ suite("query23") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10""" - qt_ds_shape_23 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_23 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query24.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query24.groovy index 6dff783f3c9821..475d4c62527581 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query24.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query24.groovy @@ -22,9 +22,10 @@ suite("query24") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10""" - qt_ds_shape_24 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10""" + qt_ckbench_shape_24 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query25.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query25.groovy index dc1c4a1512f45b..24a9b77c77bdfc 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query25.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query25.groovy @@ -22,9 +22,10 @@ suite("query25") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10""" - qt_ds_shape_25 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10""" + qt_ckbench_shape_25 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query26.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query26.groovy index 8c711153e6b356..8fae8952feedd9 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query26.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query26.groovy @@ -22,9 +22,10 @@ suite("query26") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10""" - qt_ds_shape_26 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10""" + qt_ckbench_shape_26 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query27.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query27.groovy index 77585ae1359ebb..957862bde9fd64 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query27.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query27.groovy @@ -22,9 +22,10 @@ suite("query27") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10""" - qt_ds_shape_27 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10""" + qt_ckbench_shape_27 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query28.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query28.groovy index 3c3ac2f97bc213..e188cc05ec554d 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query28.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query28.groovy @@ -22,9 +22,10 @@ suite("query28") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM hits WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25""" - qt_ds_shape_28 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM hits WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25""" + qt_ckbench_shape_28 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query29.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query29.groovy index 5138e9188bde1e..dc3a9b8fcbf563 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query29.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query29.groovy @@ -22,9 +22,10 @@ suite("query29") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\\.)?([^/]+)/.*\$', '\\\\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25""" - qt_ds_shape_29 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\\.)?([^/]+)/.*\$', '\\\\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25""" + qt_ckbench_shape_29 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query3.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query3.groovy index d262a438f0414c..7939b7151e0d05 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query3.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query3.groovy @@ -20,9 +20,10 @@ suite("query3") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits""" - qt_ds_shape_3 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits""" + qt_ckbench_shape_3 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query30.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query30.groovy index 12119f8775f8f9..68fc34030e2af3 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query30.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query30.groovy @@ -22,9 +22,10 @@ suite("query30") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SUM(ResolutionWidth), SUM(ResolutionWidth + 1), SUM(ResolutionWidth + 2), SUM(ResolutionWidth + 3), SUM(ResolutionWidth + 4), SUM(ResolutionWidth + 5), SUM(ResolutionWidth + 6), SUM(ResolutionWidth + 7), SUM(ResolutionWidth + 8), SUM(ResolutionWidth + 9), SUM(ResolutionWidth + 10), SUM(ResolutionWidth + 11), SUM(ResolutionWidth + 12), SUM(ResolutionWidth + 13), SUM(ResolutionWidth + 14), SUM(ResolutionWidth + 15), SUM(ResolutionWidth + 16), SUM(ResolutionWidth + 17), SUM(ResolutionWidth + 18), SUM(ResolutionWidth + 19), SUM(ResolutionWidth + 20), SUM(ResolutionWidth + 21), SUM(ResolutionWidth + 22), SUM(ResolutionWidth + 23), SUM(ResolutionWidth + 24), SUM(ResolutionWidth + 25), SUM(ResolutionWidth + 26), SUM(ResolutionWidth + 27), SUM(ResolutionWidth + 28), SUM(ResolutionWidth + 29), SUM(ResolutionWidth + 30), SUM(ResolutionWidth + 31), SUM(ResolutionWidth + 32), SUM(ResolutionWidth + 33), SUM(ResolutionWidth + 34), SUM(ResolutionWidth + 35), SUM(ResolutionWidth + 36), SUM(ResolutionWidth + 37), SUM(ResolutionWidth + 38), SUM(ResolutionWidth + 39), SUM(ResolutionWidth + 40), SUM(ResolutionWidth + 41), SUM(ResolutionWidth + 42), SUM(ResolutionWidth + 43), SUM(ResolutionWidth + 44), SUM(ResolutionWidth + 45), SUM(ResolutionWidth + 46), SUM(ResolutionWidth + 47), SUM(ResolutionWidth + 48), SUM(ResolutionWidth + 49), SUM(ResolutionWidth + 50), SUM(ResolutionWidth + 51), SUM(ResolutionWidth + 52), SUM(ResolutionWidth + 53), SUM(ResolutionWidth + 54), SUM(ResolutionWidth + 55), SUM(ResolutionWidth + 56), SUM(ResolutionWidth + 57), SUM(ResolutionWidth + 58), SUM(ResolutionWidth + 59), SUM(ResolutionWidth + 60), SUM(ResolutionWidth + 61), SUM(ResolutionWidth + 62), SUM(ResolutionWidth + 63), SUM(ResolutionWidth + 64), SUM(ResolutionWidth + 65), SUM(ResolutionWidth + 66), SUM(ResolutionWidth + 67), SUM(ResolutionWidth + 68), SUM(ResolutionWidth + 69), SUM(ResolutionWidth + 70), SUM(ResolutionWidth + 71), SUM(ResolutionWidth + 72), SUM(ResolutionWidth + 73), SUM(ResolutionWidth + 74), SUM(ResolutionWidth + 75), SUM(ResolutionWidth + 76), SUM(ResolutionWidth + 77), SUM(ResolutionWidth + 78), SUM(ResolutionWidth + 79), SUM(ResolutionWidth + 80), SUM(ResolutionWidth + 81), SUM(ResolutionWidth + 82), SUM(ResolutionWidth + 83), SUM(ResolutionWidth + 84), SUM(ResolutionWidth + 85), SUM(ResolutionWidth + 86), SUM(ResolutionWidth + 87), SUM(ResolutionWidth + 88), SUM(ResolutionWidth + 89) FROM hits""" - qt_ds_shape_30 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SUM(ResolutionWidth), SUM(ResolutionWidth + 1), SUM(ResolutionWidth + 2), SUM(ResolutionWidth + 3), SUM(ResolutionWidth + 4), SUM(ResolutionWidth + 5), SUM(ResolutionWidth + 6), SUM(ResolutionWidth + 7), SUM(ResolutionWidth + 8), SUM(ResolutionWidth + 9), SUM(ResolutionWidth + 10), SUM(ResolutionWidth + 11), SUM(ResolutionWidth + 12), SUM(ResolutionWidth + 13), SUM(ResolutionWidth + 14), SUM(ResolutionWidth + 15), SUM(ResolutionWidth + 16), SUM(ResolutionWidth + 17), SUM(ResolutionWidth + 18), SUM(ResolutionWidth + 19), SUM(ResolutionWidth + 20), SUM(ResolutionWidth + 21), SUM(ResolutionWidth + 22), SUM(ResolutionWidth + 23), SUM(ResolutionWidth + 24), SUM(ResolutionWidth + 25), SUM(ResolutionWidth + 26), SUM(ResolutionWidth + 27), SUM(ResolutionWidth + 28), SUM(ResolutionWidth + 29), SUM(ResolutionWidth + 30), SUM(ResolutionWidth + 31), SUM(ResolutionWidth + 32), SUM(ResolutionWidth + 33), SUM(ResolutionWidth + 34), SUM(ResolutionWidth + 35), SUM(ResolutionWidth + 36), SUM(ResolutionWidth + 37), SUM(ResolutionWidth + 38), SUM(ResolutionWidth + 39), SUM(ResolutionWidth + 40), SUM(ResolutionWidth + 41), SUM(ResolutionWidth + 42), SUM(ResolutionWidth + 43), SUM(ResolutionWidth + 44), SUM(ResolutionWidth + 45), SUM(ResolutionWidth + 46), SUM(ResolutionWidth + 47), SUM(ResolutionWidth + 48), SUM(ResolutionWidth + 49), SUM(ResolutionWidth + 50), SUM(ResolutionWidth + 51), SUM(ResolutionWidth + 52), SUM(ResolutionWidth + 53), SUM(ResolutionWidth + 54), SUM(ResolutionWidth + 55), SUM(ResolutionWidth + 56), SUM(ResolutionWidth + 57), SUM(ResolutionWidth + 58), SUM(ResolutionWidth + 59), SUM(ResolutionWidth + 60), SUM(ResolutionWidth + 61), SUM(ResolutionWidth + 62), SUM(ResolutionWidth + 63), SUM(ResolutionWidth + 64), SUM(ResolutionWidth + 65), SUM(ResolutionWidth + 66), SUM(ResolutionWidth + 67), SUM(ResolutionWidth + 68), SUM(ResolutionWidth + 69), SUM(ResolutionWidth + 70), SUM(ResolutionWidth + 71), SUM(ResolutionWidth + 72), SUM(ResolutionWidth + 73), SUM(ResolutionWidth + 74), SUM(ResolutionWidth + 75), SUM(ResolutionWidth + 76), SUM(ResolutionWidth + 77), SUM(ResolutionWidth + 78), SUM(ResolutionWidth + 79), SUM(ResolutionWidth + 80), SUM(ResolutionWidth + 81), SUM(ResolutionWidth + 82), SUM(ResolutionWidth + 83), SUM(ResolutionWidth + 84), SUM(ResolutionWidth + 85), SUM(ResolutionWidth + 86), SUM(ResolutionWidth + 87), SUM(ResolutionWidth + 88), SUM(ResolutionWidth + 89) FROM hits""" + qt_ckbench_shape_30 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query31.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query31.groovy index 7e8645462141ba..68f162b5ff7ce4 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query31.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query31.groovy @@ -22,9 +22,10 @@ suite("query31") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10""" - qt_ds_shape_31 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_31 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query32.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query32.groovy index 399b7d218b5175..1ee97f415dbb33 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query32.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query32.groovy @@ -22,9 +22,10 @@ suite("query32") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10""" - qt_ds_shape_32 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_32 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query33.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query33.groovy index 9a7f8765e5d4d1..8a59a71a6119b6 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query33.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query33.groovy @@ -22,9 +22,10 @@ suite("query33") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10""" - qt_ds_shape_33 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_33 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query34.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query34.groovy index ad97e9d90dc210..6a80ffc7a71a3d 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query34.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query34.groovy @@ -22,9 +22,10 @@ suite("query34") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10""" - qt_ds_shape_34 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_34 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query35.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query35.groovy index d530eec5af0168..dbdd745e44a02d 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query35.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query35.groovy @@ -22,9 +22,10 @@ suite("query35") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10""" - qt_ds_shape_35 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_35 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query36.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query36.groovy index 237d75e58a66dd..1aa7b9cf8c7513 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query36.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query36.groovy @@ -22,9 +22,10 @@ suite("query36") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM hits GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10""" - qt_ds_shape_36 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM hits GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10""" + qt_ckbench_shape_36 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query37.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query37.groovy index 0ec83cbdfe3e5a..794439aa06a2ed 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query37.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query37.groovy @@ -22,9 +22,10 @@ suite("query37") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10""" - qt_ds_shape_37 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10""" + qt_ckbench_shape_37 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query38.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query38.groovy index 5f8e0ada843fec..543871605a095a 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query38.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query38.groovy @@ -22,9 +22,10 @@ suite("query38") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10""" - qt_ds_shape_38 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10""" + qt_ckbench_shape_38 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query39.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query39.groovy index e1f73de71dc16d..dd5a48f45c5be6 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query39.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query39.groovy @@ -22,9 +22,10 @@ suite("query39") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000""" - qt_ds_shape_39 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000""" + qt_ckbench_shape_39 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query4.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query4.groovy index 0367c3f23dfbc3..a14ef08e312d06 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query4.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query4.groovy @@ -20,9 +20,10 @@ suite("query4") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT AVG(UserID) FROM hits""" - qt_ds_shape_4 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT AVG(UserID) FROM hits""" + qt_ckbench_shape_4 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query40.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query40.groovy index 03b71327920c8b..8d483a6210f73c 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query40.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query40.groovy @@ -22,9 +22,10 @@ suite("query40") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END AS Src, URL AS Dst, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000""" - qt_ds_shape_40 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END AS Src, URL AS Dst, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000""" + qt_ckbench_shape_40 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query41.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query41.groovy index f0435492024030..067bf6c1dc759b 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query41.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query41.groovy @@ -22,9 +22,10 @@ suite("query41") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100""" - qt_ds_shape_41 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100""" + qt_ckbench_shape_41 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query42.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query42.groovy index b8678eebdc25b0..660ea72ae8397f 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query42.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query42.groovy @@ -22,9 +22,10 @@ suite("query42") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000""" - qt_ds_shape_42 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000""" + qt_ckbench_shape_42 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query43.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query43.groovy index dfb9d303d88f63..382f254bd2e1f5 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query43.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query43.groovy @@ -21,9 +21,10 @@ suite("query43") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') LIMIT 10 OFFSET 1000""" - qt_ds_shape_43 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') LIMIT 10 OFFSET 1000""" + qt_ckbench_shape_43 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query5.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query5.groovy index e27ef96af842a5..16faeb1d0d678d 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query5.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query5.groovy @@ -20,9 +20,10 @@ suite("query5") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT COUNT(DISTINCT UserID) FROM hits""" - qt_ds_shape_5 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT COUNT(DISTINCT UserID) FROM hits""" + qt_ckbench_shape_5 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query6.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query6.groovy index 91c26622ffaf5a..1f2d89dd9234a9 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query6.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query6.groovy @@ -20,9 +20,10 @@ suite("query6") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT COUNT(DISTINCT SearchPhrase) FROM hits""" - qt_ds_shape_6 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT COUNT(DISTINCT SearchPhrase) FROM hits""" + qt_ckbench_shape_6 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query7.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query7.groovy index 4785e0f853c666..da50b6259deaa4 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query7.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query7.groovy @@ -21,9 +21,10 @@ suite("query7") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT MIN(EventDate), MAX(EventDate) FROM hits""" - qt_ds_shape_7 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT MIN(EventDate), MAX(EventDate) FROM hits""" + qt_ckbench_shape_7 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query8.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query8.groovy index 4675dcac2f8fda..1cf2626b45799f 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query8.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query8.groovy @@ -21,9 +21,10 @@ suite("query8") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC""" - qt_ds_shape_8 """ + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC""" + qt_ckbench_shape_8 """ explain shape plan - ${ds} + ${ckBench} """ } diff --git a/regression-test/suites/nereids_clickbench_shape_p0/query9.groovy b/regression-test/suites/nereids_clickbench_shape_p0/query9.groovy index 9b59357a21bf26..166b210a2f2e28 100644 --- a/regression-test/suites/nereids_clickbench_shape_p0/query9.groovy +++ b/regression-test/suites/nereids_clickbench_shape_p0/query9.groovy @@ -21,10 +21,11 @@ suite("query9") { sql 'set enable_nereids_planner=true' sql 'set enable_fallback_to_original_planner=false' - def ds = """SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10 + sql 'set topn_opt_limit_threshold = 1024' + def ckBench = """SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10 """ - qt_ds_shape_9 """ + qt_ckbench_shape_9 """ explain shape plan - ${ds} + ${ckBench} """ }