From b74e38c5c28bdb5bc2c13af89d3d7ce982470bfa Mon Sep 17 00:00:00 2001 From: Jerry Hu Date: Thu, 31 Oct 2024 14:13:31 +0800 Subject: [PATCH 01/82] [chore](profile) add task state in profile (#42895) ## Proposed changes Issue Number: close #xxx --- be/src/pipeline/exec/scan_operator.cpp | 1 + be/src/pipeline/pipeline_task.cpp | 9 +++++++++ 2 files changed, 10 insertions(+) diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index 024a737e1579a32..6200f3b12ce5a05 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -1281,6 +1281,7 @@ Status ScanOperatorX::get_block(RuntimeState* state, vectorized: if (*eos) { // reach limit, stop the scanners. local_state._scanner_ctx->stop_scanners(state); + local_state._scanner_profile->add_info_string("EOS", "True"); } return Status::OK(); diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp index e06b8028c9c7308..a8213b31ba8f47c 100644 --- a/be/src/pipeline/pipeline_task.cpp +++ b/be/src/pipeline/pipeline_task.cpp @@ -247,6 +247,12 @@ bool PipelineTask::_wait_to_start() { } bool PipelineTask::_is_blocked() { + Defer defer([this] { + if (_blocked_dep != nullptr) { + _task_profile->add_info_string("TaskState", "Blocked"); + _task_profile->add_info_string("BlockedByDependency", _blocked_dep->name()); + } + }); // `_dry_run = true` means we do not need data from source operator. if (!_dry_run) { for (int i = _read_dependencies.size() - 1; i >= 0; i--) { @@ -328,6 +334,8 @@ Status PipelineTask::execute(bool* eos) { RETURN_IF_ERROR(_open()); } + _task_profile->add_info_string("TaskState", "Runnable"); + _task_profile->add_info_string("BlockedByDependency", ""); while (!_fragment_context->is_canceled()) { if (_is_blocked()) { return Status::OK(); @@ -391,6 +399,7 @@ Status PipelineTask::execute(bool* eos) { *eos = status.is() ? true : *eos; if (*eos) { // just return, the scheduler will do finish work _eos = true; + _task_profile->add_info_string("TaskState", "Finished"); return Status::OK(); } } From c69e815083006733ce71ff44e701be665ba7e3b2 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Thu, 31 Oct 2024 14:32:53 +0800 Subject: [PATCH 02/82] [refactor](metrics) Refactor metrics for some operators (#42909) --- .../pipeline/exec/analytic_sink_operator.cpp | 56 ++++++++++------- be/src/pipeline/exec/analytic_sink_operator.h | 4 +- .../exec/analytic_source_operator.cpp | 36 ++++++----- .../pipeline/exec/analytic_source_operator.h | 8 +-- .../pipeline/exec/cache_source_operator.cpp | 20 +++--- be/src/pipeline/exec/datagen_operator.cpp | 16 +++-- be/src/pipeline/exec/datagen_operator.h | 2 + be/src/pipeline/exec/exchange_sink_buffer.h | 1 - .../pipeline/exec/exchange_sink_operator.cpp | 62 +++++++------------ be/src/pipeline/exec/exchange_sink_operator.h | 22 +------ .../exec/exchange_source_operator.cpp | 19 ++++-- .../pipeline/exec/exchange_source_operator.h | 3 + .../exec/result_file_sink_operator.cpp | 6 -- .../pipeline/exec/result_file_sink_operator.h | 14 ----- be/src/pipeline/exec/result_sink_operator.cpp | 3 +- be/src/pipeline/exec/result_sink_operator.h | 2 - be/src/vec/sink/vdata_stream_sender.cpp | 3 +- be/src/vec/sink/vdata_stream_sender.h | 1 + 18 files changed, 128 insertions(+), 150 deletions(-) diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index afe9aeab8fdb847..abde34a1d0255bc 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -30,8 +30,10 @@ Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); - _blocks_memory_usage = ADD_COUNTER_WITH_LEVEL(_profile, "MemoryUsageBlocks", TUnit::BYTES, 1); - _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); + _evaluation_timer = ADD_TIMER(profile(), "GetPartitionBoundTime"); + _compute_agg_data_timer = ADD_TIMER(profile(), "ComputeAggDataTime"); + _compute_partition_by_timer = ADD_TIMER(profile(), "ComputePartitionByTime"); + _compute_order_by_timer = ADD_TIMER(profile(), "ComputeOrderByTime"); return Status::OK(); } @@ -288,35 +290,41 @@ Status AnalyticSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block } } - for (size_t i = 0; i < _agg_functions_size; - ++i) { //insert _agg_input_columns, execute calculate for its - for (size_t j = 0; j < local_state._agg_expr_ctxs[i].size(); ++j) { - RETURN_IF_ERROR(_insert_range_column( - input_block, local_state._agg_expr_ctxs[i][j], - local_state._shared_state->agg_input_columns[i][j].get(), block_rows)); + { + SCOPED_TIMER(local_state._compute_agg_data_timer); + for (size_t i = 0; i < _agg_functions_size; + ++i) { //insert _agg_input_columns, execute calculate for its + for (size_t j = 0; j < local_state._agg_expr_ctxs[i].size(); ++j) { + RETURN_IF_ERROR(_insert_range_column( + input_block, local_state._agg_expr_ctxs[i][j], + local_state._shared_state->agg_input_columns[i][j].get(), block_rows)); + } } } - //record column idx in block - for (size_t i = 0; i < local_state._shared_state->partition_by_eq_expr_ctxs.size(); ++i) { - int result_col_id = -1; - RETURN_IF_ERROR(local_state._shared_state->partition_by_eq_expr_ctxs[i]->execute( - input_block, &result_col_id)); - DCHECK_GE(result_col_id, 0); - local_state._shared_state->partition_by_column_idxs[i] = result_col_id; + { + SCOPED_TIMER(local_state._compute_partition_by_timer); + for (size_t i = 0; i < local_state._shared_state->partition_by_eq_expr_ctxs.size(); ++i) { + int result_col_id = -1; + RETURN_IF_ERROR(local_state._shared_state->partition_by_eq_expr_ctxs[i]->execute( + input_block, &result_col_id)); + DCHECK_GE(result_col_id, 0); + local_state._shared_state->partition_by_column_idxs[i] = result_col_id; + } } - for (size_t i = 0; i < local_state._shared_state->order_by_eq_expr_ctxs.size(); ++i) { - int result_col_id = -1; - RETURN_IF_ERROR(local_state._shared_state->order_by_eq_expr_ctxs[i]->execute( - input_block, &result_col_id)); - DCHECK_GE(result_col_id, 0); - local_state._shared_state->ordey_by_column_idxs[i] = result_col_id; + { + SCOPED_TIMER(local_state._compute_order_by_timer); + for (size_t i = 0; i < local_state._shared_state->order_by_eq_expr_ctxs.size(); ++i) { + int result_col_id = -1; + RETURN_IF_ERROR(local_state._shared_state->order_by_eq_expr_ctxs[i]->execute( + input_block, &result_col_id)); + DCHECK_GE(result_col_id, 0); + local_state._shared_state->ordey_by_column_idxs[i] = result_col_id; + } } - int64_t block_mem_usage = input_block->allocated_bytes(); - COUNTER_UPDATE(local_state._memory_used_counter, block_mem_usage); + COUNTER_UPDATE(local_state._memory_used_counter, input_block->allocated_bytes()); COUNTER_SET(local_state._peak_memory_usage_counter, local_state._memory_used_counter->value()); - COUNTER_UPDATE(local_state._blocks_memory_usage, block_mem_usage); //TODO: if need improvement, the is a tips to maintain a free queue, //so the memory could reuse, no need to new/delete again; diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index b35354107f671f4..e04b220ee351e7f 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -58,7 +58,9 @@ class AnalyticSinkLocalState : public PipelineXSinkLocalState _agg_expr_ctxs; }; diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp index 019f95042c2e4a8..7dafbaa0330683f 100644 --- a/be/src/pipeline/exec/analytic_source_operator.cpp +++ b/be/src/pipeline/exec/analytic_source_operator.cpp @@ -162,7 +162,10 @@ Status AnalyticLocalState::init(RuntimeState* state, LocalStateInfo& info) { SCOPED_TIMER(_init_timer); _blocks_memory_usage = profile()->AddHighWaterMarkCounter("MemoryUsageBlocks", TUnit::BYTES, "", 1); - _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); + _evaluation_timer = ADD_TIMER(profile(), "GetPartitionBoundTime"); + _execute_timer = ADD_TIMER(profile(), "ExecuteTime"); + _get_next_timer = ADD_TIMER(profile(), "GetNextTime"); + _get_result_timer = ADD_TIMER(profile(), "GetResultsTime"); return Status::OK(); } @@ -233,12 +236,6 @@ Status AnalyticLocalState::open(RuntimeState* state) { std::placeholders::_1); } } - _executor.insert_result = - std::bind(&AnalyticLocalState::_insert_result_info, this, std::placeholders::_1); - _executor.execute = - std::bind(&AnalyticLocalState::_execute_for_win_func, this, std::placeholders::_1, - std::placeholders::_2, std::placeholders::_3, std::placeholders::_4); - _create_agg_status(); return Status::OK(); } @@ -282,6 +279,7 @@ void AnalyticLocalState::_destroy_agg_status() { void AnalyticLocalState::_execute_for_win_func(int64_t partition_start, int64_t partition_end, int64_t frame_start, int64_t frame_end) { + SCOPED_TIMER(_execute_timer); for (size_t i = 0; i < _agg_functions_size; ++i) { std::vector agg_columns; for (int j = 0; j < _shared_state->agg_input_columns[i].size(); ++j) { @@ -300,6 +298,7 @@ void AnalyticLocalState::_execute_for_win_func(int64_t partition_start, int64_t } void AnalyticLocalState::_insert_result_info(int64_t current_block_rows) { + SCOPED_TIMER(_get_result_timer); int64_t current_block_row_pos = _shared_state->input_block_first_row_positions[_output_block_index]; int64_t get_result_start = _shared_state->current_row_position - current_block_row_pos; @@ -344,6 +343,7 @@ void AnalyticLocalState::_insert_result_info(int64_t current_block_rows) { } Status AnalyticLocalState::_get_next_for_rows(size_t current_block_rows) { + SCOPED_TIMER(_get_next_timer); while (_shared_state->current_row_position < _shared_state->partition_by_end.pos && _window_end_position < current_block_rows) { int64_t range_start, range_end; @@ -367,31 +367,33 @@ Status AnalyticLocalState::_get_next_for_rows(size_t current_block_rows) { // Make sure range_start <= range_end range_start = std::min(range_start, range_end); } - _executor.execute(_partition_by_start.pos, _shared_state->partition_by_end.pos, range_start, - range_end); - _executor.insert_result(current_block_rows); + _execute_for_win_func(_partition_by_start.pos, _shared_state->partition_by_end.pos, + range_start, range_end); + _insert_result_info(current_block_rows); } return Status::OK(); } Status AnalyticLocalState::_get_next_for_partition(size_t current_block_rows) { + SCOPED_TIMER(_get_next_timer); if (_next_partition) { - _executor.execute(_partition_by_start.pos, _shared_state->partition_by_end.pos, - _partition_by_start.pos, _shared_state->partition_by_end.pos); + _execute_for_win_func(_partition_by_start.pos, _shared_state->partition_by_end.pos, + _partition_by_start.pos, _shared_state->partition_by_end.pos); } - _executor.insert_result(current_block_rows); + _insert_result_info(current_block_rows); return Status::OK(); } Status AnalyticLocalState::_get_next_for_range(size_t current_block_rows) { + SCOPED_TIMER(_get_next_timer); while (_shared_state->current_row_position < _shared_state->partition_by_end.pos && _window_end_position < current_block_rows) { if (_shared_state->current_row_position >= _order_by_end.pos) { _update_order_by_range(); - _executor.execute(_partition_by_start.pos, _shared_state->partition_by_end.pos, - _order_by_start.pos, _order_by_end.pos); + _execute_for_win_func(_partition_by_start.pos, _shared_state->partition_by_end.pos, + _order_by_start.pos, _order_by_end.pos); } - _executor.insert_result(current_block_rows); + _insert_result_info(current_block_rows); } return Status::OK(); } @@ -536,7 +538,7 @@ Status AnalyticSourceOperatorX::get_block(RuntimeState* state, vectorized::Block local_state.init_result_columns(); size_t current_block_rows = local_state._shared_state->input_blocks[local_state._output_block_index].rows(); - static_cast(local_state._executor.get_next(current_block_rows)); + RETURN_IF_ERROR(local_state._executor.get_next(current_block_rows)); if (local_state._window_end_position == current_block_rows) { break; } diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 0080ad5e03c8b02..8f44b77f567e559 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -96,17 +96,15 @@ class AnalyticLocalState final : public PipelineXLocalState std::vector _agg_functions; RuntimeProfile::Counter* _evaluation_timer = nullptr; + RuntimeProfile::Counter* _execute_timer = nullptr; + RuntimeProfile::Counter* _get_next_timer = nullptr; + RuntimeProfile::Counter* _get_result_timer = nullptr; RuntimeProfile::HighWaterMarkCounter* _blocks_memory_usage = nullptr; - using vectorized_execute = std::function; using vectorized_get_next = std::function; - using vectorized_get_result = std::function; struct executor { - vectorized_execute execute; vectorized_get_next get_next; - vectorized_get_result insert_result; }; executor _executor; diff --git a/be/src/pipeline/exec/cache_source_operator.cpp b/be/src/pipeline/exec/cache_source_operator.cpp index 2e9b21976f841ae..cace8465fc2d463 100644 --- a/be/src/pipeline/exec/cache_source_operator.cpp +++ b/be/src/pipeline/exec/cache_source_operator.cpp @@ -65,7 +65,7 @@ Status CacheSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { // 3. lookup the cache and find proper slot order hit_cache = QueryCache::instance()->lookup(_cache_key, _version, &_query_cache_handle); - _runtime_profile->add_info_string("HitCache", hit_cache ? "1" : "0"); + _runtime_profile->add_info_string("HitCache", std::to_string(hit_cache)); if (hit_cache && !cache_param.force_refresh_query_cache) { _hit_cache_results = _query_cache_handle.get_cache_result(); auto hit_cache_slot_orders = _query_cache_handle.get_cache_slot_orders(); @@ -125,13 +125,16 @@ Status CacheSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* b if (local_state._hit_cache_results == nullptr) { Defer insert_cache([&] { - if (*eos && local_state._need_insert_cache) { - local_state._runtime_profile->add_info_string("InsertCache", "1"); - local_state._global_cache->insert(local_state._cache_key, local_state._version, - local_state._local_cache_blocks, - local_state._slot_orders, - local_state._current_query_cache_bytes); - local_state._local_cache_blocks.clear(); + if (*eos) { + local_state._runtime_profile->add_info_string( + "InsertCache", std::to_string(local_state._need_insert_cache)); + if (local_state._need_insert_cache) { + local_state._global_cache->insert(local_state._cache_key, local_state._version, + local_state._local_cache_blocks, + local_state._slot_orders, + local_state._current_query_cache_bytes); + local_state._local_cache_blocks.clear(); + } } }); @@ -162,7 +165,6 @@ Status CacheSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* b // over the max bytes, pass through the data, no need to do cache local_state._local_cache_blocks.clear(); local_state._need_insert_cache = false; - local_state._runtime_profile->add_info_string("InsertCache", "0"); } else { local_state._local_cache_blocks.emplace_back(std::move(output_block)); } diff --git a/be/src/pipeline/exec/datagen_operator.cpp b/be/src/pipeline/exec/datagen_operator.cpp index 965092b7eef20f2..d400953799e5bbb 100644 --- a/be/src/pipeline/exec/datagen_operator.cpp +++ b/be/src/pipeline/exec/datagen_operator.cpp @@ -70,17 +70,25 @@ Status DataGenSourceOperatorX::get_block(RuntimeState* state, vectorized::Block* RETURN_IF_CANCELLED(state); auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); - Status res = local_state._table_func->get_next(state, block, eos); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + { + SCOPED_TIMER(local_state._table_function_execution_timer); + RETURN_IF_ERROR(local_state._table_func->get_next(state, block, eos)); + } + { + SCOPED_TIMER(local_state._filter_timer); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, + block->columns())); + } local_state.reached_limit(block, eos); - return res; + return Status::OK(); } Status DataGenLocalState::init(RuntimeState* state, LocalStateInfo& info) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); + _table_function_execution_timer = ADD_TIMER(profile(), "TableFunctionExecutionTime"); + _filter_timer = ADD_TIMER(profile(), "FilterTime"); auto& p = _parent->cast(); _table_func = std::make_shared(p._tuple_id, p._tuple_desc); _table_func->set_tuple_desc(p._tuple_desc); diff --git a/be/src/pipeline/exec/datagen_operator.h b/be/src/pipeline/exec/datagen_operator.h index c63ef97bb7a40f6..bada5ec4080d08b 100644 --- a/be/src/pipeline/exec/datagen_operator.h +++ b/be/src/pipeline/exec/datagen_operator.h @@ -44,6 +44,8 @@ class DataGenLocalState final : public PipelineXLocalState<> { private: friend class DataGenSourceOperatorX; std::shared_ptr _table_func; + RuntimeProfile::Counter* _table_function_execution_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; }; class DataGenSourceOperatorX final : public OperatorX { diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index 2ff7a20086470a1..13692532a335a42 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -195,7 +195,6 @@ class ExchangeSinkBuffer final : public HasTaskExecutionCtx { private: friend class ExchangeSinkLocalState; - void _set_ready_to_finish(bool all_done); phmap::flat_hash_map> _instance_to_package_queue_mutex; diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 55b0e43c936d5a7..1f91af01aa1f6bb 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -39,11 +39,6 @@ namespace doris::pipeline { #include "common/compile_check_begin.h" -Status ExchangeSinkLocalState::serialize_block(vectorized::Block* src, PBlock* dest, - int num_receivers) { - return _parent->cast().serialize_block(*this, src, dest, num_receivers); -} - bool ExchangeSinkLocalState::transfer_large_data_by_brpc() const { return _parent->cast()._transfer_large_data_by_brpc; } @@ -61,14 +56,10 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf _local_sent_rows = ADD_COUNTER(_profile, "LocalSentRows", TUnit::UNIT); _serialize_batch_timer = ADD_TIMER(_profile, "SerializeBatchTime"); _compress_timer = ADD_TIMER(_profile, "CompressTime"); - _brpc_send_timer = ADD_TIMER(_profile, "BrpcSendTime"); - _brpc_wait_timer = ADD_TIMER(_profile, "BrpcSendTime.Wait"); _local_send_timer = ADD_TIMER(_profile, "LocalSendTime"); _split_block_hash_compute_timer = ADD_TIMER(_profile, "SplitBlockHashComputeTime"); - _split_block_distribute_by_channel_timer = - ADD_TIMER(_profile, "SplitBlockDistributeByChannelTime"); + _distribute_rows_into_channels_timer = ADD_TIMER(_profile, "DistributeRowsIntoChannelsTime"); _blocks_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "BlocksProduced", TUnit::UNIT, 1); - _rows_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "RowsProduced", TUnit::UNIT, 1); _overall_throughput = _profile->add_derived_counter( "OverallThroughput", TUnit::BYTES_PER_SECOND, [this]() { @@ -141,7 +132,7 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { std::mt19937 g(rd()); shuffle(channels.begin(), channels.end(), g); } - int local_size = 0; + size_t local_size = 0; for (int i = 0; i < channels.size(); ++i) { RETURN_IF_ERROR(channels[i]->open(state)); if (channels[i]->is_local()) { @@ -151,6 +142,8 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { } only_local_exchange = local_size == channels.size(); + _rpc_channels_num = channels.size() - local_size; + PUniqueId id; id.set_hi(_state->query_id().hi); id.set_lo(_state->query_id().lo); @@ -389,7 +382,6 @@ void ExchangeSinkOperatorX::_handle_eof_channel(RuntimeState* state, ChannelPtrT Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, bool eos) { auto& local_state = get_local_state(state); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); - COUNTER_UPDATE(local_state.rows_sent_counter(), (int64_t)block->rows()); SCOPED_TIMER(local_state.exec_time_counter()); bool all_receiver_eof = true; for (auto& channel : local_state.channels) { @@ -431,14 +423,15 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block { bool serialized = false; RETURN_IF_ERROR(local_state._serializer.next_serialized_block( - block, block_holder->get_block(), local_state.channels.size(), &serialized, - eos)); + block, block_holder->get_block(), local_state._rpc_channels_num, + &serialized, eos)); if (serialized) { auto cur_block = local_state._serializer.get_block()->to_block(); if (!cur_block.empty()) { + DCHECK(eos || local_state._serializer.is_local()) << debug_string(state, 0); RETURN_IF_ERROR(local_state._serializer.serialize_block( &cur_block, block_holder->get_block(), - local_state.channels.size())); + local_state._rpc_channels_num)); } else { block_holder->reset_block(); } @@ -504,10 +497,12 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block old_channel_mem_usage += channel->mem_usage(); } if (_part_type == TPartitionType::HASH_PARTITIONED) { + SCOPED_TIMER(local_state._distribute_rows_into_channels_timer); RETURN_IF_ERROR(channel_add_rows( state, local_state.channels, local_state._partition_count, local_state._partitioner->get_channel_ids().get(), rows, block, eos)); } else { + SCOPED_TIMER(local_state._distribute_rows_into_channels_timer); RETURN_IF_ERROR(channel_add_rows( state, local_state.channels, local_state._partition_count, local_state._partitioner->get_channel_ids().get(), rows, block, eos)); @@ -556,10 +551,13 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block local_state._row_distribution._deal_batched = true; RETURN_IF_ERROR(local_state._send_new_partition_batch()); } - // the convert_block maybe different with block after execute exprs - // when send data we still use block - RETURN_IF_ERROR(channel_add_rows_with_idx(state, local_state.channels, num_channels, - channel2rows, block, eos)); + { + SCOPED_TIMER(local_state._distribute_rows_into_channels_timer); + // the convert_block maybe different with block after execute exprs + // when send data we still use block + RETURN_IF_ERROR(channel_add_rows_with_idx(state, local_state.channels, num_channels, + channel2rows, block, eos)); + } int64_t new_channel_mem_usage = 0; for (const auto& channel : local_state.channels) { new_channel_mem_usage += channel->mem_usage(); @@ -579,8 +577,12 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block } std::vector> assignments = local_state.scale_writer_partitioning_exchanger->accept(block); - RETURN_IF_ERROR(channel_add_rows_with_idx( - state, local_state.channels, local_state.channels.size(), assignments, block, eos)); + { + SCOPED_TIMER(local_state._distribute_rows_into_channels_timer); + RETURN_IF_ERROR(channel_add_rows_with_idx(state, local_state.channels, + local_state.channels.size(), assignments, + block, eos)); + } int64_t new_channel_mem_usage = 0; for (const auto& channel : local_state.channels) { @@ -635,24 +637,6 @@ Status ExchangeSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block return final_st; } -Status ExchangeSinkOperatorX::serialize_block(ExchangeSinkLocalState& state, vectorized::Block* src, - PBlock* dest, int num_receivers) { - { - SCOPED_TIMER(state.serialize_batch_timer()); - dest->Clear(); - size_t uncompressed_bytes = 0; - size_t compressed_bytes = 0; - RETURN_IF_ERROR(src->serialize(_state->be_exec_version(), dest, &uncompressed_bytes, - &compressed_bytes, _compression_type, - _transfer_large_data_by_brpc)); - COUNTER_UPDATE(state.bytes_sent_counter(), compressed_bytes * num_receivers); - COUNTER_UPDATE(state.uncompressed_bytes_counter(), uncompressed_bytes * num_receivers); - COUNTER_UPDATE(state.compress_timer(), src->get_compress_time()); - } - - return Status::OK(); -} - void ExchangeSinkLocalState::register_channels(pipeline::ExchangeSinkBuffer* buffer) { for (auto& channel : channels) { channel->register_exchange_buffer(buffer); diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 141693eb820f4a5..63d502900054703 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -77,27 +77,13 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { Status open(RuntimeState* state) override; Status close(RuntimeState* state, Status exec_status) override; Dependency* finishdependency() override { return _finish_dependency.get(); } - Status serialize_block(vectorized::Block* src, PBlock* dest, int num_receivers = 1); void register_channels(pipeline::ExchangeSinkBuffer* buffer); - RuntimeProfile::Counter* brpc_wait_timer() { return _brpc_wait_timer; } RuntimeProfile::Counter* blocks_sent_counter() { return _blocks_sent_counter; } - RuntimeProfile::Counter* rows_sent_counter() { return _rows_sent_counter; } RuntimeProfile::Counter* local_send_timer() { return _local_send_timer; } RuntimeProfile::Counter* local_bytes_send_counter() { return _local_bytes_send_counter; } RuntimeProfile::Counter* local_sent_rows() { return _local_sent_rows; } - RuntimeProfile::Counter* brpc_send_timer() { return _brpc_send_timer; } - RuntimeProfile::Counter* serialize_batch_timer() { return _serialize_batch_timer; } - RuntimeProfile::Counter* split_block_distribute_by_channel_timer() { - return _split_block_distribute_by_channel_timer; - } - RuntimeProfile::Counter* bytes_sent_counter() { return _bytes_sent_counter; } - RuntimeProfile::Counter* split_block_hash_compute_timer() { - return _split_block_hash_compute_timer; - } RuntimeProfile::Counter* merge_block_timer() { return _merge_block_timer; } - RuntimeProfile::Counter* compress_timer() { return _compress_timer; } - RuntimeProfile::Counter* uncompressed_bytes_counter() { return _uncompressed_bytes_counter; } [[nodiscard]] bool transfer_large_data_by_brpc() const; bool is_finished() const override { return _reach_limit.load(); } void set_reach_limit() { _reach_limit = true; }; @@ -129,16 +115,13 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { std::unique_ptr _sink_buffer = nullptr; RuntimeProfile::Counter* _serialize_batch_timer = nullptr; RuntimeProfile::Counter* _compress_timer = nullptr; - RuntimeProfile::Counter* _brpc_send_timer = nullptr; - RuntimeProfile::Counter* _brpc_wait_timer = nullptr; RuntimeProfile::Counter* _bytes_sent_counter = nullptr; RuntimeProfile::Counter* _uncompressed_bytes_counter = nullptr; RuntimeProfile::Counter* _local_sent_rows = nullptr; RuntimeProfile::Counter* _local_send_timer = nullptr; RuntimeProfile::Counter* _split_block_hash_compute_timer = nullptr; - RuntimeProfile::Counter* _split_block_distribute_by_channel_timer = nullptr; + RuntimeProfile::Counter* _distribute_rows_into_channels_timer = nullptr; RuntimeProfile::Counter* _blocks_sent_counter = nullptr; - RuntimeProfile::Counter* _rows_sent_counter = nullptr; // Throughput per total time spent in sender RuntimeProfile::Counter* _overall_throughput = nullptr; // Used to counter send bytes under local data exchange @@ -153,6 +136,7 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { int _sender_id; std::shared_ptr _broadcast_pb_mem_limiter; + size_t _rpc_channels_num = 0; vectorized::BlockSerializer _serializer; std::shared_ptr _queue_dependency = nullptr; @@ -221,8 +205,6 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorXcreate_merger( local_state.vsort_exec_exprs.lhs_ordering_expr_ctxs(), _is_asc_order, _nulls_first, state->batch_size(), _limit, _offset)); local_state.is_ready = true; return Status::OK(); } - auto status = local_state.stream_recvr->get_next(block, eos); - RETURN_IF_ERROR(doris::vectorized::VExprContext::filter_block(local_state.conjuncts(), block, - block->columns())); + { + SCOPED_TIMER(local_state.get_data_from_recvr_timer); + RETURN_IF_ERROR(local_state.stream_recvr->get_next(block, eos)); + } + { + SCOPED_TIMER(local_state.filter_timer); + RETURN_IF_ERROR(doris::vectorized::VExprContext::filter_block(local_state.conjuncts(), + block, block->columns())); + } // In vsortrunmerger, it will set eos=true, and block not empty // so that eos==true, could not make sure that block not have valid data if (!*eos || block->rows() > 0) { @@ -176,7 +187,7 @@ Status ExchangeSourceOperatorX::get_block(RuntimeState* state, vectorized::Block local_state.set_num_rows_returned(_limit); } } - return status; + return Status::OK(); } Status ExchangeLocalState::close(RuntimeState* state) { diff --git a/be/src/pipeline/exec/exchange_source_operator.h b/be/src/pipeline/exec/exchange_source_operator.h index c8ef674d2698538..f938f5007d16430 100644 --- a/be/src/pipeline/exec/exchange_source_operator.h +++ b/be/src/pipeline/exec/exchange_source_operator.h @@ -59,6 +59,9 @@ class ExchangeLocalState final : public PipelineXLocalState<> { std::vector> deps; std::vector metrics; + RuntimeProfile::Counter* get_data_from_recvr_timer = nullptr; + RuntimeProfile::Counter* filter_timer = nullptr; + RuntimeProfile::Counter* create_merger_timer = nullptr; }; class ExchangeSourceOperatorX final : public OperatorX { diff --git a/be/src/pipeline/exec/result_file_sink_operator.cpp b/be/src/pipeline/exec/result_file_sink_operator.cpp index 93026427b86d56d..7c9c38ece5c4e94 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.cpp +++ b/be/src/pipeline/exec/result_file_sink_operator.cpp @@ -85,12 +85,6 @@ Status ResultFileSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& i SCOPED_TIMER(_init_timer); _sender_id = info.sender_id; - _brpc_wait_timer = ADD_TIMER(_profile, "BrpcSendTime.Wait"); - _local_send_timer = ADD_TIMER(_profile, "LocalSendTime"); - _brpc_send_timer = ADD_TIMER(_profile, "BrpcSendTime"); - _split_block_distribute_by_channel_timer = - ADD_TIMER(_profile, "SplitBlockDistributeByChannelTime"); - _brpc_send_timer = ADD_TIMER(_profile, "BrpcSendTime"); auto& p = _parent->cast(); CHECK(p._file_opts.get() != nullptr); // create sender diff --git a/be/src/pipeline/exec/result_file_sink_operator.h b/be/src/pipeline/exec/result_file_sink_operator.h index 7268efe4de40654..e9f2b8eeb9c6700 100644 --- a/be/src/pipeline/exec/result_file_sink_operator.h +++ b/be/src/pipeline/exec/result_file_sink_operator.h @@ -40,26 +40,12 @@ class ResultFileSinkLocalState final [[nodiscard]] int sender_id() const { return _sender_id; } - RuntimeProfile::Counter* brpc_wait_timer() { return _brpc_wait_timer; } - RuntimeProfile::Counter* local_send_timer() { return _local_send_timer; } - RuntimeProfile::Counter* brpc_send_timer() { return _brpc_send_timer; } - RuntimeProfile::Counter* merge_block_timer() { return _merge_block_timer; } - RuntimeProfile::Counter* split_block_distribute_by_channel_timer() { - return _split_block_distribute_by_channel_timer; - } - private: friend class ResultFileSinkOperatorX; std::shared_ptr _sender; std::shared_ptr _block_holder; - RuntimeProfile::Counter* _brpc_wait_timer = nullptr; - RuntimeProfile::Counter* _local_send_timer = nullptr; - RuntimeProfile::Counter* _brpc_send_timer = nullptr; - RuntimeProfile::Counter* _merge_block_timer = nullptr; - RuntimeProfile::Counter* _split_block_distribute_by_channel_timer = nullptr; - int _sender_id; }; diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index f04ace2e292595c..99c20e3c2e64429 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -46,7 +46,6 @@ Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) auto fragment_instance_id = state->fragment_instance_id(); _blocks_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "BlocksProduced", TUnit::UNIT, 1); - _rows_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "RowsProduced", TUnit::UNIT, 1); if (state->query_options().enable_parallel_result_sink) { _sender = _parent->cast()._sender; @@ -146,7 +145,7 @@ Status ResultSinkOperatorX::open(RuntimeState* state) { Status ResultSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, bool eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); - COUNTER_UPDATE(local_state.rows_sent_counter(), (int64_t)block->rows()); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); COUNTER_UPDATE(local_state.blocks_sent_counter(), 1); if (_fetch_option.use_two_phase_fetch && block->rows() > 0) { RETURN_IF_ERROR(_second_phase_fetch_data(state, block)); diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index 3c503096ecb51e8..ec5d480c38d1782 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -129,7 +129,6 @@ class ResultSinkLocalState final : public PipelineXSinkLocalState _sender = nullptr; std::shared_ptr _writer = nullptr; RuntimeProfile::Counter* _blocks_sent_counter = nullptr; - RuntimeProfile::Counter* _rows_sent_counter = nullptr; }; class ResultSinkOperatorX final : public DataSinkOperatorX { diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index ac820bcab2929a9..8460a62883dad91 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -238,14 +238,13 @@ Status BlockSerializer::next_serialized_block(Block* block, PBlock* dest, size_t } { + SCOPED_TIMER(_parent->merge_block_timer()); if (rows) { if (!rows->empty()) { - SCOPED_TIMER(_parent->split_block_distribute_by_channel_timer()); const auto* begin = rows->data(); RETURN_IF_ERROR(_mutable_block->add_rows(block, begin, begin + rows->size())); } } else if (!block->empty()) { - SCOPED_TIMER(_parent->merge_block_timer()); RETURN_IF_ERROR(_mutable_block->merge(*block)); } } diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index 88bb804fd8004f8..024d87ab32f49c7 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -86,6 +86,7 @@ class BlockSerializer { void reset_block() { _mutable_block.reset(); } void set_is_local(bool is_local) { _is_local = is_local; } + bool is_local() const { return _is_local; } private: pipeline::ExchangeSinkLocalState* _parent; From 1dbd1469b4a531bdc862a42dd4a95833f4eba5fe Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 31 Oct 2024 14:33:14 +0800 Subject: [PATCH 03/82] [Improvement](join) support single column nullable join hash table context (#42398) ## Proposed changes support single column nullable join hash table context ![QQ_1729828619996](https://github.com/user-attachments/assets/477353c6-b27b-4cb0-a9af-0773c70f4819) --- be/src/pipeline/common/join_utils.h | 41 ++----- be/src/pipeline/dependency.h | 3 +- be/src/pipeline/exec/hashjoin_build_sink.cpp | 101 +++++++----------- be/src/pipeline/exec/hashjoin_build_sink.h | 32 ++++-- .../pipeline/exec/hashjoin_probe_operator.cpp | 68 ++++-------- .../pipeline/exec/hashjoin_probe_operator.h | 2 - .../exec/join/process_hash_table_probe.h | 6 +- .../exec/join/process_hash_table_probe_impl.h | 47 +++----- be/src/vec/common/hash_table/hash_key_type.h | 14 +-- be/src/vec/common/hash_table/hash_map_util.h | 5 +- .../vec/common/hash_table/join_hash_table.h | 12 +-- .../nereids_p0/join/test_join_on.groovy | 2 +- 12 files changed, 123 insertions(+), 210 deletions(-) diff --git a/be/src/pipeline/common/join_utils.h b/be/src/pipeline/common/join_utils.h index 5be3e4af2f374bd..52c56abde1afef2 100644 --- a/be/src/pipeline/common/join_utils.h +++ b/be/src/pipeline/common/join_utils.h @@ -80,51 +80,22 @@ struct JoinDataVariants { method_variant.emplace(); break; case HashKeyType::int8_key: - if (nullable) { - method_variant.emplace>( - get_key_sizes(data_types)); - } else { - method_variant.emplace(); - } + method_variant.emplace(); break; case HashKeyType::int16_key: - if (nullable) { - method_variant.emplace>( - get_key_sizes(data_types)); - } else { - method_variant.emplace(); - } + method_variant.emplace(); break; case HashKeyType::int32_key: - if (nullable) { - method_variant.emplace>( - get_key_sizes(data_types)); - } else { - method_variant.emplace(); - } + method_variant.emplace(); break; case HashKeyType::int64_key: - if (nullable) { - method_variant.emplace>( - get_key_sizes(data_types)); - } else { - method_variant.emplace(); - } + method_variant.emplace(); break; case HashKeyType::int128_key: - if (nullable) { - method_variant.emplace>( - get_key_sizes(data_types)); - } else { - method_variant.emplace(); - } + method_variant.emplace(); break; case HashKeyType::int256_key: - if (nullable) { - method_variant.emplace(); - } else { - method_variant.emplace(); - } + method_variant.emplace(); break; case HashKeyType::string_key: method_variant.emplace(); diff --git a/be/src/pipeline/dependency.h b/be/src/pipeline/dependency.h index a035d57a8379ea0..c6100fe0d8b7cc3 100644 --- a/be/src/pipeline/dependency.h +++ b/be/src/pipeline/dependency.h @@ -606,8 +606,9 @@ struct HashJoinSharedState : public JoinSharedState { ENABLE_FACTORY_CREATOR(HashJoinSharedState) // mark the join column whether support null eq std::vector is_null_safe_eq_join; + // mark the build hash table whether it needs to store null value - std::vector store_null_in_hash_table; + std::vector serialize_null_into_key; std::shared_ptr arena = std::make_shared(); // maybe share hash table with other fragment instances diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 5ead4ba13a389c1..cbe9209eeb424dd 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -43,7 +43,7 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo _shared_state->join_op_variants = p._join_op_variants; _shared_state->is_null_safe_eq_join = p._is_null_safe_eq_join; - _shared_state->store_null_in_hash_table = p._store_null_in_hash_table; + _shared_state->serialize_null_into_key = p._serialize_null_into_key; _build_expr_ctxs.resize(p._build_expr_ctxs.size()); for (size_t i = 0; i < _build_expr_ctxs.size(); i++) { RETURN_IF_ERROR(p._build_expr_ctxs[i]->clone(state, _build_expr_ctxs[i])); @@ -227,33 +227,22 @@ Status HashJoinBuildSinkLocalState::_extract_join_column( vectorized::Block& block, vectorized::ColumnUInt8::MutablePtr& null_map, vectorized::ColumnRawPtrs& raw_ptrs, const std::vector& res_col_ids) { auto& shared_state = *_shared_state; - auto& p = _parent->cast(); for (size_t i = 0; i < shared_state.build_exprs_size; ++i) { - if (p._should_convert_to_nullable[i]) { + const auto* column = block.get_by_position(res_col_ids[i]).column.get(); + if (!column->is_nullable() && shared_state.serialize_null_into_key[i]) { _key_columns_holder.emplace_back( vectorized::make_nullable(block.get_by_position(res_col_ids[i]).column)); raw_ptrs[i] = _key_columns_holder.back().get(); - continue; - } - - if (shared_state.is_null_safe_eq_join[i]) { - raw_ptrs[i] = block.get_by_position(res_col_ids[i]).column.get(); + } else if (const auto* nullable = check_and_get_column(*column); + !shared_state.serialize_null_into_key[i] && nullable) { + // update nulllmap and split nested out of ColumnNullable when serialize_null_into_key is false and column is nullable + const auto& col_nested = nullable->get_nested_column(); + const auto& col_nullmap = nullable->get_null_map_data(); + DCHECK(null_map != nullptr); + vectorized::VectorizedUtils::update_null_map(null_map->get_data(), col_nullmap); + raw_ptrs[i] = &col_nested; } else { - const auto* column = block.get_by_position(res_col_ids[i]).column.get(); - if (const auto* nullable = check_and_get_column(*column)) { - const auto& col_nested = nullable->get_nested_column(); - const auto& col_nullmap = nullable->get_null_map_data(); - - if (shared_state.store_null_in_hash_table[i]) { - raw_ptrs[i] = nullable; - } else { - DCHECK(null_map != nullptr); - vectorized::VectorizedUtils::update_null_map(null_map->get_data(), col_nullmap); - raw_ptrs[i] = &col_nested; - } - } else { - raw_ptrs[i] = column; - } + raw_ptrs[i] = column; } } return Status::OK(); @@ -284,13 +273,9 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, .data()[0] = 1; } } - // TODO: Now we are not sure whether a column is nullable only by ExecNode's `row_desc` - // so we have to initialize this flag by the first build block. - if (!_has_set_need_null_map_for_build) { - _has_set_need_null_map_for_build = true; - _set_build_ignore_flag(block, _build_col_ids); - } - if (p._short_circuit_for_null_in_build_side || _build_side_ignore_null) { + + _set_build_side_has_external_nullmap(block, _build_col_ids); + if (_build_side_has_external_nullmap) { null_map_val = vectorized::ColumnUInt8::create(); null_map_val->get_data().assign(rows, (uint8_t)0); } @@ -300,27 +285,23 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, st = std::visit( vectorized::Overload { - [&](std::monostate& arg, auto join_op, auto has_null_value, + [&](std::monostate& arg, auto join_op, auto short_circuit_for_null_in_build_side, auto with_other_conjuncts) -> Status { LOG(FATAL) << "FATAL: uninited hash table"; __builtin_unreachable(); return Status::OK(); }, - [&](auto&& arg, auto&& join_op, auto has_null_value, - auto short_circuit_for_null_in_build_side, + [&](auto&& arg, auto&& join_op, auto short_circuit_for_null_in_build_side, auto with_other_conjuncts) -> Status { using HashTableCtxType = std::decay_t; using JoinOpType = std::decay_t; ProcessHashTableBuild hash_table_build_process( rows, raw_ptrs, this, state->batch_size(), state); auto st = hash_table_build_process.template run< - JoinOpType::value, has_null_value, - short_circuit_for_null_in_build_side, with_other_conjuncts>( - arg, - has_null_value || short_circuit_for_null_in_build_side - ? &null_map_val->get_data() - : nullptr, + JoinOpType::value, short_circuit_for_null_in_build_side, + with_other_conjuncts>( + arg, null_map_val ? &null_map_val->get_data() : nullptr, &_shared_state->_has_null_in_build_side); COUNTER_SET(_memory_used_counter, _build_blocks_memory_usage->value() + @@ -330,22 +311,24 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, return st; }}, _shared_state->hash_table_variants->method_variant, _shared_state->join_op_variants, - vectorized::make_bool_variant(_build_side_ignore_null), vectorized::make_bool_variant(p._short_circuit_for_null_in_build_side), vectorized::make_bool_variant((p._have_other_join_conjunct))); return st; } -void HashJoinBuildSinkLocalState::_set_build_ignore_flag(vectorized::Block& block, - const std::vector& res_col_ids) { +void HashJoinBuildSinkLocalState::_set_build_side_has_external_nullmap( + vectorized::Block& block, const std::vector& res_col_ids) { auto& p = _parent->cast(); + if (p._short_circuit_for_null_in_build_side) { + _build_side_has_external_nullmap = true; + return; + } for (size_t i = 0; i < _build_expr_ctxs.size(); ++i) { - if (!_shared_state->is_null_safe_eq_join[i] && !p._short_circuit_for_null_in_build_side) { - const auto* column = block.get_by_position(res_col_ids[i]).column.get(); - if (check_and_get_column(*column)) { - _build_side_ignore_null |= !_shared_state->store_null_in_hash_table[i]; - } + const auto* column = block.get_by_position(res_col_ids[i]).column.get(); + if (column->is_nullable() && !_shared_state->serialize_null_into_key[i]) { + _build_side_has_external_nullmap = true; + return; } } } @@ -359,7 +342,7 @@ Status HashJoinBuildSinkLocalState::_hash_table_init(RuntimeState* state) { /// For 'null safe equal' join, /// the build key column maybe be converted to nullable from non-nullable. - if (p._should_convert_to_nullable[i]) { + if (p._serialize_null_into_key[i]) { data_type = vectorized::make_nullable(data_type); } data_types.emplace_back(std::move(data_type)); @@ -393,10 +376,6 @@ Status HashJoinBuildSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* st _hash_output_slot_ids = tnode.hash_join_node.hash_output_slot_ids; } - const bool build_stores_null = _join_op == TJoinOp::RIGHT_OUTER_JOIN || - _join_op == TJoinOp::FULL_OUTER_JOIN || - _join_op == TJoinOp::RIGHT_ANTI_JOIN; - const std::vector& eq_join_conjuncts = tnode.hash_join_node.eq_join_conjuncts; for (const auto& eq_join_conjunct : eq_join_conjuncts) { vectorized::VExprContextSPtr build_ctx; @@ -430,16 +409,18 @@ Status HashJoinBuildSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* st (eq_join_conjunct.right.nodes[0].is_nullable || eq_join_conjunct.left.nodes[0].is_nullable); - const bool should_convert_to_nullable = is_null_safe_equal && - !eq_join_conjunct.right.nodes[0].is_nullable && - eq_join_conjunct.left.nodes[0].is_nullable; _is_null_safe_eq_join.push_back(is_null_safe_equal); - _should_convert_to_nullable.emplace_back(should_convert_to_nullable); - // if is null aware, build join column and probe join column both need dispose null value - _store_null_in_hash_table.emplace_back( - is_null_safe_equal || - (_build_expr_ctxs.back()->root()->is_nullable() && build_stores_null)); + if (eq_join_conjuncts.size() == 1) { + // single column key serialize method must use nullmap for represent null to instead serialize null into key + _serialize_null_into_key.emplace_back(false); + } else if (is_null_safe_equal) { + // use serialize null into key to represent multi column null value + _serialize_null_into_key.emplace_back(true); + } else { + // on normal conditions, because null!=null, it can be expressed directly with nullmap. + _serialize_null_into_key.emplace_back(false); + } } return Status::OK(); diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index 1d52feaccff5262..9f1cf486fe50271 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -56,7 +56,8 @@ class HashJoinBuildSinkLocalState final protected: Status _hash_table_init(RuntimeState* state); - void _set_build_ignore_flag(vectorized::Block& block, const std::vector& res_col_ids); + void _set_build_side_has_external_nullmap(vectorized::Block& block, + const std::vector& res_col_ids); Status _do_evaluate(vectorized::Block& block, vectorized::VExprContextSPtrs& exprs, RuntimeProfile::Counter& expr_call_timer, std::vector& res_col_ids); std::vector _convert_block_to_null(vectorized::Block& block); @@ -79,7 +80,6 @@ class HashJoinBuildSinkLocalState final vectorized::MutableBlock _build_side_mutable_block; std::shared_ptr _runtime_filter_slots; - bool _has_set_need_null_map_for_build = false; /* * The comparison result of a null value with any other value is null, @@ -87,7 +87,7 @@ class HashJoinBuildSinkLocalState final * the result of an equality condition involving null should be false, * so null does not need to be added to the hash table. */ - bool _build_side_ignore_null = false; + bool _build_side_has_external_nullmap = false; std::vector _build_col_ids; std::shared_ptr _finish_dependency; @@ -154,13 +154,11 @@ class HashJoinBuildSinkOperatorX final // build expr vectorized::VExprContextSPtrs _build_expr_ctxs; // mark the build hash table whether it needs to store null value - std::vector _store_null_in_hash_table; + std::vector _serialize_null_into_key; // mark the join column whether support null eq std::vector _is_null_safe_eq_join; - std::vector _should_convert_to_nullable; - bool _is_broadcast_join = false; std::shared_ptr _shared_hashtable_controller; @@ -184,11 +182,10 @@ struct ProcessHashTableBuild { _batch_size(batch_size), _state(state) {} - template + template Status run(HashTableContext& hash_table_ctx, vectorized::ConstNullMapPtr null_map, bool* has_null_key) { - if (short_circuit_for_null || ignore_null) { + if (null_map) { // first row is mocked and is null // TODO: Need to test the for loop. break may better for (uint32_t i = 1; i < _rows; i++) { @@ -208,8 +205,21 @@ struct ProcessHashTableBuild { hash_table_ctx.init_serialized_keys(_build_raw_ptrs, _rows, null_map ? null_map->data() : nullptr, true, true, hash_table_ctx.hash_table->get_bucket_size()); - hash_table_ctx.hash_table->template build( - hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(), _rows); + // only 2 cases need to access the null value in hash table + bool keep_null_key = false; + if ((JoinOpType == TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || + JoinOpType == TJoinOp::NULL_AWARE_LEFT_SEMI_JOIN) && + with_other_conjuncts) { + //null aware join with other conjuncts + keep_null_key = true; + } else if (_parent->_shared_state->is_null_safe_eq_join.size() == 1 && + _parent->_shared_state->is_null_safe_eq_join[0]) { + // single null safe eq + keep_null_key = true; + } + + hash_table_ctx.hash_table->build(hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(), + _rows, keep_null_key); hash_table_ctx.bucket_nums.resize(_batch_size); hash_table_ctx.bucket_nums.shrink_to_fit(); diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index a9d3c962b767e80..e7b784d4d77ab39 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -154,11 +154,9 @@ Status HashJoinProbeLocalState::close(RuntimeState* state) { bool HashJoinProbeLocalState::_need_probe_null_map(vectorized::Block& block, const std::vector& res_col_ids) { for (size_t i = 0; i < _probe_expr_ctxs.size(); ++i) { - if (!_shared_state->is_null_safe_eq_join[i]) { - const auto* column = block.get_by_position(res_col_ids[i]).column.get(); - if (check_and_get_column(*column)) { - return true; - } + const auto* column = block.get_by_position(res_col_ids[i]).column.get(); + if (column->is_nullable() && !_shared_state->serialize_null_into_key[i]) { + return true; } } return false; @@ -290,16 +288,14 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc if (local_state._probe_index < local_state._probe_block.rows()) { DCHECK(local_state._has_set_need_null_map_for_probe); std::visit( - [&](auto&& arg, auto&& process_hashtable_ctx, auto need_null_map_for_probe, - auto ignore_null) { + [&](auto&& arg, auto&& process_hashtable_ctx, auto need_judge_null) { using HashTableProbeType = std::decay_t; if constexpr (!std::is_same_v) { using HashTableCtxType = std::decay_t; if constexpr (!std::is_same_v) { - st = process_hashtable_ctx.template process( + st = process_hashtable_ctx.template process( arg, - need_null_map_for_probe + local_state._null_map_column ? &local_state._null_map_column->get_data() : nullptr, mutable_join_block, &temp_block, @@ -314,8 +310,8 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc }, local_state._shared_state->hash_table_variants->method_variant, *local_state._process_hashtable_ctx_variants, - vectorized::make_bool_variant(local_state._need_null_map_for_probe), - vectorized::make_bool_variant(local_state._shared_state->probe_ignore_null)); + vectorized::make_bool_variant(local_state._need_null_map_for_probe && + local_state._shared_state->probe_ignore_null)); } else if (local_state._probe_eos) { if (_is_right_semi_anti || (_is_outer_join && _join_op != TJoinOp::LEFT_OUTER_JOIN)) { std::visit( @@ -383,34 +379,22 @@ Status HashJoinProbeLocalState::_extract_join_column(vectorized::Block& block, } auto& shared_state = *_shared_state; - auto& p = _parent->cast(); for (size_t i = 0; i < shared_state.build_exprs_size; ++i) { - if (p._should_convert_to_nullable[i]) { + const auto* column = block.get_by_position(res_col_ids[i]).column.get(); + if (!column->is_nullable() && shared_state.serialize_null_into_key[i]) { _key_columns_holder.emplace_back( vectorized::make_nullable(block.get_by_position(res_col_ids[i]).column)); _probe_columns[i] = _key_columns_holder.back().get(); - continue; - } - - if (shared_state.is_null_safe_eq_join[i]) { - _probe_columns[i] = block.get_by_position(res_col_ids[i]).column.get(); + } else if (const auto* nullable = check_and_get_column(*column); + nullable && !shared_state.serialize_null_into_key[i]) { + // update nulllmap and split nested out of ColumnNullable when serialize_null_into_key is false and column is nullable + const auto& col_nested = nullable->get_nested_column(); + const auto& col_nullmap = nullable->get_null_map_data(); + DCHECK(_null_map_column != nullptr); + vectorized::VectorizedUtils::update_null_map(_null_map_column->get_data(), col_nullmap); + _probe_columns[i] = &col_nested; } else { - const auto* column = block.get_by_position(res_col_ids[i]).column.get(); - if (const auto* nullable = check_and_get_column(*column)) { - const auto& col_nested = nullable->get_nested_column(); - const auto& col_nullmap = nullable->get_null_map_data(); - - DCHECK(_null_map_column != nullptr); - vectorized::VectorizedUtils::update_null_map(_null_map_column->get_data(), - col_nullmap); - if (shared_state.store_null_in_hash_table[i]) { - _probe_columns[i] = nullable; - } else { - _probe_columns[i] = &col_nested; - } - } else { - _probe_columns[i] = column; - } + _probe_columns[i] = column; } } return Status::OK(); @@ -531,20 +515,6 @@ Status HashJoinProbeOperatorX::init(const TPlanNode& tnode, RuntimeState* state) null_aware || (_probe_expr_ctxs.back()->root()->is_nullable() && probe_dispose_null); conjuncts_index++; - const bool is_null_safe_equal = eq_join_conjunct.__isset.opcode && - (eq_join_conjunct.opcode == TExprOpcode::EQ_FOR_NULL) && - (eq_join_conjunct.right.nodes[0].is_nullable || - eq_join_conjunct.left.nodes[0].is_nullable); - - /// If it's right anti join, - /// we should convert the probe to nullable if the build side is nullable. - /// And if it is 'null safe equal', - /// we must make sure the build side and the probe side are both nullable or non-nullable. - const bool should_convert_to_nullable = - (is_null_safe_equal || _join_op == TJoinOp::RIGHT_ANTI_JOIN) && - !eq_join_conjunct.left.nodes[0].is_nullable && - eq_join_conjunct.right.nodes[0].is_nullable; - _should_convert_to_nullable.emplace_back(should_convert_to_nullable); } for (size_t i = 0; i < _probe_expr_ctxs.size(); ++i) { _probe_ignore_null |= !probe_not_ignore_null[i]; diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index 7da7a3b238d3b86..63673edc410fc15 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -178,8 +178,6 @@ class HashJoinProbeOperatorX final : public JoinProbeOperatorX _should_convert_to_nullable; - vectorized::DataTypes _right_table_data_types; vectorized::DataTypes _left_table_data_types; std::vector _hash_output_slot_ids; diff --git a/be/src/pipeline/exec/join/process_hash_table_probe.h b/be/src/pipeline/exec/join/process_hash_table_probe.h index 692b91f6a0120a8..739783eb1fd1ecb 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe.h @@ -55,7 +55,7 @@ struct ProcessHashTableProbe { int last_probe_index, bool all_match_one, bool have_other_join_conjunct); - template + template Status process(HashTableType& hash_table_ctx, ConstNullMapPtr null_map, vectorized::MutableBlock& mutable_block, vectorized::Block* output_block, uint32_t probe_rows, bool is_mark_join, bool have_other_join_conjunct); @@ -64,8 +64,8 @@ struct ProcessHashTableProbe { // the output block struct is same with mutable block. we can do more opt on it and simplify // the logic of probe // TODO: opt the visited here to reduce the size of hash table - template + template Status do_process(HashTableType& hash_table_ctx, ConstNullMapPtr null_map, vectorized::MutableBlock& mutable_block, vectorized::Block* output_block, uint32_t probe_rows); diff --git a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h index 7fc639b47a4d015..bf4325ccece042b 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h @@ -187,8 +187,8 @@ typename HashTableType::State ProcessHashTableProbe::_init_probe_sid } template -template +template Status ProcessHashTableProbe::do_process(HashTableType& hash_table_ctx, vectorized::ConstNullMapPtr null_map, vectorized::MutableBlock& mutable_block, @@ -206,8 +206,8 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash_table_c SCOPED_TIMER(_init_probe_side_timer); _init_probe_side( hash_table_ctx, probe_rows, with_other_conjuncts, - need_null_map_for_probe ? null_map->data() : nullptr, - need_null_map_for_probe && ignore_null && + null_map ? null_map->data() : nullptr, + need_judge_null && (JoinOpType == doris::TJoinOp::LEFT_ANTI_JOIN || JoinOpType == doris::TJoinOp::LEFT_SEMI_JOIN || JoinOpType == doris::TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN || @@ -255,14 +255,12 @@ Status ProcessHashTableProbe::do_process(HashTableType& hash_table_c } } else { SCOPED_TIMER(_search_hashtable_timer); - auto [new_probe_idx, new_build_idx, - new_current_offset] = hash_table_ctx.hash_table->template find_batch < JoinOpType, - with_other_conjuncts, is_mark_join, - need_null_map_for_probe && - ignore_null > (hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(), - probe_index, build_index, cast_set(probe_rows), - _probe_indexs.data(), _probe_visited, _build_indexs.data(), - has_mark_join_conjunct); + auto [new_probe_idx, new_build_idx, new_current_offset] = + hash_table_ctx.hash_table->template find_batch( + hash_table_ctx.keys, hash_table_ctx.bucket_nums.data(), probe_index, + build_index, cast_set(probe_rows), _probe_indexs.data(), + _probe_visited, _build_indexs.data(), has_mark_join_conjunct); probe_index = new_probe_idx; build_index = new_build_idx; current_offset = new_current_offset; @@ -675,7 +673,7 @@ Status ProcessHashTableProbe::process_data_in_hashtable( } template -template +template Status ProcessHashTableProbe::process(HashTableType& hash_table_ctx, vectorized::ConstNullMapPtr null_map, vectorized::MutableBlock& mutable_block, @@ -685,9 +683,9 @@ Status ProcessHashTableProbe::process(HashTableType& hash_table_ctx, Status res; std::visit( [&](auto is_mark_join, auto have_other_join_conjunct) { - res = do_process( - hash_table_ctx, null_map, mutable_block, output_block, probe_rows); + res = do_process(hash_table_ctx, null_map, mutable_block, + output_block, probe_rows); }, vectorized::make_bool_variant(is_mark_join), vectorized::make_bool_variant(have_other_join_conjunct)); @@ -703,27 +701,14 @@ struct ExtractType { }; #define INSTANTIATION(JoinOpType, T) \ - template Status \ - ProcessHashTableProbe::process::Type>( \ - ExtractType::Type & hash_table_ctx, vectorized::ConstNullMapPtr null_map, \ - vectorized::MutableBlock & mutable_block, vectorized::Block * output_block, \ - uint32_t probe_rows, bool is_mark_join, bool have_other_join_conjunct); \ - template Status \ - ProcessHashTableProbe::process::Type>( \ + template Status ProcessHashTableProbe::process::Type>( \ ExtractType::Type & hash_table_ctx, vectorized::ConstNullMapPtr null_map, \ vectorized::MutableBlock & mutable_block, vectorized::Block * output_block, \ uint32_t probe_rows, bool is_mark_join, bool have_other_join_conjunct); \ - template Status \ - ProcessHashTableProbe::process::Type>( \ - ExtractType::Type & hash_table_ctx, vectorized::ConstNullMapPtr null_map, \ - vectorized::MutableBlock & mutable_block, vectorized::Block * output_block, \ - uint32_t probe_rows, bool is_mark_join, bool have_other_join_conjunct); \ - template Status \ - ProcessHashTableProbe::process::Type>( \ + template Status ProcessHashTableProbe::process::Type>( \ ExtractType::Type & hash_table_ctx, vectorized::ConstNullMapPtr null_map, \ vectorized::MutableBlock & mutable_block, vectorized::Block * output_block, \ uint32_t probe_rows, bool is_mark_join, bool have_other_join_conjunct); \ - \ template Status \ ProcessHashTableProbe::process_data_in_hashtable::Type>( \ ExtractType::Type & hash_table_ctx, vectorized::MutableBlock & mutable_block, \ diff --git a/be/src/vec/common/hash_table/hash_key_type.h b/be/src/vec/common/hash_table/hash_key_type.h index 38802fe716711f4..2c14e4ab687f87b 100644 --- a/be/src/vec/common/hash_table/hash_key_type.h +++ b/be/src/vec/common/hash_table/hash_key_type.h @@ -97,16 +97,16 @@ inline HashKeyType get_hash_key_type(const std::vector& return HashKeyType::without_key; } - if (!data_types[0]->have_maximum_size_of_value()) { - if (is_string(data_types[0])) { + auto t = remove_nullable(data_types[0]); + // serialized cannot be used in the case of single column, because the join operator will have some processing of column nullable, resulting in incorrect serialized results. + if (!t->have_maximum_size_of_value()) { + if (is_string(t)) { return HashKeyType::string_key; - } else { - return HashKeyType::serialized; } + throw Exception(ErrorCode::INTERNAL_ERROR, "meet invalid type, type={}", t->get_name()); } - size_t size = - data_types[0]->get_maximum_size_of_value_in_memory() - data_types[0]->is_nullable(); + size_t size = t->get_maximum_size_of_value_in_memory(); if (size == sizeof(vectorized::UInt8)) { return HashKeyType::int8_key; } else if (size == sizeof(vectorized::UInt16)) { @@ -121,7 +121,7 @@ inline HashKeyType get_hash_key_type(const std::vector& return HashKeyType::int256_key; } else { throw Exception(ErrorCode::INTERNAL_ERROR, "meet invalid type size, size={}, type={}", size, - data_types[0]->get_name()); + t->get_name()); } } diff --git a/be/src/vec/common/hash_table/hash_map_util.h b/be/src/vec/common/hash_table/hash_map_util.h index 200e6372ea8ac4e..292e6307851c58d 100644 --- a/be/src/vec/common/hash_table/hash_map_util.h +++ b/be/src/vec/common/hash_table/hash_map_util.h @@ -33,8 +33,9 @@ inline std::vector get_data_types( template Status init_hash_method(DataVariants* data, const std::vector& data_types, bool is_first_phase) { - auto type = get_hash_key_type_with_phase(get_hash_key_type(data_types), !is_first_phase); + auto type = HashKeyType::EMPTY; try { + type = get_hash_key_type_with_phase(get_hash_key_type(data_types), !is_first_phase); if (has_nullable_key(data_types)) { data->template init(data_types, type); } else { @@ -48,7 +49,7 @@ Status init_hash_method(DataVariants* data, const std::vectormethod_variant.valueless_by_exception()); - if (type != HashKeyType::without_key && + if (type != HashKeyType::without_key && type != HashKeyType::EMPTY && data->method_variant.index() == 0) { // index is 0 means variant is monostate return Status::InternalError("method_variant init failed"); } diff --git a/be/src/vec/common/hash_table/join_hash_table.h b/be/src/vec/common/hash_table/join_hash_table.h index 485c5f7b3b22c9a..25ca8844cd280f2 100644 --- a/be/src/vec/common/hash_table/join_hash_table.h +++ b/be/src/vec/common/hash_table/join_hash_table.h @@ -71,20 +71,16 @@ class JoinHashTable { std::vector& get_visited() { return visited; } - template - void build(const Key* __restrict keys, const uint32_t* __restrict bucket_nums, - size_t num_elem) { + void build(const Key* __restrict keys, const uint32_t* __restrict bucket_nums, size_t num_elem, + bool keep_null_key) { build_keys = keys; for (size_t i = 1; i < num_elem; i++) { uint32_t bucket_num = bucket_nums[i]; next[i] = first[bucket_num]; first[bucket_num] = i; } - if constexpr ((JoinOpType != TJoinOp::NULL_AWARE_LEFT_ANTI_JOIN && - JoinOpType != TJoinOp::NULL_AWARE_LEFT_SEMI_JOIN) || - !with_other_conjuncts) { - /// Only null aware join with other conjuncts need to access the null value in hash table - first[bucket_size] = 0; // index = bucket_num means null + if (!keep_null_key) { + first[bucket_size] = 0; // index = bucket_size means null } } diff --git a/regression-test/suites/nereids_p0/join/test_join_on.groovy b/regression-test/suites/nereids_p0/join/test_join_on.groovy index 02b04479e986880..2415a3496cf7183 100644 --- a/regression-test/suites/nereids_p0/join/test_join_on.groovy +++ b/regression-test/suites/nereids_p0/join/test_join_on.groovy @@ -39,7 +39,7 @@ suite("test_join_on", "nereids_p0") { qt_sql """ select * from join_on order by k1; """ test { sql """ select * from join_on as j1 inner join join_on as j2 on j1.d_array = j2.d_array; """ - exception "Method get_max_row_byte_size is not supported for Array" + exception "meet invalid type, type=Array(Nullable(Int32))" } test { sql """ select * from join_on as j1 inner join join_on as j2 on j1.hll_col = j2.hll_col; """ From 4494b9c04d99a624efd6f901c775809b7db81b5b Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 31 Oct 2024 14:34:10 +0800 Subject: [PATCH 04/82] [Improvement](runtime-filter) enlarge merge_filter_callback timeout_ms (#42602) ## Proposed changes Sometimes we encounter [E1008]Reached timeout=1000ms, at this time, like other rf-related rpc requests, the timeout should be set larger. --- be/src/common/config.cpp | 4 ++-- be/src/common/config.h | 3 ++- be/src/exprs/runtime_filter.cpp | 15 +++++++++++---- be/src/pipeline/exec/exchange_sink_buffer.cpp | 4 ++-- be/src/runtime/runtime_filter_mgr.cpp | 15 +++++++++++---- be/src/runtime/runtime_state.h | 5 +++++ be/src/vec/sink/vdata_stream_sender.cpp | 2 +- 7 files changed, 34 insertions(+), 14 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index f7c17aefee8e054..2fdebbd09c2c802 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -540,7 +540,6 @@ DEFINE_mInt32(streaming_load_rpc_max_alive_time_sec, "1200"); DEFINE_Int32(tablet_writer_open_rpc_timeout_sec, "60"); // You can ignore brpc error '[E1011]The server is overcrowded' when writing data. DEFINE_mBool(tablet_writer_ignore_eovercrowded, "true"); -DEFINE_mBool(exchange_sink_ignore_eovercrowded, "true"); DEFINE_mInt32(slave_replica_writer_rpc_timeout_sec, "60"); // Whether to enable stream load record function, the default is false. // False: disable stream load record @@ -903,7 +902,8 @@ DEFINE_mInt64(small_column_size_buffer, "100"); // Perform the always_true check at intervals determined by runtime_filter_sampling_frequency DEFINE_mInt32(runtime_filter_sampling_frequency, "64"); - +DEFINE_mInt32(execution_max_rpc_timeout_sec, "3600"); +DEFINE_mBool(execution_ignore_eovercrowded, "true"); // cooldown task configs DEFINE_Int32(cooldown_thread_num, "5"); DEFINE_mInt64(generate_cooldown_task_interval_sec, "20"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 5f73c31dcdbe342..791ca0b5e1a3ab1 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -587,7 +587,6 @@ DECLARE_mInt32(streaming_load_rpc_max_alive_time_sec); DECLARE_Int32(tablet_writer_open_rpc_timeout_sec); // You can ignore brpc error '[E1011]The server is overcrowded' when writing data. DECLARE_mBool(tablet_writer_ignore_eovercrowded); -DECLARE_mBool(exchange_sink_ignore_eovercrowded); DECLARE_mInt32(slave_replica_writer_rpc_timeout_sec); // Whether to enable stream load record function, the default is false. // False: disable stream load record @@ -958,6 +957,8 @@ DECLARE_mInt64(big_column_size_buffer); DECLARE_mInt64(small_column_size_buffer); DECLARE_mInt32(runtime_filter_sampling_frequency); +DECLARE_mInt32(execution_max_rpc_timeout_sec); +DECLARE_mBool(execution_ignore_eovercrowded); // cooldown task configs DECLARE_Int32(cooldown_thread_num); diff --git a/be/src/exprs/runtime_filter.cpp b/be/src/exprs/runtime_filter.cpp index 84a964f5c3865ce..bd4cd3353b80683 100644 --- a/be/src/exprs/runtime_filter.cpp +++ b/be/src/exprs/runtime_filter.cpp @@ -1146,8 +1146,11 @@ Status IRuntimeFilter::send_filter_size(RuntimeState* state, uint64_t local_filt request->set_filter_size(local_filter_size); request->set_filter_id(_filter_id); - callback->cntl_->set_timeout_ms(std::min(3600, state->execution_timeout()) * 1000); - callback->cntl_->ignore_eovercrowded(); + + callback->cntl_->set_timeout_ms(get_execution_rpc_timeout_ms(state->execution_timeout())); + if (config::execution_ignore_eovercrowded) { + callback->cntl_->ignore_eovercrowded(); + } stub->send_filter_size(closure->cntl_.get(), closure->request_.get(), closure->response_.get(), closure.get()); @@ -1184,8 +1187,12 @@ Status IRuntimeFilter::push_to_remote(const TNetworkAddress* addr) { merge_filter_request->set_is_pipeline(true); auto column_type = _wrapper->column_type(); RETURN_IF_CATCH_EXCEPTION(merge_filter_request->set_column_type(to_proto(column_type))); - merge_filter_callback->cntl_->set_timeout_ms(wait_time_ms()); - merge_filter_callback->cntl_->ignore_eovercrowded(); + + merge_filter_callback->cntl_->set_timeout_ms( + get_execution_rpc_timeout_ms(_state->execution_timeout)); + if (config::execution_ignore_eovercrowded) { + merge_filter_callback->cntl_->ignore_eovercrowded(); + } if (get_ignored()) { merge_filter_request->set_filter_type(PFilterType::UNKNOW_FILTER); diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 016802f8f73bd8e..7163299d766f4e8 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -235,7 +235,7 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { auto send_callback = request.channel->get_send_callback(id, request.eos); send_callback->cntl_->set_timeout_ms(request.channel->_brpc_timeout_ms); - if (config::exchange_sink_ignore_eovercrowded) { + if (config::execution_ignore_eovercrowded) { send_callback->cntl_->ignore_eovercrowded(); } send_callback->addFailedHandler([&, weak_task_ctx = weak_task_exec_ctx()]( @@ -313,7 +313,7 @@ Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) { } auto send_callback = request.channel->get_send_callback(id, request.eos); send_callback->cntl_->set_timeout_ms(request.channel->_brpc_timeout_ms); - if (config::exchange_sink_ignore_eovercrowded) { + if (config::execution_ignore_eovercrowded) { send_callback->cntl_->ignore_eovercrowded(); } send_callback->addFailedHandler([&, weak_task_ctx = weak_task_exec_ctx()]( diff --git a/be/src/runtime/runtime_filter_mgr.cpp b/be/src/runtime/runtime_filter_mgr.cpp index 08a229c0ecf72b6..77d2097d20c0106 100644 --- a/be/src/runtime/runtime_filter_mgr.cpp +++ b/be/src/runtime/runtime_filter_mgr.cpp @@ -29,6 +29,7 @@ #include #include +#include "common/config.h" #include "common/logging.h" #include "common/status.h" #include "exprs/bloom_filter_func.h" @@ -343,8 +344,10 @@ Status RuntimeFilterMergeControllerEntity::send_filter_size(const PSendFilterSiz auto* pquery_id = closure->request_->mutable_query_id(); pquery_id->set_hi(_state->query_id.hi()); pquery_id->set_lo(_state->query_id.lo()); - closure->cntl_->set_timeout_ms(std::min(3600, _state->execution_timeout) * 1000); - closure->cntl_->ignore_eovercrowded(); + closure->cntl_->set_timeout_ms(get_execution_rpc_timeout_ms(_state->execution_timeout)); + if (config::execution_ignore_eovercrowded) { + closure->cntl_->ignore_eovercrowded(); + } closure->request_->set_filter_id(filter_id); closure->request_->set_filter_size(cnt_val->global_size); @@ -456,8 +459,12 @@ Status RuntimeFilterMergeControllerEntity::merge(const PMergeFilterRequest* requ if (has_attachment) { closure->cntl_->request_attachment().append(request_attachment); } - closure->cntl_->set_timeout_ms(std::min(3600, _state->execution_timeout) * 1000); - closure->cntl_->ignore_eovercrowded(); + + closure->cntl_->set_timeout_ms(get_execution_rpc_timeout_ms(_state->execution_timeout)); + if (config::execution_ignore_eovercrowded) { + closure->cntl_->ignore_eovercrowded(); + } + // set fragment-id if (target.__isset.target_fragment_ids) { for (auto& target_fragment_id : target.target_fragment_ids) { diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 34ce79ec7a749a9..abc823bc25b291d 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -38,6 +38,7 @@ #include "agent/be_exec_version_manager.h" #include "cctz/time_zone.h" #include "common/compiler_util.h" // IWYU pragma: keep +#include "common/config.h" #include "common/factory_creator.h" #include "common/status.h" #include "gutil/integral_types.h" @@ -51,6 +52,10 @@ namespace doris { class IRuntimeFilter; +inline int32_t get_execution_rpc_timeout_ms(int32_t execution_timeout_sec) { + return std::min(config::execution_max_rpc_timeout_sec, execution_timeout_sec) * 1000; +} + namespace pipeline { class PipelineXLocalStateBase; class PipelineXSinkLocalStateBase; diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index 8460a62883dad91..d21b87561b5da7d 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -95,7 +95,7 @@ Status Channel::open(RuntimeState* state) { } _be_number = state->be_number(); - _brpc_timeout_ms = std::min(3600, state->execution_timeout()) * 1000; + _brpc_timeout_ms = get_execution_rpc_timeout_ms(state->execution_timeout()); _serializer.set_is_local(_is_local); From ac6a8683ef374b41f21a8f743cddef8a7efe1d0e Mon Sep 17 00:00:00 2001 From: yujun Date: Thu, 31 Oct 2024 14:48:37 +0800 Subject: [PATCH 05/82] [fix](heartbeat) fix heartbeat editlog no persist hbTime (#42653) Backend persist lastUpdateMs, it will be modified by heartbeat editlog. But heartbeat editlog not persist hbTime, and hbTime always equal 0, it will make backend's lastUpdateMs = 0 in bdb image. fix details: 1. heartbeat response persist hbTime; 2. only be state change will write an editlog. but we make a change: even a backend is healthy, still write a healthy response editlog every 5 min. Inorder to make backend's lastUpdateMs periodly updated in bdb image. But notice that this change wouldn't increase real editlog num. Because heartbeat mgr will patch all fe/be's heartbeat into one editlog. Even no fe/be state change, it still write an editlog which not contains any node's response. 3. for a dead heartbeat response, set hbTime to last succ hbTime, then replayer can set correct lastUpdateMs; --- .../main/java/org/apache/doris/common/Config.java | 6 ++++++ .../main/java/org/apache/doris/system/Backend.java | 14 ++++++++++++++ .../org/apache/doris/system/BackendHbResponse.java | 10 ++-------- .../java/org/apache/doris/system/HeartbeatMgr.java | 4 ++-- .../org/apache/doris/system/HeartbeatResponse.java | 4 +++- 5 files changed, 27 insertions(+), 11 deletions(-) diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index f4e96bbd7a83661..dd0aca5923e74a4 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -2033,6 +2033,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = true) public static long max_backend_heartbeat_failure_tolerance_count = 1; + /** + * Even if a backend is healthy, still write a heartbeat editlog to update backend's lastUpdateMs of bdb image. + */ + @ConfField(mutable = true, masterOnly = true) + public static int editlog_healthy_heartbeat_seconds = 300; + /** * Abort transaction time after lost heartbeat. * The default value is 300s, which means transactions of be will be aborted after lost heartbeat 300s. diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java index da55ecee0de6038..974c0e0cae13a7c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java @@ -47,6 +47,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -153,6 +154,8 @@ public class Backend implements Writable { // send some queries to this BE, it is not an important problem. private AtomicBoolean isShutDown = new AtomicBoolean(false); + private long nextForceEditlogHeartbeatTime = System.currentTimeMillis() + (new SecureRandom()).nextInt(60 * 1000); + public Backend() { this.host = ""; this.version = ""; @@ -876,7 +879,18 @@ public boolean handleHbResponse(BackendHbResponse hbResponse, boolean isReplay) heartbeatErrMsg = ""; this.heartbeatFailureCounter = 0; + + // even if no change, write an editlog to make lastUpdateMs in image update + if (System.currentTimeMillis() >= this.nextForceEditlogHeartbeatTime) { + isChanged = true; + int delaySecond = Config.editlog_healthy_heartbeat_seconds + (new SecureRandom()).nextInt(60); + this.nextForceEditlogHeartbeatTime = System.currentTimeMillis() + delaySecond * 1000L; + } } else { + // for a bad BackendHbResponse, its hbTime is last succ hbTime, not this hbTime + if (hbResponse.getHbTime() > 0) { + this.lastUpdateMs = hbResponse.getHbTime(); + } // Only set backend to dead if the heartbeat failure counter exceed threshold. // And if it is a replay process, must set backend to dead. if (isReplay || ++this.heartbeatFailureCounter >= Config.max_backend_heartbeat_failure_tolerance_count) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/BackendHbResponse.java b/fe/fe-core/src/main/java/org/apache/doris/system/BackendHbResponse.java index a0311a9b7378479..479966d2ff3c8ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/BackendHbResponse.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/BackendHbResponse.java @@ -98,18 +98,12 @@ public BackendHbResponse(long beId, int bePort, int httpPort, int brpcPort, long this.beMemory = beMemory; } - public BackendHbResponse(long beId, String errMsg) { - super(HeartbeatResponse.Type.BACKEND); - this.status = HbStatus.BAD; - this.beId = beId; - this.msg = errMsg; - } - - public BackendHbResponse(long beId, String host, String errMsg) { + public BackendHbResponse(long beId, String host, long lastHbTime, String errMsg) { super(HeartbeatResponse.Type.BACKEND); this.status = HbStatus.BAD; this.beId = beId; this.host = host; + this.hbTime = lastHbTime; this.msg = errMsg; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java index 3fc09b31f2d3127..89f55239f7fb56a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java @@ -315,13 +315,13 @@ public HeartbeatResponse call() { System.currentTimeMillis(), beStartTime, version, nodeRole, fragmentNum, lastFragmentUpdateTime, isShutDown, arrowFlightSqlPort, beMemory); } else { - return new BackendHbResponse(backendId, backend.getHost(), + return new BackendHbResponse(backendId, backend.getHost(), backend.getLastUpdateMs(), result.getStatus().getErrorMsgs().isEmpty() ? "Unknown error" : result.getStatus().getErrorMsgs().get(0)); } } catch (Exception e) { LOG.warn("backend heartbeat got exception", e); - return new BackendHbResponse(backendId, backend.getHost(), + return new BackendHbResponse(backendId, backend.getHost(), backend.getLastUpdateMs(), Strings.isNullOrEmpty(e.getMessage()) ? "got exception" : e.getMessage()); } finally { if (client != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatResponse.java b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatResponse.java index 447ffad81899aab..3fffd1214503d5d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatResponse.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatResponse.java @@ -51,10 +51,12 @@ public enum HbStatus { protected boolean isTypeRead = false; /** - * msg and hbTime are no need to be synchronized to other Frontends, + * msg no need to be synchronized to other Frontends, * and only Master Frontend has these info */ protected String msg; + + @SerializedName(value = "hbTime") protected long hbTime; public HeartbeatResponse(Type type) { From 20e07b7a0ba590ee9e87033557912b4093fbd1a8 Mon Sep 17 00:00:00 2001 From: "Mingyu Chen (Rayner)" Date: Thu, 31 Oct 2024 15:07:29 +0800 Subject: [PATCH 06/82] [fix](export) remove export task executor in TransientTaskExecutor (#42880) (#42950) cherry-pick #42880 --- .../java/org/apache/doris/catalog/Env.java | 8 ---- .../java/org/apache/doris/load/ExportJob.java | 14 +++---- .../java/org/apache/doris/load/ExportMgr.java | 3 +- .../scheduler/disruptor/TaskHandler.java | 2 + .../manager/TransientTaskManager.java | 15 ++++++- .../registry/ExportTaskRegister.java | 40 ------------------- .../doris/analysis/CancelExportStmtTest.java | 1 - 7 files changed, 24 insertions(+), 59 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/ExportTaskRegister.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index ef23713a5010b49..cd0c0e80d8f27ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -254,7 +254,6 @@ import org.apache.doris.resource.workloadschedpolicy.WorkloadSchedPolicyMgr; import org.apache.doris.resource.workloadschedpolicy.WorkloadSchedPolicyPublisher; import org.apache.doris.scheduler.manager.TransientTaskManager; -import org.apache.doris.scheduler.registry.ExportTaskRegister; import org.apache.doris.service.ExecuteEnv; import org.apache.doris.service.FrontendOptions; import org.apache.doris.statistics.AnalysisManager; @@ -395,7 +394,6 @@ public class Env { private ExternalMetaIdMgr externalMetaIdMgr; private MetastoreEventsProcessor metastoreEventsProcessor; - private ExportTaskRegister exportTaskRegister; private JobManager, ?> jobManager; private LabelProcessor labelProcessor; private TransientTaskManager transientTaskManager; @@ -709,7 +707,6 @@ public Env(boolean isCheckpointCatalog) { this.jobManager = new JobManager<>(); this.labelProcessor = new LabelProcessor(); this.transientTaskManager = new TransientTaskManager(); - this.exportTaskRegister = new ExportTaskRegister(transientTaskManager); this.replayedJournalId = new AtomicLong(0L); this.stmtIdCounter = new AtomicLong(0L); @@ -4425,11 +4422,6 @@ public SyncJobManager getSyncJobManager() { return this.syncJobManager; } - - public ExportTaskRegister getExportTaskRegister() { - return exportTaskRegister; - } - public JobManager getJobManager() { return jobManager; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java index 33418531f2cda8f..5fe9c4826335909 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java @@ -98,7 +98,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; @Data @@ -207,9 +206,7 @@ public class ExportJob implements Writable { // backend_address => snapshot path private List> snapshotPaths = Lists.newArrayList(); - private List jobExecutorList; - - private ConcurrentHashMap taskIdToExecutor = new ConcurrentHashMap<>(); + private List jobExecutorList = Lists.newArrayList(); private Integer finishedTaskCount = 0; private List> allOutfileInfo = Lists.newArrayList(); @@ -690,11 +687,11 @@ private void cancelExportTask(ExportFailMsg.CancelType type, String msg) throws } // we need cancel all task - taskIdToExecutor.keySet().forEach(id -> { + jobExecutorList.forEach(executor -> { try { - Env.getCurrentEnv().getTransientTaskManager().cancelMemoryTask(id); + Env.getCurrentEnv().getTransientTaskManager().cancelMemoryTask(executor.getId()); } catch (JobException e) { - LOG.warn("cancel export task {} exception: {}", id, e); + LOG.warn("cancel export task {} exception: {}", executor.getId(), e); } }); @@ -705,6 +702,7 @@ private void cancelExportJobUnprotected(ExportFailMsg.CancelType type, String ms setExportJobState(ExportJobState.CANCELLED); finishTimeMs = System.currentTimeMillis(); failMsg = new ExportFailMsg(type, msg); + jobExecutorList.clear(); if (FeConstants.runningUnitTest) { return; } @@ -749,6 +747,8 @@ private void finishExportJobUnprotected() { setExportJobState(ExportJobState.FINISHED); finishTimeMs = System.currentTimeMillis(); outfileInfo = GsonUtils.GSON.toJson(allOutfileInfo); + // Clear the jobExecutorList to release memory. + jobExecutorList.clear(); Env.getCurrentEnv().getEditLog().logExportUpdateState(id, ExportJobState.FINISHED); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java index 7dbe953cf9bdbc0..5636f1aaad3e719 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java @@ -118,8 +118,7 @@ public void addExportJobAndRegisterTask(ExportJob job) throws Exception { job.getBrokerDesc()); } job.getTaskExecutors().forEach(executor -> { - Long taskId = Env.getCurrentEnv().getTransientTaskManager().addMemoryTask(executor); - job.getTaskIdToExecutor().put(taskId, executor); + Env.getCurrentEnv().getTransientTaskManager().addMemoryTask(executor); }); Env.getCurrentEnv().getEditLog().logExportCreate(job); } finally { diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java index de889c1b2e49d95..193f8ece9f7a2ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/disruptor/TaskHandler.java @@ -68,6 +68,8 @@ public void onTransientTaskHandle(TaskEvent taskEvent) { taskExecutor.execute(); } catch (JobException e) { log.warn("Memory task execute failed, taskId: {}, msg : {}", taskId, e.getMessage()); + } finally { + transientTaskManager.removeMemoryTask(taskId); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java index 51edd4af318bb01..7461399c8eb0c5a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/scheduler/manager/TransientTaskManager.java @@ -22,10 +22,13 @@ import org.apache.doris.scheduler.executor.TransientTaskExecutor; import lombok.Setter; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.concurrent.ConcurrentHashMap; public class TransientTaskManager { + private static final Logger LOG = LogManager.getLogger(TransientTaskManager.class); /** * key: taskId * value: memory task executor of this task @@ -57,10 +60,20 @@ public Long addMemoryTask(TransientTaskExecutor executor) { Long taskId = executor.getId(); taskExecutorMap.put(taskId, executor); disruptor.tryPublishTask(taskId); + LOG.info("add memory task, taskId: {}", taskId); return taskId; } public void cancelMemoryTask(Long taskId) throws JobException { - taskExecutorMap.get(taskId).cancel(); + try { + taskExecutorMap.get(taskId).cancel(); + } finally { + removeMemoryTask(taskId); + } + } + + public void removeMemoryTask(Long taskId) { + taskExecutorMap.remove(taskId); + LOG.info("remove memory task, taskId: {}", taskId); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/ExportTaskRegister.java b/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/ExportTaskRegister.java deleted file mode 100644 index 0241f57fea00269..000000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/scheduler/registry/ExportTaskRegister.java +++ /dev/null @@ -1,40 +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.scheduler.registry; - -import org.apache.doris.scheduler.exception.JobException; -import org.apache.doris.scheduler.executor.TransientTaskExecutor; -import org.apache.doris.scheduler.manager.TransientTaskManager; - -public class ExportTaskRegister implements TransientTaskRegister { - private final TransientTaskManager transientTaskManager; - - public ExportTaskRegister(TransientTaskManager transientTaskManager) { - this.transientTaskManager = transientTaskManager; - } - - @Override - public Long registerTask(TransientTaskExecutor executor) { - return transientTaskManager.addMemoryTask(executor); - } - - @Override - public void cancelTask(Long taskId) throws JobException { - transientTaskManager.cancelMemoryTask(taskId); - } -} diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelExportStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelExportStmtTest.java index 2d188230d8b5a88..4ff15653fa08429 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelExportStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CancelExportStmtTest.java @@ -234,7 +234,6 @@ public void testExportMgrCancelJob() throws UserException { exportMgr.unprotectAddJob(job3); exportMgr.unprotectAddJob(job4); - // cancel export job where state = "PENDING" Assert.assertTrue(job1.getState() == ExportJobState.PENDING); SlotRef stateSlotRef = new SlotRef(null, "state"); From 6f41abada0d10d7566523c6861ef8afccae4c337 Mon Sep 17 00:00:00 2001 From: shee <13843187+qzsee@users.noreply.github.com> Date: Thu, 31 Oct 2024 16:26:22 +0800 Subject: [PATCH 07/82] [feature](Nereids) support qualify stmt (#40048) like bigquery https://cloud.google.com/bigquery/docs/reference/standard-sql/query-syntax#qualify_clause Examples SELECT item, RANK() OVER (PARTITION BY category ORDER BY purchases DESC) as rank FROM Produce WHERE Produce.category = 'vegetable' QUALIFY rank <= 3 /*---------+------* | item | rank | +---------+------+ | kale | 1 | | lettuce | 2 | | cabbage | 3 | *---------+------*/ You don't have to include a window function in the SELECT list to use QUALIFY. The following query returns the most popular vegetables SELECT item FROM Produce WHERE Produce.category = 'vegetable' QUALIFY RANK() OVER (PARTITION BY category ORDER BY purchases DESC) <= 3 /*---------* | item | +---------+ | kale | | lettuce | | cabbage | *---------*/ Co-authored-by: garenshi --- .../org/apache/doris/nereids/DorisLexer.g4 | 1 + .../org/apache/doris/nereids/DorisParser.g4 | 6 + .../doris/nereids/jobs/executor/Analyzer.java | 4 + .../nereids/parser/LogicalPlanBuilder.java | 21 +- .../apache/doris/nereids/rules/RuleType.java | 8 + .../rules/analysis/BindExpression.java | 195 +++++++++++++ .../rules/analysis/FillUpMissingSlots.java | 7 +- .../analysis/FillUpQualifyMissingSlot.java | 272 ++++++++++++++++++ .../rules/analysis/QualifyToFilter.java | 34 +++ .../doris/nereids/trees/plans/PlanType.java | 1 + .../trees/plans/logical/LogicalQualify.java | 154 ++++++++++ .../trees/plans/visitor/PlanVisitor.java | 5 + .../nereids/parser/NereidsParserTest.java | 44 +++ .../analysis/FillUpMissingSlotsTest.java | 122 ++++++++ .../window_functions/test_qualify_query.out | 122 ++++++++ .../test_qualify_query.groovy | 120 ++++++++ 16 files changed, 1109 insertions(+), 7 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpQualifyMissingSlot.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/QualifyToFilter.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java create mode 100644 regression-test/data/query_p0/sql_functions/window_functions/test_qualify_query.out create mode 100644 regression-test/suites/query_p0/sql_functions/window_functions/test_qualify_query.groovy 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 47a45b67aa7b365..8ce8d033108367f 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 @@ -429,6 +429,7 @@ QUANTILE_STATE: 'QUANTILE_STATE'; QUANTILE_UNION: 'QUANTILE_UNION'; QUERY: 'QUERY'; QUOTA: 'QUOTA'; +QUALIFY: 'QUALIFY'; RANDOM: 'RANDOM'; RANGE: 'RANGE'; READ: 'READ'; 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 80da53a51bde331..acd139c010e968a 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 @@ -1117,6 +1117,7 @@ querySpecification whereClause? aggClause? havingClause? + qualifyClause? {doris_legacy_SQL_syntax}? queryOrganization #regularQuerySpecification ; @@ -1203,6 +1204,10 @@ havingClause : HAVING booleanExpression ; +qualifyClause + : QUALIFY booleanExpression + ; + selectHint: hintStatements+=hintStatement (COMMA? hintStatements+=hintStatement)* HINT_END; hintStatement @@ -2018,6 +2023,7 @@ nonReserved | QUANTILE_UNION | QUERY | QUOTA + | QUALIFY | RANDOM | RECENT | RECOVER diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index 6f6c022117c3378..894d42642015338 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -35,6 +35,7 @@ import org.apache.doris.nereids.rules.analysis.EliminateGroupByConstant; import org.apache.doris.nereids.rules.analysis.EliminateLogicalSelectHint; import org.apache.doris.nereids.rules.analysis.FillUpMissingSlots; +import org.apache.doris.nereids.rules.analysis.FillUpQualifyMissingSlot; import org.apache.doris.nereids.rules.analysis.HavingToFilter; import org.apache.doris.nereids.rules.analysis.LeadingJoin; import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; @@ -43,6 +44,7 @@ import org.apache.doris.nereids.rules.analysis.OneRowRelationExtractAggregate; import org.apache.doris.nereids.rules.analysis.ProjectToGlobalAggregate; import org.apache.doris.nereids.rules.analysis.ProjectWithDistinctToAggregate; +import org.apache.doris.nereids.rules.analysis.QualifyToFilter; import org.apache.doris.nereids.rules.analysis.ReplaceExpressionByChildOutput; import org.apache.doris.nereids.rules.analysis.SubqueryToApply; import org.apache.doris.nereids.rules.analysis.VariableToLiteral; @@ -125,6 +127,7 @@ private static List buildAnalyzerJobs(Optional topDown(new BindSink()), bottomUp(new CheckAfterBind()), bottomUp(new AddInitMaterializationHook()), + topDown(new FillUpQualifyMissingSlot()), bottomUp( new ProjectToGlobalAggregate(), // this rule check's the logicalProject node's isDistinct property @@ -165,6 +168,7 @@ private static List buildAnalyzerJobs(Optional topDown(new SimplifyAggGroupBy()), topDown(new NormalizeAggregate()), topDown(new HavingToFilter()), + topDown(new QualifyToFilter()), bottomUp(new SemiJoinCommute()), bottomUp( new CollectSubQueryAlias(), 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 a15966ed55f3be5..b5e6d928d6cbaf0 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 @@ -154,6 +154,7 @@ import org.apache.doris.nereids.DorisParser.PropertyKeyContext; import org.apache.doris.nereids.DorisParser.PropertyValueContext; import org.apache.doris.nereids.DorisParser.QualifiedNameContext; +import org.apache.doris.nereids.DorisParser.QualifyClauseContext; import org.apache.doris.nereids.DorisParser.QueryContext; import org.apache.doris.nereids.DorisParser.QueryOrganizationContext; import org.apache.doris.nereids.DorisParser.QueryTermContext; @@ -484,6 +485,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalQualify; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; import org.apache.doris.nereids.trees.plans.logical.LogicalSelectHint; import org.apache.doris.nereids.trees.plans.logical.LogicalSink; @@ -1459,7 +1461,8 @@ public LogicalPlan visitRegularQuerySpecification(RegularQuerySpecificationConte selectCtx, Optional.ofNullable(ctx.whereClause()), Optional.ofNullable(ctx.aggClause()), - Optional.ofNullable(ctx.havingClause())); + Optional.ofNullable(ctx.havingClause()), + Optional.ofNullable(ctx.qualifyClause())); selectPlan = withQueryOrganization(selectPlan, ctx.queryOrganization()); if ((selectHintMap == null) || selectHintMap.isEmpty()) { return selectPlan; @@ -3162,24 +3165,32 @@ protected LogicalPlan withSelectQuerySpecification( SelectClauseContext selectClause, Optional whereClause, Optional aggClause, - Optional havingClause) { + Optional havingClause, + Optional qualifyClause) { return ParserUtils.withOrigin(ctx, () -> { // from -> where -> group by -> having -> select LogicalPlan filter = withFilter(inputRelation, whereClause); SelectColumnClauseContext selectColumnCtx = selectClause.selectColumnClause(); LogicalPlan aggregate = withAggregate(filter, selectColumnCtx, aggClause); boolean isDistinct = (selectClause.DISTINCT() != null); + LogicalPlan selectPlan; if (!(aggregate instanceof Aggregate) && havingClause.isPresent()) { // create a project node for pattern match of ProjectToGlobalAggregate rule // then ProjectToGlobalAggregate rule can insert agg node as LogicalHaving node's child List projects = getNamedExpressions(selectColumnCtx.namedExpressionSeq()); LogicalPlan project = new LogicalProject<>(projects, isDistinct, aggregate); - return new LogicalHaving<>(ExpressionUtils.extractConjunctionToSet( + selectPlan = new LogicalHaving<>(ExpressionUtils.extractConjunctionToSet( getExpression((havingClause.get().booleanExpression()))), project); } else { LogicalPlan having = withHaving(aggregate, havingClause); - return withProjection(having, selectColumnCtx, aggClause, isDistinct); + selectPlan = withProjection(having, selectColumnCtx, aggClause, isDistinct); + } + // support qualify clause + if (qualifyClause.isPresent()) { + Expression qualifyExpr = getExpression(qualifyClause.get().booleanExpression()); + selectPlan = new LogicalQualify<>(Sets.newHashSet(qualifyExpr), selectPlan); } + return selectPlan; }); } @@ -3387,7 +3398,7 @@ public Object visitCommentRelationHint(CommentRelationHintContext ctx) { } protected LogicalPlan withProjection(LogicalPlan input, SelectColumnClauseContext selectCtx, - Optional aggCtx, boolean isDistinct) { + Optional aggCtx, boolean isDistinct) { return ParserUtils.withOrigin(selectCtx, () -> { if (aggCtx.isPresent()) { if (isDistinct) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index 08fbc9aafec1362..dbf96ef2f1f2fbc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -43,6 +43,9 @@ public enum RuleType { BINDING_USING_JOIN_SLOT(RuleTypeClass.REWRITE), BINDING_JOIN_SLOT(RuleTypeClass.REWRITE), BINDING_FILTER_SLOT(RuleTypeClass.REWRITE), + BINDING_QUALIFY_PROJECT_SLOT(RuleTypeClass.REWRITE), + BINDING_QUALIFY_AGGREGATE_SLOT(RuleTypeClass.REWRITE), + BINDING_QUALIFY_HAVING_SLOT(RuleTypeClass.REWRITE), BINDING_AGGREGATE_SLOT(RuleTypeClass.REWRITE), BINDING_REPEAT_SLOT(RuleTypeClass.REWRITE), BINDING_HAVING_SLOT(RuleTypeClass.REWRITE), @@ -67,11 +70,16 @@ public enum RuleType { FILL_UP_SORT_HAVING_PROJECT(RuleTypeClass.REWRITE), FILL_UP_SORT_HAVING_AGGREGATE(RuleTypeClass.REWRITE), FILL_UP_SORT_PROJECT(RuleTypeClass.REWRITE), + FILL_UP_QUALIFY_PROJECT(RuleTypeClass.REWRITE), + FILL_UP_QUALIFY_AGGREGATE(RuleTypeClass.REWRITE), + FILL_UP_QUALIFY_HAVING_AGGREGATE(RuleTypeClass.REWRITE), + FILL_UP_QUALIFY_HAVING_PROJECT(RuleTypeClass.REWRITE), RESOLVE_PROJECT_ALIAS(RuleTypeClass.REWRITE), RESOLVE_AGGREGATE_ALIAS(RuleTypeClass.REWRITE), PROJECT_TO_GLOBAL_AGGREGATE(RuleTypeClass.REWRITE), HAVING_TO_FILTER(RuleTypeClass.REWRITE), + QUALIFY_TO_FILTER(RuleTypeClass.REWRITE), ONE_ROW_RELATION_EXTRACT_AGGREGATE(RuleTypeClass.REWRITE), PROJECT_WITH_DISTINCT_TO_AGGREGATE(RuleTypeClass.REWRITE), AVG_DISTINCT_TO_SUM_DIV_COUNT(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index bebe2702cc5b005..9cdda51e4764fac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -80,6 +80,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalQualify; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; @@ -184,6 +185,15 @@ protected boolean condition(Rule rule, Plan plan) { RuleType.BINDING_HAVING_SLOT.build( logicalHaving(any().whenNot(Aggregate.class::isInstance)).thenApply(this::bindHaving) ), + RuleType.BINDING_QUALIFY_PROJECT_SLOT.build( + logicalQualify(logicalProject()).thenApply(this::bindQualifyProject) + ), + RuleType.BINDING_QUALIFY_AGGREGATE_SLOT.build( + logicalQualify(aggregate()).thenApply(this::bindQualifyAggregate) + ), + RuleType.BINDING_QUALIFY_HAVING_SLOT.build( + logicalQualify(logicalHaving()).thenApply(this::bindQualifyHaving) + ), RuleType.BINDING_INLINE_TABLE_SLOT.build( logicalInlineTable().thenApply(this::bindInlineTable) ), @@ -693,6 +703,191 @@ private Plan bindFilter(MatchingContext> ctx) { return new LogicalFilter<>(boundConjuncts.build(), filter.child()); } + /** + * there a dup table sales + * CREATE TABLE sales ( + * year INT, + * country STRING, + * product STRING, + * profit INT + * ) + * DISTRIBUTED BY HASH(`year`) + * PROPERTIES ( + * "replication_num" = "1" + * ); + * 1.qualify -> project + * for example : + * select year + 1 as year from sales qualify row_number() over (order by year, country) = 1; + * We are binding the year field of table sales. Instead of renaming year + * ----------------------------------------------------------------------------------------------------------------- + * 2.qualify -> project(distinct) + * for example: + * select distinct year + 1, country from sales qualify row_number() over (order by year + 1) > 1; + * We are binding the year field of table sales. + * ----------------------------------------------------------------------------------------------------------------- + * 3.qualify -> project(distinct) -> agg + * for example: + * select distinct year + 1 as year from sales group by year qualify row_number() over (order by year) = 1; + * We are binding the year field of group by output. Instead of renaming year + * ----------------------------------------------------------------------------------------------------------------- + * 4.qualify -> project(distinct) -> having -> agg + * for example: + * select distinct year,country from sales group by year,country having year > 2000 + * qualify row_number() over (order by year + 1) > 1; + * We are binding the year field of group output. + *----------------------------------------------------------------------------------------------------------------- + * Note: For the query without agg, we first bind slot from the child of the project. + * If it cannot be bound in the child, then bind slot from the project. + * If query with agg, we bind slot from the group by first. if not then bind slot from the group output + * or not bind slot from the agg child output finally. + */ + private Plan bindQualifyProject(MatchingContext>> ctx) { + LogicalQualify> qualify = ctx.root; + CascadesContext cascadesContext = ctx.cascadesContext; + LogicalProject project = qualify.child(); + ImmutableSet.Builder boundConjuncts = ImmutableSet.builderWithExpectedSize( + qualify.getConjuncts().size()); + if (project.child() instanceof Aggregate) { + Aggregate aggregate = (Aggregate) project.child(); + bindQualifyByAggregate(aggregate, cascadesContext, qualify, boundConjuncts); + } else if (project.child() instanceof LogicalHaving) { + LogicalHaving having = (LogicalHaving) project.child(); + if (having.child() instanceof Aggregate) { + Aggregate aggregate = (Aggregate) having.child(); + bindQualifyByAggregate(aggregate, cascadesContext, qualify, boundConjuncts); + } else { + throw new AnalysisException("unknown query structure"); + } + } else { + bindQualifyByProject(project, cascadesContext, qualify, boundConjuncts); + } + return new LogicalQualify<>(boundConjuncts.build(), qualify.child()); + } + + /** + * 1.qualify -> having -> agg + * for example: + * select country, sum(profit) as total, row_number() over (order by country) as rk from sales where year >= 2000 + * group by country having sum(profit) > 100 qualify rk = 1 + * We are binding the country field from group by. + * ----------------------------------------------------------------------------------------------------------------- + * 2.qualify -> having -> project + * for example: + * select year, country, profit, row_number() over (partition by year, country order by profit desc) as rk from + * (select * from sales) a where year >= 2000 having profit > 200 qualify rk = 1 order by profit,country limit 3 + * We are binding year/country/profit from sales + * ----------------------------------------------------------------------------------------------------------------- + * 3.qualify -> having -> project(distinct) + * for example: + * select distinct year + 1 as year from sales qualify row_number() over (order by year) = 1; + * we are binding year from sales. Instead of renaming year + */ + private Plan bindQualifyHaving(MatchingContext>> ctx) { + LogicalQualify> qualify = ctx.root; + CascadesContext cascadesContext = ctx.cascadesContext; + LogicalHaving having = qualify.child(); + ImmutableSet.Builder boundConjuncts = ImmutableSet.builderWithExpectedSize( + qualify.getConjuncts().size()); + if (having.child() instanceof Aggregate) { + bindQualifyByAggregate((Aggregate) having.child(), cascadesContext, qualify, + boundConjuncts); + } else { + bindQualifyByProject((LogicalProject) having.child(), cascadesContext, qualify, + boundConjuncts); + } + return new LogicalQualify<>(boundConjuncts.build(), qualify.child()); + } + + /** + * qualify -> agg + * for example: + * select country, sum(profit) as total, row_number() over (order by country) as rk from sales qualify rk > 1 + * we are binding the country field from group by. + */ + private Plan bindQualifyAggregate(MatchingContext>> ctx) { + LogicalQualify> qualify = ctx.root; + CascadesContext cascadesContext = ctx.cascadesContext; + Aggregate aggregate = qualify.child(); + ImmutableSet.Builder boundConjuncts = ImmutableSet.builderWithExpectedSize( + qualify.getConjuncts().size()); + bindQualifyByAggregate(aggregate, cascadesContext, qualify, boundConjuncts); + return new LogicalQualify<>(boundConjuncts.build(), qualify.child()); + } + + private void bindQualifyByProject(LogicalProject project, CascadesContext cascadesContext, + LogicalQualify qualify, + ImmutableSet.Builder boundConjuncts) { + Supplier defaultScope = Suppliers.memoize(() -> + toScope(cascadesContext, PlanUtils.fastGetChildrenOutputs(project.children())) + ); + Scope backupScope = toScope(cascadesContext, project.getOutput()); + + SimpleExprAnalyzer analyzer = buildCustomSlotBinderAnalyzer( + qualify, cascadesContext, defaultScope.get(), true, true, + (self, unboundSlot) -> { + List slots = self.bindSlotByScope(unboundSlot, defaultScope.get()); + if (!slots.isEmpty()) { + return slots; + } + return self.bindSlotByScope(unboundSlot, backupScope); + }); + + for (Expression conjunct : qualify.getConjuncts()) { + conjunct = analyzer.analyze(conjunct); + conjunct = TypeCoercionUtils.castIfNotSameType(conjunct, BooleanType.INSTANCE); + boundConjuncts.add(conjunct); + } + } + + private void bindQualifyByAggregate(Aggregate aggregate, CascadesContext cascadesContext, + LogicalQualify qualify, + ImmutableSet.Builder boundConjuncts) { + Supplier bindByAggChild = Suppliers.memoize(() -> { + Scope aggChildOutputScope + = toScope(cascadesContext, PlanUtils.fastGetChildrenOutputs(aggregate.children())); + return (analyzer, unboundSlot) -> analyzer.bindSlotByScope(unboundSlot, aggChildOutputScope); + }); + Scope aggOutputScope = toScope(cascadesContext, aggregate.getOutput()); + Supplier bindByGroupByThenAggOutputThenAggChildOutput = Suppliers.memoize(() -> { + List groupByExprs = aggregate.getGroupByExpressions(); + ImmutableList.Builder groupBySlots = ImmutableList.builderWithExpectedSize(groupByExprs.size()); + for (Expression groupBy : groupByExprs) { + if (groupBy instanceof Slot) { + groupBySlots.add((Slot) groupBy); + } + } + Scope groupBySlotsScope = toScope(cascadesContext, groupBySlots.build()); + + return (analyzer, unboundSlot) -> { + List boundInGroupBy = analyzer.bindSlotByScope(unboundSlot, groupBySlotsScope); + if (!boundInGroupBy.isEmpty()) { + return ImmutableList.of(boundInGroupBy.get(0)); + } + List boundInAggOutput = analyzer.bindSlotByScope(unboundSlot, aggOutputScope); + if (!boundInAggOutput.isEmpty()) { + return ImmutableList.of(boundInAggOutput.get(0)); + } + List expressions = bindByAggChild.get().bindSlot(analyzer, unboundSlot); + return expressions.isEmpty() ? expressions : ImmutableList.of(expressions.get(0)); + }; + }); + + ExpressionAnalyzer qualifyAnalyzer = new ExpressionAnalyzer(qualify, aggOutputScope, cascadesContext, + true, true) { + @Override + protected List bindSlotByThisScope(UnboundSlot unboundSlot) { + return bindByGroupByThenAggOutputThenAggChildOutput.get().bindSlot(this, unboundSlot); + } + }; + + ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(cascadesContext); + for (Expression expression : qualify.getConjuncts()) { + Expression boundConjunct = qualifyAnalyzer.analyze(expression, rewriteContext); + boundConjunct = TypeCoercionUtils.castIfNotSameType(boundConjunct, BooleanType.INSTANCE); + boundConjuncts.add(boundConjunct); + } + } + private List exceptStarSlots(Set boundExcepts, BoundStar boundStar) { List slots = boundStar.getSlots(); if (!boundExcepts.isEmpty()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlots.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlots.java index c55ed5957ba20c7..c392b85317dc283 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlots.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlots.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.Aggregate; @@ -245,7 +246,7 @@ public List buildRules() { having.withChildren(new LogicalProject<>(projects, project.child()))); } }) - ) + ) ); } @@ -316,6 +317,8 @@ public void resolve(Expression expression) { + expression.toSql() + "."); } generateAliasForNewOutputSlots(expression); + } else if (expression instanceof WindowExpression) { + generateAliasForNewOutputSlots(expression); } else { // Try to resolve the children. for (Expression child : expression.children()) { @@ -387,7 +390,7 @@ interface PlanGenerator { Plan apply(Resolver resolver, Aggregate aggregate); } - private Plan createPlan(Resolver resolver, Aggregate aggregate, PlanGenerator planGenerator) { + protected Plan createPlan(Resolver resolver, Aggregate aggregate, PlanGenerator planGenerator) { Aggregate newAggregate; if (resolver.getNewOutputSlots().isEmpty()) { newAggregate = aggregate; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpQualifyMissingSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpQualifyMissingSlot.java new file mode 100644 index 000000000000000..bb99883254729f2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/FillUpQualifyMissingSlot.java @@ -0,0 +1,272 @@ +// 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.rules.analysis; + +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.Aggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalQualify; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +/** + * We don't fill the missing slots in FillUpMissingSlots. + * Because for distinct queries, + * for example: + * select distinct year,country from sales having year > 2000 qualify row_number() over (order by year + 1) > 1; + * It would be converted into the form of agg. + * before logical plan: + * qualify + * | + * project(distinct) + * | + * scan + * apply ProjectWithDistinctToAggregate rule + * after logical plan: + * qualify + * | + * agg + * | + * scan + * if fill the missing slots in FillUpMissingSlots(after ProjectWithDistinctToAggregate). qualify could hardly be + * pushed under the agg of distinct. + * But apply FillUpQualifyMissingSlot rule before ProjectWithDistinctToAggregate + * logical plan: + * project(distinct) + * | + * qualify + * | + * project + * | + * scan + * and then apply ProjectWithDistinctToAggregate rule + * logical plan: + * agg + * | + * qualify + * | + * project + * | + * scan + * So it is easy to handle. + */ +public class FillUpQualifyMissingSlot extends FillUpMissingSlots { + @Override + public List buildRules() { + return ImmutableList.of( + /* + qualify -> project + qualify -> project(distinct) + qualify -> project(distinct) -> agg + qualify -> project(distinct) -> having -> agg + */ + RuleType.FILL_UP_QUALIFY_PROJECT.build( + logicalQualify(logicalProject()) + .then(qualify -> { + checkWindow(qualify); + LogicalProject project = qualify.child(); + return createPlan(project, qualify.getConjuncts(), (newConjuncts, projects) -> { + LogicalProject bottomProject = new LogicalProject<>(projects, project.child()); + LogicalQualify logicalQualify = new LogicalQualify<>(newConjuncts, bottomProject); + ImmutableList copyOutput = ImmutableList.copyOf(project.getOutput()); + return new LogicalProject<>(copyOutput, project.isDistinct(), logicalQualify); + }); + }) + ), + /* + qualify -> agg + */ + RuleType.FILL_UP_QUALIFY_AGGREGATE.build( + logicalQualify(aggregate()).then(qualify -> { + checkWindow(qualify); + Aggregate agg = qualify.child(); + Resolver resolver = new Resolver(agg); + qualify.getConjuncts().forEach(resolver::resolve); + return createPlan(resolver, agg, (r, a) -> { + Set newConjuncts = ExpressionUtils.replace( + qualify.getConjuncts(), r.getSubstitution()); + boolean notChanged = newConjuncts.equals(qualify.getConjuncts()); + if (notChanged && a.equals(agg)) { + return null; + } + return notChanged ? qualify.withChildren(a) : new LogicalQualify<>(newConjuncts, a); + }); + }) + ), + /* + qualify -> having -> agg + */ + RuleType.FILL_UP_QUALIFY_HAVING_AGGREGATE.build( + logicalQualify(logicalHaving(aggregate())).then(qualify -> { + checkWindow(qualify); + LogicalHaving> having = qualify.child(); + Aggregate agg = qualify.child().child(); + Resolver resolver = new Resolver(agg); + qualify.getConjuncts().forEach(resolver::resolve); + return createPlan(resolver, agg, (r, a) -> { + Set newConjuncts = ExpressionUtils.replace( + qualify.getConjuncts(), r.getSubstitution()); + boolean notChanged = newConjuncts.equals(qualify.getConjuncts()); + if (notChanged && a.equals(agg)) { + return null; + } + return notChanged ? qualify.withChildren(having.withChildren(a)) : + new LogicalQualify<>(newConjuncts, having.withChildren(a)); + }); + }) + ), + /* + qualify -> having -> project + qualify -> having -> project(distinct) + */ + RuleType.FILL_UP_QUALIFY_HAVING_PROJECT.build( + logicalQualify(logicalHaving(logicalProject())).then(qualify -> { + checkWindow(qualify); + LogicalHaving> having = qualify.child(); + LogicalProject project = qualify.child().child(); + return createPlan(project, qualify.getConjuncts(), (newConjuncts, projects) -> { + ImmutableList copyOutput = ImmutableList.copyOf(project.getOutput()); + if (project.isDistinct()) { + Set missingSlots = having.getExpressions().stream() + .map(Expression::getInputSlots) + .flatMap(Set::stream) + .filter(s -> !projects.contains(s)) + .collect(Collectors.toSet()); + List output = ImmutableList.builder() + .addAll(projects).addAll(missingSlots).build(); + LogicalQualify> logicalQualify = + new LogicalQualify<>(newConjuncts, new LogicalProject<>(output, project.child())); + return having.withChildren(project.withProjects(copyOutput).withChildren(logicalQualify)); + } else { + return new LogicalProject<>(copyOutput, new LogicalQualify<>(newConjuncts, + having.withChildren(project.withProjects(projects)))); + } + }); + }) + ) + ); + } + + interface PlanGenerator { + Plan apply(Set newConjuncts, List projects); + } + + private Plan createPlan(LogicalProject project, Set conjuncts, PlanGenerator planGenerator) { + Set projectOutputSet = project.getOutputSet(); + List newOutputSlots = Lists.newArrayList(); + Set newConjuncts = new HashSet<>(); + for (Expression conjunct : conjuncts) { + conjunct = conjunct.accept(new DefaultExpressionRewriter>() { + @Override + public Expression visitWindow(WindowExpression window, List context) { + Alias alias = new Alias(window); + context.add(alias); + return alias.toSlot(); + } + }, newOutputSlots); + newConjuncts.add(conjunct); + } + Set notExistedInProject = conjuncts.stream() + .map(Expression::getInputSlots) + .flatMap(Set::stream) + .filter(s -> !projectOutputSet.contains(s)) + .collect(Collectors.toSet()); + + newOutputSlots.addAll(notExistedInProject); + if (newOutputSlots.isEmpty()) { + return null; + } + List projects = ImmutableList.builder() + .addAll(project.getProjects()) + .addAll(newOutputSlots).build(); + + return planGenerator.apply(newConjuncts, projects); + } + + private void checkWindow(LogicalQualify qualify) throws AnalysisException { + Set inputSlots = new HashSet<>(); + AtomicBoolean hasWindow = new AtomicBoolean(false); + for (Expression conjunct : qualify.getConjuncts()) { + conjunct.accept(new DefaultExpressionVisitor>() { + @Override + public Void visitWindow(WindowExpression windowExpression, Set context) { + hasWindow.set(true); + return null; + } + + @Override + public Void visitSlotReference(SlotReference slotReference, Set context) { + context.add(slotReference); + return null; + } + + }, inputSlots); + } + if (hasWindow.get()) { + return; + } + qualify.accept(new DefaultPlanVisitor() { + private void findWindow(List namedExpressions) { + for (NamedExpression slot : namedExpressions) { + if (slot instanceof Alias && slot.child(0) instanceof WindowExpression) { + if (inputSlots.contains(slot.toSlot())) { + hasWindow.set(true); + } + } + } + } + + @Override + public Void visitLogicalProject(LogicalProject project, Void context) { + findWindow(project.getProjects()); + return visit(project, context); + } + + @Override + public Void visitLogicalAggregate(LogicalAggregate aggregate, Void context) { + findWindow(aggregate.getOutputExpressions()); + return visit(aggregate, context); + } + }, null); + if (!hasWindow.get()) { + throw new AnalysisException("qualify only used for window expression"); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/QualifyToFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/QualifyToFilter.java new file mode 100644 index 000000000000000..8bd933010acc84d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/QualifyToFilter.java @@ -0,0 +1,34 @@ +// 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.rules.analysis; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; + +/** + * qualify to filter. + */ +public class QualifyToFilter extends OneAnalysisRuleFactory { + @Override + public Rule build() { + return logicalQualify() + .then(qualify -> new LogicalFilter<>(qualify.getConjuncts(), qualify.child())) + .toRule(RuleType.QUALIFY_TO_FILTER); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 988ca381304f8e0..b87dfaf08ae4974 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -75,6 +75,7 @@ public enum PlanType { LOGICAL_MULTI_JOIN, LOGICAL_PARTITION_TOP_N, LOGICAL_PROJECT, + LOGICAL_QUALIFY, LOGICAL_REPEAT, LOGICAL_SELECT_HINT, LOGICAL_SUBQUERY_ALIAS, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java new file mode 100644 index 000000000000000..ced6730dfb54a20 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalQualify.java @@ -0,0 +1,154 @@ +// 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.plans.logical; + +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.DataTrait.Builder; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Filter; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** + * Logical qualify plan. + */ +public class LogicalQualify extends LogicalUnary implements Filter { + + private final Set conjuncts; + + public LogicalQualify(Set conjuncts, CHILD_TYPE child) { + this(conjuncts, Optional.empty(), Optional.empty(), child); + } + + private LogicalQualify(Set conjuncts, Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_QUALIFY, groupExpression, logicalProperties, child); + this.conjuncts = ImmutableSet.copyOf(Objects.requireNonNull(conjuncts, "conjuncts can not be null")); + } + + @Override + public Set getConjuncts() { + return conjuncts; + } + + @Override + public List computeOutput() { + return child().getOutput(); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new LogicalQualify<>(conjuncts, groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1); + return new LogicalQualify<>(conjuncts, groupExpression, logicalProperties, children.get(0)); + } + + public LogicalQualify withConjuncts(Set conjuncts) { + return new LogicalQualify<>(conjuncts, Optional.empty(), Optional.of(getLogicalProperties()), child()); + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalQualify[" + id.asInt() + "]", + "predicates", getPredicate() + ); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogicalQualify that = (LogicalQualify) o; + return conjuncts.equals(that.conjuncts); + } + + @Override + public int hashCode() { + return Objects.hash(conjuncts); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalQualify(this, context); + } + + @Override + public List getExpressions() { + return ImmutableList.copyOf(conjuncts); + } + + @Override + public LogicalQualify withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new LogicalQualify<>(conjuncts, children.get(0)); + } + + @Override + public void computeUnique(Builder builder) { + builder.addUniqueSlot(child(0).getLogicalProperties().getTrait()); + } + + @Override + public void computeUniform(Builder builder) { + for (Expression e : getConjuncts()) { + Set uniformSlots = ExpressionUtils.extractUniformSlot(e); + for (Slot slot : uniformSlots) { + builder.addUniformSlot(slot); + } + } + builder.addUniformSlot(child(0).getLogicalProperties().getTrait()); + } + + @Override + public void computeEqualSet(Builder builder) { + builder.addEqualSet(child().getLogicalProperties().getTrait()); + for (Expression expression : getConjuncts()) { + Optional> equalSlot = ExpressionUtils.extractEqualSlot(expression); + equalSlot.ifPresent(slotSlotPair -> builder.addEqualPair(slotSlotPair.first, slotSlotPair.second)); + } + } + + @Override + public void computeFd(Builder builder) { + builder.addFuncDepsDG(child().getLogicalProperties().getTrait()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java index 8db1407220cf083..396c6e4f26569f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java @@ -39,6 +39,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalQualify; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; import org.apache.doris.nereids.trees.plans.logical.LogicalSelectHint; @@ -169,6 +170,10 @@ public R visitLogicalFilter(LogicalFilter filter, C context) { return visit(filter, context); } + public R visitLogicalQualify(LogicalQualify filter, C context) { + return visit(filter, context); + } + public R visitLogicalGenerate(LogicalGenerate generate, C context) { return visit(generate, context); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java index c8a5364b7108281..ff9e81f2bf3cfb0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java @@ -47,6 +47,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.Set; @@ -660,4 +661,47 @@ public void testCreateRole() { String sql = "create role a comment 'create user'"; nereidsParser.parseSingle(sql); } + + @Test + public void testQualify() { + NereidsParser nereidsParser = new NereidsParser(); + + List sqls = new ArrayList<>(); + sqls.add("select year, country, profit, row_number() over (order by year) as rk from (select * from sales) a where year >= 2000 qualify rk > 1"); + sqls.add("select year, country, profit from (select * from sales) a where year >= 2000 qualify row_number() over (order by year) > 1"); + sqls.add("select year, country, profit from (select * from sales) a where year >= 2000 qualify rank() over (order by year) > 1"); + sqls.add("select year, country, profit from (select * from sales) a where year >= 2000 qualify dense_rank() over (order by year) > 1"); + + sqls.add("select country, sum(profit) as total, row_number() over (order by country) as rk from sales where year >= 2000 group by country having sum(profit) > 100 qualify rk = 1"); + sqls.add("select country, sum(profit) as total from sales where year >= 2000 group by country having sum(profit) > 100 qualify row_number() over (order by country) = 1"); + sqls.add("select country, sum(profit) as total from sales where year >= 2000 group by country having sum(profit) > 100 qualify rank() over (order by country) = 1"); + sqls.add("select country, sum(profit) as total from sales where year >= 2000 group by country having sum(profit) > 100 qualify dense_rank() over (order by country) = 1"); + + sqls.add("select country, sum(profit) as total, row_number() over (order by country) as rk from sales where year >= 2000 group by country qualify rk = 1"); + sqls.add("select country, sum(profit) as total from sales where year >= 2000 group by country qualify row_number() over (order by country) = 1"); + sqls.add("select country, sum(profit) as total from sales where year >= 2000 group by country qualify rank() over (order by country) = 1"); + sqls.add("select country, sum(profit) as total from sales where year >= 2000 group by country qualify dense_rank() over (order by country) = 1"); + + sqls.add("select year, country, product, profit, row_number() over (partition by year, country order by profit desc) as rk from sales where year >= 2000 qualify rk = 1 order by profit"); + sqls.add("select year, country, product, profit from sales where year >= 2000 qualify row_number() over (partition by year, country order by profit desc) = 1 order by profit"); + sqls.add("select year, country, product, profit from sales where year >= 2000 qualify rank() over (partition by year, country order by profit desc) = 1 order by profit"); + sqls.add("select year, country, product, profit from sales where year >= 2000 qualify dense_rank() over (partition by year, country order by profit desc) = 1 order by profit"); + + sqls.add("select year, country, profit, row_number() over (partition by year, country order by profit desc) as rk from (select * from sales) a where year >= 2000 having profit > 200 qualify rk = 1"); + sqls.add("select year, country, profit from (select * from sales) a where year >= 2000 having profit > 200 qualify row_number() over (partition by year, country order by profit desc) = 1"); + sqls.add("select year, country, profit from (select * from sales) a where year >= 2000 having profit > 200 qualify rank() over (partition by year, country order by profit desc) = 1"); + sqls.add("select year, country, profit from (select * from sales) a where year >= 2000 having profit > 200 qualify dense_rank() over (partition by year, country order by profit desc) = 1"); + + sqls.add("select distinct year, row_number() over (order by year) as rk from sales group by year qualify rk = 1"); + sqls.add("select distinct year from sales group by year qualify row_number() over (order by year) = 1"); + sqls.add("select distinct year from sales group by year qualify rank() over (order by year) = 1"); + sqls.add("select distinct year from sales group by year qualify dense_rank() over (order by year) = 1"); + + sqls.add("select year, country, profit from (select year, country, profit from (select year, country, profit, row_number() over (partition by year, country order by profit desc) as rk from (select * from sales) a where year >= 2000 having profit > 200) t where rk = 1) a where year >= 2000 qualify row_number() over (order by profit) = 1"); + sqls.add("select year, country, profit from (select year, country, profit from (select * from sales) a where year >= 2000 having profit > 200 qualify row_number() over (partition by year, country order by profit desc) = 1) a qualify row_number() over (order by profit) = 1"); + + for (String sql : sqls) { + nereidsParser.parseSingle(sql); + } + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java index 02f3caffa801634..31b135e7fbb970a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java @@ -72,6 +72,16 @@ public void runBeforeAll() throws Exception { + "DISTRIBUTED BY HASH (pk)\n" + "PROPERTIES(\n" + " 'replication_num' = '1'\n" + + ");", + "CREATE TABLE sales (\n" + + " year INT,\n" + + " country STRING,\n" + + " product STRING,\n" + + " profit INT\n" + + ") \n" + + "DISTRIBUTED BY HASH(`year`)\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + ");" ); } @@ -597,4 +607,116 @@ void testSortHaving() { PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new CheckAfterRewrite()); } + + @Test + void testQualify() { + connectContext.getSessionVariable().setDisableNereidsRules("ELIMINATE_AGG_ON_EMPTYRELATION"); + String sql = "select year + 1, country from sales where year >= 2000 qualify row_number() over (order by profit) > 1"; + PlanChecker.from(connectContext).analyze(sql).rewrite().matches( + logicalResultSink( + logicalProject( + logicalFilter( + logicalWindow( + logicalEmptyRelation()) + ).when(filter -> filter.toString().contains("predicates=(row_number() OVER(ORDER BY profit asc null first)#5 > 1)")) + ) + ) + ); + + sql = "select year + 1, country, row_number() over (order by year) as rk from sales where year >= 2000 qualify rk > profit"; + PlanChecker.from(connectContext).analyze(sql).rewrite().matches( + logicalResultSink( + logicalProject( + logicalFilter( + logicalWindow( + logicalEmptyRelation()) + ).when(filter -> filter.toString().contains("predicates=(rk#5 > cast(profit#3 as BIGINT))")) + ) + ) + ); + + sql = "select year + 1, country from sales where year >= 2000 group by year,country qualify rank() over (order by year) > 1"; + PlanChecker.from(connectContext).analyze(sql).rewrite().matches( + logicalResultSink( + logicalProject( + logicalFilter( + logicalWindow( + logicalProject( + logicalAggregate(logicalEmptyRelation()))) + ).when(filter -> filter.toString().contains("predicates=(rank() OVER(ORDER BY year asc null first)#5 > 1)")) + ) + ) + ); + + sql = "select year + 1, country, sum(profit) as total from sales where year >= 2000 group by year,country having sum(profit) > 100 qualify row_number() over (order by year) > 1"; + PlanChecker.from(connectContext).analyze(sql).rewrite().matches( + logicalResultSink( + logicalProject( + logicalFilter( + logicalWindow( + logicalProject( + logicalFilter( + logicalAggregate(logicalEmptyRelation()) + ).when(filter -> filter.toString().contains("predicates=(total#5 > 100)")) + ) + ) + ).when(filter -> filter.toString().contains("predicates=(row_number() OVER(ORDER BY year asc null first)#6 > 1)")) + ) + ) + ); + + sql = "select distinct year + 1,country from sales qualify row_number() over (order by profit + 1) > 1"; + PlanChecker.from(connectContext).analyze(sql).rewrite().matches( + logicalResultSink( + logicalAggregate( + logicalProject( + logicalFilter( + logicalWindow( + logicalEmptyRelation()) + ).when(filter -> filter.toString().contains("predicates=(row_number() OVER(ORDER BY (profit + 1) asc null first)#5 > 1)")) + ) + ) + ) + ); + + sql = "select distinct year + 1 as year,country from sales group by year, country qualify row_number() over (order by year) > 1"; + PlanChecker.from(connectContext).analyze(sql).rewrite().matches( + logicalResultSink( + logicalAggregate( + logicalProject( + logicalFilter( + logicalWindow( + logicalProject(logicalAggregate(logicalEmptyRelation()))) + ).when(filter -> filter.toString().contains("predicates=(row_number() OVER(ORDER BY year asc null first)#5 > 1)")) + ) + ) + ) + ); + + sql = "select distinct year,country,rank() over (order by year) from sales having sum(profit) > 100 qualify row_number() over (order by year) > 1"; + PlanChecker.from(connectContext).analyze(sql).rewrite().matches( + logicalResultSink( + logicalProject( + logicalFilter( + logicalAggregate( + logicalProject( + logicalFilter( + logicalWindow( + logicalEmptyRelation()) + ).when(filter -> filter.toString().contains("predicates=(row_number() OVER(ORDER BY year asc null first)#5 > 1)")) + ) + ) + ).when(filter -> filter.toString().contains("predicates=(sum(profit)#6 > 100)")) + ) + ) + ); + + ExceptionChecker.expectThrowsWithMsg( + AnalysisException.class, + "qualify only used for window expression", + () -> PlanChecker.from(connectContext).analyze( + "select year + 1, country from sales where year >= 2000 qualify year > 1" + ) + ); + } } diff --git a/regression-test/data/query_p0/sql_functions/window_functions/test_qualify_query.out b/regression-test/data/query_p0/sql_functions/window_functions/test_qualify_query.out new file mode 100644 index 000000000000000..29dac1331067d44 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/window_functions/test_qualify_query.out @@ -0,0 +1,122 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_1 -- +2001 Finland +2001 Finland +2001 India +2001 India +2001 India +2001 USA +2002 Finland +2002 USA +2002 USA +2002 USA +2002 USA +2002 USA + +-- !select_4 -- +2000 USA 1502 1 + +-- !select_5 -- +2000 Finland Computer 1501 1 +2000 India Computer 1201 1 +2000 USA Computer 1502 1 + +-- !select_6 -- +2000 Finland 1501 1 +2000 India 1201 1 +2000 USA 1502 1 +2001 USA 1503 1 + +-- !select_7 -- +2000 India 1201 + +-- !select_8 -- +2000 India 1201 + +-- !select_9 -- +Finland Phone 11 1 + +-- !select_10 -- +Finland Phone 11 + +-- !select_12 -- +2001 India 1201 1 +2001 Finland 1501 1 +2001 usa 1502 1 +2002 usa 1503 1 + +-- !select_13 -- +2001 India 1201 +2001 Finland 1501 +2001 usa 1502 +2002 usa 1503 + +-- !select_14 -- +2000 USA Computer 1502 +2001 USA Computer 1503 + +-- !select_15 -- +2000 USA Computer 1502 1 +2001 USA Computer 1503 1 + +-- !select_16 -- + +-- !select_17 -- + +-- !select_18 -- + +-- !select_19 -- + +-- !select_20 -- + +-- !select_21 -- +2001 Finland 10 1 +2001 USA 50 2 +2000 India 75 3 + +-- !select_22 -- +2001 Finland 10 1 + +-- !select_23 -- + +-- !select_24 -- +2001 Finland 1601 + +-- !select_25 -- +2000 Finland 1501 +2000 India 1201 +2000 USA 1502 +2001 Finland 10 +2001 USA 1503 + +-- !select_26 -- +2002 Finland + +-- !select_27 -- +2002 Finland 2 + +-- !select_28 -- +2002 USA 3006 + +-- !select_29 -- +2002 Finland + +-- !select_30 -- +2001 Finland 1 + +-- !select_31 -- +2001 Finland + +-- !select_32 -- + +-- !select_33 -- +2001 USA 3 + +-- !select_34 -- +2000 India 1 + +-- !select_35 -- +2001 Finland + +-- !select_36 -- +2001 Finland 6 diff --git a/regression-test/suites/query_p0/sql_functions/window_functions/test_qualify_query.groovy b/regression-test/suites/query_p0/sql_functions/window_functions/test_qualify_query.groovy new file mode 100644 index 000000000000000..04ed0cd74157fc4 --- /dev/null +++ b/regression-test/suites/query_p0/sql_functions/window_functions/test_qualify_query.groovy @@ -0,0 +1,120 @@ +// 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. +suite("test_qualify_query") { + sql "create database if not exists qualify_test" + sql "use qualify_test" + sql "DROP TABLE IF EXISTS sales" + sql """ + CREATE TABLE sales ( + year INT, + country STRING, + product STRING, + profit INT + ) + DISTRIBUTED BY HASH(`year`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1" + ) + """ + sql """ + INSERT INTO sales VALUES + (2000,'Finland','Computer',1501), + (2000,'Finland','Phone',100), + (2001,'Finland','Phone',10), + (2000,'India','Calculator',75), + (2000,'India','Calculator',76), + (2000,'India','Computer',1201), + (2000,'USA','Calculator',77), + (2000,'USA','Computer',1502), + (2001,'USA','Calculator',50), + (2001,'USA','Computer',1503), + (2001,'USA','Computer',1202), + (2001,'USA','TV',150), + (2001,'USA','TV',101); + """ + + qt_select_1 "select year + 1 as year, country from sales where year >= 2000 qualify row_number() over (order by year) > 1 order by year,country;" + + qt_select_4 "select year, country, profit, row_number() over (order by year) as rk from (select * from sales) a where year = 2000 qualify rk = 1;" + + qt_select_5 "select year, country, product, profit, row_number() over (partition by year, country order by profit desc) as rk from sales where year = 2000 qualify rk = 1 order by year, country, product, profit;" + + qt_select_6 "select year, country, profit, row_number() over (partition by year, country order by profit desc) as rk from (select * from sales) a where year >= 2000 having profit > 200 qualify rk = 1 order by year, country;" + + qt_select_7 "select year, country, profit from (select year, country, profit from (select year, country, profit, row_number() over (partition by year, country order by profit desc) as rk from (select * from sales) a where year >= 2000 having profit > 200) t where rk = 1) a where year >= 2000 qualify row_number() over (order by profit) = 1;" + + qt_select_8 "select year, country, profit from (select year, country, profit from (select * from sales) a where year >= 2000 having profit > 200 qualify row_number() over (partition by year, country order by profit desc) = 1) a qualify row_number() over (order by profit) = 1;" + + qt_select_9 "select * except(year) replace(profit+1 as profit), row_number() over (order by profit) as rk from sales where year >= 2000 qualify rk = 1;" + + qt_select_10 "select * except(year) replace(profit+1 as profit) from sales where year >= 2000 qualify row_number() over (order by year) > profit;" + + qt_select_12 "select year + 1, if(country = 'USA', 'usa' , country), case when profit < 200 then 200 else profit end as new_profit, row_number() over (partition by year, country order by profit desc) as rk from (select * from sales) a where year >= 2000 having profit > 200 qualify rk = 1 order by new_profit;" + + qt_select_13 "select year + 1, if(country = 'USA', 'usa' , country), case when profit < 200 then 200 else profit end as new_profit from (select * from sales) a where year >= 2000 having profit > 200 qualify row_number() over (partition by year, country order by profit desc) = 1 order by new_profit;" + + qt_select_14 "select * from sales where year >= 2000 qualify row_number() over (partition by year order by profit desc, country) = 1 order by country,profit;" + + qt_select_15 "select *,row_number() over (partition by year order by profit desc, country) as rk from sales where year >= 2000 qualify rk = 1 order by country,profit;" + + qt_select_16 "select * from sales where year >= 2000 qualify row_number() over (partition by year order by if(profit > 200, profit, profit+200) desc, country) = profit order by country;" + + qt_select_17 "select * from sales where year >= 2000 qualify row_number() over (partition by year order by case when profit > 200 then profit else profit+200 end desc, country) = profit order by country;" + + qt_select_18 "select distinct x.year, x.country, x.product from sales x left join sales y on x.year = y.year left join sales z on x.year = z.year where x.year >= 2000 qualify row_number() over (partition by x.year order by x.profit desc) = x.profit order by year;" + + qt_select_19 "select year, country, profit, row_number() over (order by profit) as rk1, row_number() over (order by country) as rk2 from (select * from sales) a where year >= 2000 qualify rk1 = 1 and rk2 > 2;" + + qt_select_20 "select year, country, profit, row_number() over (order by year) as rk from (select * from sales) a where year >= 2000 qualify rk + 1 > 1 * 100;" + + qt_select_21 "select year, country, profit, row_number() over (order by profit) as rk from (select * from sales) a where year >= 2000 qualify rk in (1,2,3);" + + qt_select_22 "select year, country, profit, row_number() over (order by profit) as rk from (select * from sales) a where year >= 2000 qualify rk = (select 1);" + + qt_select_23 "select year, country, profit, row_number() over (order by year) as rk from (select * from sales) a where year >= 2000 qualify rk = (select max(year) from sales);" + + qt_select_24 "select year+1, country, sum(profit) as total from sales where year >= 2000 and country = 'Finland' group by year,country having sum(profit) > 100 qualify row_number() over (order by year) = 1;" + + qt_select_25 "select year, country, profit from (select * from sales) a where year >= 2000 qualify row_number() over (partition by year, country order by profit desc) = 1 order by year, country, profit;" + + qt_select_26 "select year + 1, country from sales where year >= 2000 and country = 'Finland' group by year,country qualify row_number() over (order by year) > 1;" + + qt_select_27 "select year + 1, country, row_number() over (order by year) as rk from sales where year >= 2000 and country = 'Finland' group by year,country qualify rk > 1;" + + qt_select_28 "select year + 1, country, sum(profit) as total from sales where year >= 2000 group by year,country having sum(profit) > 1700 qualify row_number() over (order by year) = 1;" + + qt_select_29 "select distinct year + 1,country from sales qualify row_number() over (order by profit + 1) = 1;" + + qt_select_30 "select distinct year,country, row_number() over (order by profit + 1) as rk from sales qualify row_number() over (order by profit + 1) = 1;" + + qt_select_31 "select distinct year + 1 as year,country from sales where country = 'Finland' group by year, country qualify row_number() over (order by year) = 1;" + + qt_select_32 "select distinct year,country from sales having sum(profit) > 100 qualify row_number() over (order by year) > 100;" + + qt_select_33 "select distinct year,country,rank() over (order by year) from sales where country = 'USA' having sum(profit) > 100 qualify row_number() over (order by year) > 1;" + + qt_select_34 "select distinct year,country,rank() over (order by year) from sales where country = 'India' having sum(profit) > 100;" + + qt_select_35 "select year + 1, country from sales having profit >= 100 qualify row_number() over (order by profit) = 6;" + + qt_select_36 "select year + 1, country, row_number() over (order by profit) rk from sales having profit >= 100 qualify rk = 6;" +} + + + + + From 1713e26143a898c723fc1a969f7f261e520a18f2 Mon Sep 17 00:00:00 2001 From: minghong Date: Thu, 31 Oct 2024 16:41:21 +0800 Subject: [PATCH 08/82] [fix](nereids)add physical property in post processor (#42862) ## Proposed changes Post rule ProjectAggregateExpressionsForCse may add new Project node into plan, but does not set physical properties for the new project, and hence breaks the function of local shuffle. Issue Number: close #xxx --- .../ProjectAggregateExpressionsForCse.java | 11 +- .../dist_expr_list/dist_expr_list.out | 16 + .../dist_expr_list/dist_expr_list.groovy | 308 ++++++++++++++++++ 3 files changed, 334 insertions(+), 1 deletion(-) create mode 100644 regression-test/data/nereids_p0/dist_expr_list/dist_expr_list.out create mode 100644 regression-test/suites/nereids_p0/dist_expr_list/dist_expr_list.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ProjectAggregateExpressionsForCse.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ProjectAggregateExpressionsForCse.java index 00220a00ffd784c..a8038ab30b04aec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ProjectAggregateExpressionsForCse.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/ProjectAggregateExpressionsForCse.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.properties.DataTrait; import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -29,6 +30,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.Aggregate; +import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalPlan; import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; @@ -42,6 +44,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; /** @@ -118,8 +121,14 @@ public Plan visitPhysicalHashAggregate(PhysicalHashAggregate agg () -> projectOutput, () -> DataTrait.EMPTY_TRAIT ); - PhysicalProject project = new PhysicalProject<>(projections, + AbstractPhysicalPlan child = ((AbstractPhysicalPlan) aggregate.child()); + PhysicalProperties projectPhysicalProperties = new PhysicalProperties( + child.getPhysicalProperties().getDistributionSpec(), + child.getPhysicalProperties().getOrderSpec()); + PhysicalProject project = new PhysicalProject<>(projections, Optional.empty(), projectLogicalProperties, + projectPhysicalProperties, + child.getStats(), aggregate.child()); aggregate = (PhysicalHashAggregate) aggregate .withAggOutput(aggOutputReplaced) diff --git a/regression-test/data/nereids_p0/dist_expr_list/dist_expr_list.out b/regression-test/data/nereids_p0/dist_expr_list/dist_expr_list.out new file mode 100644 index 000000000000000..b0ee877195aef80 --- /dev/null +++ b/regression-test/data/nereids_p0/dist_expr_list/dist_expr_list.out @@ -0,0 +1,16 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !shuffle -- +2 2 2 2 1 2 2 2.00000000 4.00000000 109.20 109.20 109.20 1 +2 2 2 2 1 2 2 6.00000000 4.00000000 109.20 109.20 109.20 1 +2 2 2 2 3 2 2 2.00000000 4.00000000 109.20 109.20 109.20 1 +2 2 2 2 3 2 2 6.00000000 4.00000000 109.20 109.20 109.20 1 +3 3 3 3 1 3 3 3.00000000 9.00000000 99.50 99.50 99.50 1 +3 3 3 3 1 3 3 6.00000000 9.00000000 99.50 99.50 99.50 1 +3 3 3 3 1 3 3 9.00000000 9.00000000 99.50 99.50 99.50 1 +3 3 3 3 2 3 3 3.00000000 9.00000000 99.50 99.50 99.50 1 +3 3 3 3 2 3 3 6.00000000 9.00000000 99.50 99.50 99.50 1 +3 3 3 3 2 3 3 9.00000000 9.00000000 99.50 99.50 99.50 1 +3 3 3 3 3 3 3 3.00000000 9.00000000 99.50 99.50 99.50 1 +3 3 3 3 3 3 3 6.00000000 9.00000000 99.50 99.50 99.50 1 +3 3 3 3 3 3 3 9.00000000 9.00000000 99.50 99.50 99.50 1 + diff --git a/regression-test/suites/nereids_p0/dist_expr_list/dist_expr_list.groovy b/regression-test/suites/nereids_p0/dist_expr_list/dist_expr_list.groovy new file mode 100644 index 000000000000000..e211eeb142c1e07 --- /dev/null +++ b/regression-test/suites/nereids_p0/dist_expr_list/dist_expr_list.groovy @@ -0,0 +1,308 @@ +// 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. + +suite("dist_expr_list") { + sql """ + drop table if exists orders_1; + CREATE TABLE `orders_1` ( + `o_orderkey` BIGINT NULL, + `o_custkey` INT NULL, + `o_orderstatus` VARCHAR(1) NULL, + `o_totalprice` DECIMAL(15, 2) NULL, + `o_orderpriority` VARCHAR(15) NULL, + `o_clerk` VARCHAR(15) NULL, + `o_shippriority` INT NULL, + `o_comment` VARCHAR(79) NULL, + `o_orderdate` DATE not NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + drop table if exists lineitem_1; + CREATE TABLE `lineitem_1` ( + `l_orderkey` BIGINT NULL, + `l_linenumber` INT NULL, + `l_partkey` INT NULL, + `l_suppkey` INT NULL, + `l_quantity` DECIMAL(15, 2) NULL, + `l_extendedprice` DECIMAL(15, 2) NULL, + `l_discount` DECIMAL(15, 2) NULL, + `l_tax` DECIMAL(15, 2) NULL, + `l_returnflag` VARCHAR(1) NULL, + `l_linestatus` VARCHAR(1) NULL, + `l_commitdate` DATE NULL, + `l_receiptdate` DATE NULL, + `l_shipinstruct` VARCHAR(25) NULL, + `l_shipmode` VARCHAR(10) NULL, + `l_comment` VARCHAR(44) NULL, + `l_shipdate` DATE not NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + drop table if exists partsupp_1; + CREATE TABLE `partsupp_1` ( + `ps_partkey` INT NULL, + `ps_suppkey` INT NULL, + `ps_availqty` INT NULL, + `ps_supplycost` DECIMAL(15, 2) NULL, + `ps_comment` VARCHAR(199) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`ps_partkey`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`ps_partkey`) BUCKETS 24 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + insert into orders_1 values + (null, 1, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (1, null, 'o', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'k', 99.5, 'a', null, 1, 'yy', '2023-10-22'), + (1, 3, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-19'); + + insert into lineitem_1 values + (null, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (1, 1, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (3, 3, 3, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 2, 3, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (2, 1, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (3, 1, 3, 1, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 2, 1, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (2, 2, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (3, 3, 3, 3, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 1, 1, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); + + insert into partsupp_1 values + (1, 1, 1, 99.5, 'yy'), + (2, 2, 2, 109.2, 'mm'), + (3, 3, 1, 99.5, 'yy'), + (3, null, 1, 99.5, 'yy'); + """ + + sql """ + set enable_aggregate_cse=true; + set enable_local_shuffle=true; + """ + // test the query result should be the same when enable_local_shuffle=true or false + // set enable_local_shuffle=false, then generate result.out + // then set enable_local_shuffle=true, test the result is the same + def query = """ + select + t1.l_orderkey, + t2.l_partkey, + t1.l_suppkey, + t2.o_orderkey, + t1.o_custkey, + t2.ps_partkey, + t1.ps_suppkey, + t2.agg1, + t1.agg2, + t2.agg3, + t1.agg4, + t2.agg5, + t1.agg6 + from + ( + select + l_orderkey, + l_partkey, + l_suppkey, + o_orderkey, + o_custkey, + ps_partkey, + ps_suppkey, + t.agg1 as agg1, + t.sum_total as agg3, + t.max_total as agg4, + t.min_total as agg5, + t.count_all as agg6, + cast( + sum( + IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0) + ) as decimal(28, 8) + ) as agg2 + from + ( + select + l_orderkey, + l_partkey, + l_suppkey, + o_orderkey, + o_custkey, + cast( + sum( + IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0) + ) as decimal(28, 8) + ) as agg1, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union( + to_bitmap( + case when o_shippriority > 1 + and o_orderkey IN (1, 3) then o_custkey else null end + ) + ) cnt_1, + bitmap_union( + to_bitmap( + case when o_shippriority > 2 + and o_orderkey IN (2) then o_custkey else null end + ) + ) as cnt_2 + from + lineitem_1 + inner join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey + where + lineitem_1.l_shipdate >= "2023-10-17" + group by + l_orderkey, + l_partkey, + l_suppkey, + o_orderkey, + o_custkey + ) as t + inner join partsupp_1 on t.l_partkey = partsupp_1.ps_partkey + and t.l_suppkey = partsupp_1.ps_suppkey + where + partsupp_1.ps_suppkey > 1 + group by + l_orderkey, + l_partkey, + l_suppkey, + o_orderkey, + o_custkey, + ps_partkey, + ps_suppkey, + agg1, + agg3, + agg4, + agg5, + agg6 + ) as t1 + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + o_orderkey, + o_custkey, + ps_partkey, + ps_suppkey, + t.agg1 as agg1, + t.sum_total as agg3, + t.max_total as agg4, + t.min_total as agg5, + t.count_all as agg6, + cast( + sum( + IFNULL(ps_suppkey, 0) * IFNULL(ps_partkey, 0) + ) as decimal(28, 8) + ) as agg2 + from + ( + select + l_orderkey, + l_partkey, + l_suppkey, + o_orderkey, + o_custkey, + cast( + sum( + IFNULL(o_orderkey, 0) * IFNULL(o_custkey, 0) + ) as decimal(28, 8) + ) as agg1, + sum(o_totalprice) as sum_total, + max(o_totalprice) as max_total, + min(o_totalprice) as min_total, + count(*) as count_all, + bitmap_union( + to_bitmap( + case when o_shippriority > 1 + and o_orderkey IN (1, 3) then o_custkey else null end + ) + ) cnt_1, + bitmap_union( + to_bitmap( + case when o_shippriority > 2 + and o_orderkey IN (2) then o_custkey else null end + ) + ) as cnt_2 + from + lineitem_1 + inner join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey + where + lineitem_1.l_shipdate >= "2023-10-17" + group by + l_orderkey, + l_partkey, + l_suppkey, + o_orderkey, + o_custkey + ) as t + inner join partsupp_1 on t.l_partkey = partsupp_1.ps_partkey + and t.l_suppkey = partsupp_1.ps_suppkey + where + partsupp_1.ps_suppkey > 1 + group by + l_orderkey, + l_partkey, + l_suppkey, + o_orderkey, + o_custkey, + ps_partkey, + ps_suppkey, + agg1, + agg3, + agg4, + agg5, + agg6 + ) as t2 on t1.l_orderkey = t2.l_orderkey + where + t1.l_orderkey > 1 + group by + t1.l_orderkey, + t2.l_partkey, + t1.l_suppkey, + t2.o_orderkey, + t1.o_custkey, + t2.ps_partkey, + t1.ps_suppkey, + t2.agg1, + t1.agg2, + t2.agg3, + t1.agg4, + t2.agg5, + t1.agg6 + order by 1, 2, 3, 4, 5, 6,7, 8, 9; + """ + order_qt_shuffle "${query}" +} \ No newline at end of file From bf8b3ad1c2d2c4ac883cbe1a71c11a352a3ab02d Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Thu, 31 Oct 2024 17:08:56 +0800 Subject: [PATCH 09/82] [fix](case)Fix multiple case tables with the same name causing case failure (#42203) Fix multiple case tables with the same name causing case failure --- .../suites/nereids_p0/union/test_union.groovy | 12 +++++++----- .../suites/query_p0/union/test_union.groovy | 12 +++++++----- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/regression-test/suites/nereids_p0/union/test_union.groovy b/regression-test/suites/nereids_p0/union/test_union.groovy index 5e9a9e71bf38b05..2d6f84235fe7275 100644 --- a/regression-test/suites/nereids_p0/union/test_union.groovy +++ b/regression-test/suites/nereids_p0/union/test_union.groovy @@ -21,6 +21,8 @@ suite("test_union") { SET enable_fallback_to_original_planner=false; set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; """ + String suiteName = "nereids_union_test_union" + String viewName = "${suiteName}_view" def db = "nereids_test_query_db" sql "use ${db}" @@ -177,14 +179,14 @@ suite("test_union") { // test_union_bug // PALO-3617 qt_union36 """select * from (select 1 as a, 2 as b union select 3, 3) c where a = 1""" - sql """drop view if exists nullable""" - sql """CREATE VIEW `nullable` AS SELECT `a`.`k1` AS `n1`, `b`.`k2` AS `n2` + sql """drop view if exists ${viewName}""" + sql """CREATE VIEW `${viewName}` AS SELECT `a`.`k1` AS `n1`, `b`.`k2` AS `n2` FROM `${db}`.`baseall` a LEFT OUTER JOIN `${db}`.`bigtable` b ON `a`.`k1` = `b`.`k1` + 10 WHERE `b`.`k2` IS NULL""" - order_qt_union37 """select n1 from nullable union all select n2 from nullable""" - qt_union38 """(select n1 from nullable) union all (select n2 from nullable order by n1) order by n1""" - qt_union39 """(select n1 from nullable) union all (select n2 from nullable) order by n1""" + order_qt_union37 """select n1 from ${viewName} union all select n2 from ${viewName}""" + qt_union38 """(select n1 from ${viewName}) union all (select n2 from ${viewName} order by n1) order by n1""" + qt_union39 """(select n1 from ${viewName}) union all (select n2 from ${viewName}) order by n1""" // test_union_different_column diff --git a/regression-test/suites/query_p0/union/test_union.groovy b/regression-test/suites/query_p0/union/test_union.groovy index f6b9f1e329b1d84..87523ff81cc2d1e 100644 --- a/regression-test/suites/query_p0/union/test_union.groovy +++ b/regression-test/suites/query_p0/union/test_union.groovy @@ -16,6 +16,8 @@ // under the License. suite("test_union") { + String suiteName = "query_union_test_union" + String viewName = "${suiteName}_view" def db = "test_query_db" sql "use ${db}" @@ -172,14 +174,14 @@ suite("test_union") { // test_union_bug // PALO-3617 qt_union36 """select * from (select 1 as a, 2 as b union select 3, 3) c where a = 1""" - sql """drop view if exists nullable""" - sql """CREATE VIEW `nullable` AS SELECT `a`.`k1` AS `n1`, `b`.`k2` AS `n2` + sql """drop view if exists ${viewName}""" + sql """CREATE VIEW `${viewName}` AS SELECT `a`.`k1` AS `n1`, `b`.`k2` AS `n2` FROM `${db}`.`baseall` a LEFT OUTER JOIN `${db}`.`bigtable` b ON `a`.`k1` = `b`.`k1` + 10 WHERE `b`.`k2` IS NULL""" - order_qt_union37 """select n1 from nullable union all select n2 from nullable""" - qt_union38 """(select n1 from nullable) union all (select n2 from nullable order by n1) order by n1""" - qt_union39 """(select n1 from nullable) union all (select n2 from nullable) order by n1""" + order_qt_union37 """select n1 from ${viewName} union all select n2 from ${viewName}""" + qt_union38 """(select n1 from ${viewName}) union all (select n2 from ${viewName} order by n1) order by n1""" + qt_union39 """(select n1 from ${viewName}) union all (select n2 from ${viewName}) order by n1""" // test_union_different_column From fb7d0301d554f18221df2b0943a984e48b606135 Mon Sep 17 00:00:00 2001 From: seawinde <149132972+seawinde@users.noreply.github.com> Date: Thu, 31 Oct 2024 17:17:24 +0800 Subject: [PATCH 10/82] [fix](nereids) Fix not check when duplicated column names are in cte select list (#42400) This is brought by #21727 If you run query as following: Should throw analyze exception, because the duplicated column name in `tmp1` but now not. this pr fix this with tmp1 as ( select unique_id, unique_id from test_table ) select * from tmp1; --- .../nereids/rules/analysis/AnalyzeCTE.java | 2 + .../rules/analysis/BindExpression.java | 6 +- .../test_cte_with_duplicate_consumer.groovy | 14 ++- .../query_p0/cte/query_with_dup_column.groovy | 112 ++++++++++++++++++ 4 files changed, 131 insertions(+), 3 deletions(-) create mode 100644 regression-test/suites/query_p0/cte/query_with_dup_column.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java index 129b0860a74ee49..36a0459375b840b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java @@ -101,6 +101,8 @@ private Pair>> analyzeCte( CTEId cteId = StatementScopeIdGenerator.newCTEId(); LogicalSubQueryAlias logicalSubQueryAlias = aliasQuery.withChildren(ImmutableList.of(analyzedCtePlan)); + BindExpression.checkSameNameSlot(logicalSubQueryAlias.child(0).getOutput(), + logicalSubQueryAlias.getAlias()); outerCteCtx = new CTEContext(cteId, logicalSubQueryAlias, outerCteCtx); outerCteCtx.setAnalyzedPlan(logicalSubQueryAlias); cteProducerPlans.add(new LogicalCTEProducer<>(cteId, logicalSubQueryAlias)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 9cdda51e4764fac..cde659b91021ad8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -1142,7 +1142,11 @@ private LogicalTVFRelation bindTableValuedFunction(MatchingContext childOutputs, String subQueryAlias) { + /** + * Check the slot in childOutputs is duplicated or not + * If childOutputs has duplicated column name, would throw analysis exception + */ + public static void checkSameNameSlot(List childOutputs, String subQueryAlias) { Set nameSlots = new HashSet<>(childOutputs.size() * 2); for (Slot s : childOutputs) { if (!nameSlots.add(s.getInternalName())) { diff --git a/regression-test/suites/nereids_p0/cte/test_cte_with_duplicate_consumer.groovy b/regression-test/suites/nereids_p0/cte/test_cte_with_duplicate_consumer.groovy index 4064efcfc6e2e9b..b5cb03ea1744053 100644 --- a/regression-test/suites/nereids_p0/cte/test_cte_with_duplicate_consumer.groovy +++ b/regression-test/suites/nereids_p0/cte/test_cte_with_duplicate_consumer.groovy @@ -14,12 +14,22 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. +import org.junit.Assert; suite("test_cte_with_duplicate_consumer") { - test { + try { sql """ WITH cte1(col1) AS (SELECT 1), cte2(col2_1, col2_2) AS (SELECT col1, col1 FROM cte1) SELECT * FROM cte2 """ + } catch (Exception e) { + // Duplicated inline view column alias: 'col1' in inline view: 'cte2'' + assertTrue(e.message.contains(" Duplicated inline view column alias")) + } + + test { + sql """ + WITH cte1(col1) AS (SELECT 1), cte2(col2_1) AS (SELECT col1 FROM cte1) SELECT * FROM cte2 + """ - result([[1, 1]]) + result([[1]]) } } diff --git a/regression-test/suites/query_p0/cte/query_with_dup_column.groovy b/regression-test/suites/query_p0/cte/query_with_dup_column.groovy new file mode 100644 index 000000000000000..be88cb44bab1969 --- /dev/null +++ b/regression-test/suites/query_p0/cte/query_with_dup_column.groovy @@ -0,0 +1,112 @@ +// 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. + +import org.junit.Assert; + +suite("query_with_dup_column") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + + sql """ + drop table if exists test_table; + """ + + sql """ + CREATE TABLE `test_table` ( + `unique_id` varchar(256) NULL, + `name` varchar(256) NULL + ) + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into test_table values ("yyyxxxzzz", "abc000000") + """ + + // should fail + try { + sql """ + with tmp1 as ( + select unique_id, unique_id from test_table + ) + select * from tmp1; + """ + } catch (Exception e) { + assertTrue(e.message.contains("Duplicated inline view column alias")) + } + + // should fail + try { + sql """ + with tmp1 as ( + select unique_id, unique_id from test_table + ) + select * from tmp1 t; + """ + } catch (Exception e) { + assertTrue(e.message.contains("Duplicated inline view column alias")) + } + + + try { + sql """ + with tmp1 as ( + select *, unique_id from test_table + ) + select * from tmp1; + """ + } catch (Exception e) { + assertTrue(e.message.contains("Duplicated inline view column alias")) + } + + // should fail + try { + sql """ + with tmp1 as ( + select *, unique_id from test_table + ) + select * from tmp1 t; + """ + } catch (Exception e) { + assertTrue(e.message.contains("Duplicated inline view column alias")) + } + + // should success + sql """ + select *, unique_id from test_table; + """ + + // should success + sql """ + select *, unique_id from test_table t; + """ + + // should success + sql """ + select unique_id, unique_id from test_table + """ + + // should success + sql """ + select unique_id, unique_id from test_table t + """ +} + From 102c8c87798cd58550e1a2ffd0b96fe3b284ca21 Mon Sep 17 00:00:00 2001 From: "Mingyu Chen (Rayner)" Date: Thu, 31 Oct 2024 18:05:10 +0800 Subject: [PATCH 11/82] [opt](pr-template) add new pr template (#42999) ### What problem does this PR solve? Problem Summary: What you see now is the new PR template. Every contributor and reviewer should take filling out the PR template seriously. --- .github/PULL_REQUEST_TEMPLATE.md | 61 ++++++++++++++++++++++++++++++-- 1 file changed, 59 insertions(+), 2 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 98febd914c2724b..8c5fc5e3a98e427 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -1,6 +1,63 @@ -## Proposed changes +### What problem does this PR solve? + + + Issue Number: close #xxx - + +Related PR: #xxx + +Problem Summary: + +### Check List (For Committer) + +- Test + + - [ ] Regression test + - [ ] Unit Test + - [ ] Manual test (add detailed scripts or steps below) + - [ ] No need to test or manual test. Explain why: + - [ ] This is a refactor/code format and no logic has been changed. + - [ ] Previous test can cover this change. + - [ ] No colde files have been changed. + - [ ] Other reason + +- Behavior changed: + + - [ ] No. + - [ ] Yes. + +- Does this need documentation? + + - [ ] No. + - [ ] Yes. + +- Release note + + + + None + +### Check List (For Reviewer who merge this PR) + +- [ ] Confirm the release note +- [ ] Confirm test cases +- [ ] Confirm document +- [ ] Add branch pick label From 3ff0d1b5ca586d16991c923bdfb151ca543c5b8e Mon Sep 17 00:00:00 2001 From: Lei Zhang <27994433+SWJTU-ZhangLei@users.noreply.github.com> Date: Thu, 31 Oct 2024 19:19:54 +0800 Subject: [PATCH 12/82] [fix](be) Fix creating hdfs connection coredump within bthread (#42629) * https://brpc.apache.org/docs/server/basics/ * According to the brpc doc, `JNI code checks stack layout and cannot be run in bthreads` so create a pthread for creating hdfs connection if necessary. Issue Number: close #42610 Co-authored-by: Gavin Chou --- be/src/io/hdfs_util.cpp | 37 ++++++++++++++++++++++++++++++++++++- 1 file changed, 36 insertions(+), 1 deletion(-) diff --git a/be/src/io/hdfs_util.cpp b/be/src/io/hdfs_util.cpp index 6c1bbf80a1526f1..62546c9bbd4ffb6 100644 --- a/be/src/io/hdfs_util.cpp +++ b/be/src/io/hdfs_util.cpp @@ -17,10 +17,13 @@ #include "io/hdfs_util.h" +#include +#include #include #include #include +#include #include "common/logging.h" #include "io/fs/err_utils.h" @@ -30,7 +33,7 @@ namespace doris::io { namespace { -Status create_hdfs_fs(const THdfsParams& hdfs_params, const std::string& fs_name, hdfsFS* fs) { +Status _create_hdfs_fs(const THdfsParams& hdfs_params, const std::string& fs_name, hdfsFS* fs) { HDFSCommonBuilder builder; RETURN_IF_ERROR(create_hdfs_builder(hdfs_params, fs_name, &builder)); hdfsFS hdfs_fs = hdfsBuilderConnect(builder.get()); @@ -41,6 +44,38 @@ Status create_hdfs_fs(const THdfsParams& hdfs_params, const std::string& fs_name return Status::OK(); } +// https://brpc.apache.org/docs/server/basics/ +// According to the brpc doc, JNI code checks stack layout and cannot be run in +// bthreads so create a pthread for creating hdfs connection if necessary. +Status create_hdfs_fs(const THdfsParams& hdfs_params, const std::string& fs_name, hdfsFS* fs) { + bool is_pthread = bthread_self() == 0; + LOG(INFO) << "create hfdfs fs, is_pthread=" << is_pthread << " fs_name=" << fs_name; + if (is_pthread) { // running in pthread + return _create_hdfs_fs(hdfs_params, fs_name, fs); + } + + // running in bthread, switch to a pthread and wait + Status st; + auto btx = bthread::butex_create(); + *(int*)btx = 0; + std::thread t([&] { + st = _create_hdfs_fs(hdfs_params, fs_name, fs); + *(int*)btx = 1; + bthread::butex_wake_all(btx); + }); + std::unique_ptr> defer((int*)0x01, [&t, &btx](...) { + if (t.joinable()) t.join(); + bthread::butex_destroy(btx); + }); + timespec tmout {.tv_sec = std::chrono::system_clock::now().time_since_epoch().count() + 60}; + if (int ret = bthread::butex_wait(btx, 1, &tmout); ret != 0) { + std::string msg = "failed to wait _create_hdfs_fs fs_name=" + fs_name; + LOG(WARNING) << msg << " error=" << std::strerror(errno); + st = Status::Error(msg); + } + return st; +} + uint64_t hdfs_hash_code(const THdfsParams& hdfs_params, const std::string& fs_name) { uint64_t hash_code = 0; // The specified fsname is used first. From 78a5aed8df6badbd0ba121a73b80af4b484a8723 Mon Sep 17 00:00:00 2001 From: airborne12 Date: Thu, 31 Oct 2024 19:32:25 +0800 Subject: [PATCH 13/82] [Test](count on index) add statistics check for unique table (#42982) ## Proposed changes analyze unique table and wait for statistic upload --- .../test_count_on_index.groovy | 48 +++++++++++-------- 1 file changed, 28 insertions(+), 20 deletions(-) diff --git a/regression-test/suites/inverted_index_p0/test_count_on_index.groovy b/regression-test/suites/inverted_index_p0/test_count_on_index.groovy index c0e7e6845d202b3..89d1e8f93b25eae 100644 --- a/regression-test/suites/inverted_index_p0/test_count_on_index.groovy +++ b/regression-test/suites/inverted_index_p0/test_count_on_index.groovy @@ -146,30 +146,38 @@ suite("test_count_on_index_httplogs", "p0") { sql """set enable_fallback_to_original_planner=false;""" sql """analyze table ${testTable_dup} with sync"""; // case1: test duplicate table - def maxRetries = 3 - def attempt = 0 - def success = false + def executeSqlWithRetry = { String sqlQuery, int maxRetries = 3, int waitSeconds = 1 -> + def attempt = 0 + def success = false - while (attempt < maxRetries && !success) { - try { - explain { - sleep(10000) - sql("select COUNT() from ${testTable_dup}") - notContains("cardinality=0") - } - success = true - } catch (Exception e) { - attempt++ - log.error("Attempt ${attempt} failed: ${e.message}") - if (attempt < maxRetries) { - log.info("Retrying... (${attempt + 1}/${maxRetries})") - sleep(1000) - } else { - log.error("All ${maxRetries} attempts failed.") - throw e + while (attempt < maxRetries && !success) { + try { + explain { + // Wait for BE to report every partition's row count + sleep(10000) + sql(sqlQuery) + notContains("cardinality=0") + } + success = true + } catch (Exception e) { + attempt++ + log.error("Attempt ${attempt} failed: ${e.message}") + if (attempt < maxRetries) { + log.info("Retrying... (${attempt + 1}/${maxRetries}) after ${waitSeconds} second(s).") + sleep(waitSeconds * 1000) + } else { + log.error("All ${maxRetries} attempts failed.") + throw e + } } } } + // make sure row count stats is not 0 for duplicate table + executeSqlWithRetry("SELECT COUNT() FROM ${testTable_dup}") + // make sure row count stats is not 0 for unique table + sql """analyze table ${testTable_unique} with sync"""; + executeSqlWithRetry("SELECT COUNT() FROM ${testTable_unique}") + explain { sql("select COUNT() from ${testTable_dup} where request match 'GET'") contains "pushAggOp=COUNT_ON_INDEX" From fdc44ffb51b69e89d000f6a0857dbe6cd7cf6d84 Mon Sep 17 00:00:00 2001 From: Siyang Tang <82279870+TangSiyang2001@users.noreply.github.com> Date: Thu, 31 Oct 2024 19:37:55 +0800 Subject: [PATCH 14/82] [fix](delete) Fix static type dispatch by mistake due to typo (#42260) ## Proposed changes DeletePredicatePB should be DeleteSubPredicatePB. Test case is too ambiguous to add, since this bug is triggered by a huge random test and failed to find the minimal case. However, this fix is verified under the wild test that it does works. Note that this problem may be triggered by another bug, cuz schema in delete predicate rowset should contain column referred in delete condition. Even if we don't have this fix, this error should never happend. But this error occurred under wild tests, means that schema in delete predicate rowset is not adaptable with delete condition. I think it is under some status that delete operation use BE tablet schema rather than schema from FE, and the former rename operation result in that status. But I failed to add a test case to reproduce, and think that by no way will it happend occurding to the related code. ``` (1105, 'errCode = 2, detailMessage = ([172.20.50.7](http://172.20.50.7/))[INTERNAL_ERROR]failed to initialize storage reader. tablet=78026, res=[INTERNAL_ERROR]column not found, name=loc1, table_id=-1, schema_version=2 \t0# doris::TabletSchema::column(std::__cxx11::basic_string, std::allocator > const&) const at /home/zcp/repo_center/doris_master/doris/be/src/common/status.h:375 \t1# doris::Status doris::DeleteHandler::_parse_column_pred(std::shared_ptr, std::shared_ptr, google::protobuf::RepeatedPtrField const&, doris::DeleteConditions*) at /home/zcp/repo_center/doris_master/doris/be/src/util/expected.hpp:1986 \t2# doris::DeleteHandler::init(std::shared_ptr, std::vector, std::allocator > > const&, long) at /var/local/ldb-toolchain/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/shared_ptr_base.h:701 \t3# doris::TabletReader::_init_delete_condition(doris::TabletReader::ReaderParams const&) at /var/local/ldb-toolchain/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/shared_ptr_base.h:701 \t4# doris::TabletReader::_init_params(doris::TabletReader::ReaderParams const&) at /home/zcp/repo_center/doris_master/doris/be/src/common/status.h:499 \t5# doris::TabletReader::init(doris::TabletReader::ReaderParams const&) at /home/zcp/repo_center/doris_master/doris/be/src/common/status.h:499 \t6# doris::vectorized::BlockReader::init(doris::TabletReader::ReaderParams const&) at /home/zcp/repo_center/doris_master/doris/be/src/common/status.h:499 \t7# doris::vectorized::NewOlapScanner::open(doris::RuntimeState*) at /home/zcp/repo_center/doris_master/doris/be/src/common/status.h:499 \t8# doris::vectorized::ScannerScheduler::_scanner_scan(std::shared_ptr, std::shared_ptr) at /home/zcp/repo_center/doris_master/doris/be/src/common/status.h:388 \t9# std::_Function_handler, std::shared_ptr)::$_1::operator()() const::{lambda()#1}>::_M_invoke(std::_Any_data const&) at /var/local/ldb-toolchain/bin/../lib/gcc/x86_64-linux-gnu/11/../../../../include/c++/11/bits/shared_ptr_base.h:701 \t10# doris::ThreadPool::dispatch_thread() at /home/zcp/repo_center/doris_master/doris/be/src/util/threadpool.cpp:0 \t11# doris::Thread::supervise_thread(void*) at /var/local/ldb-toolchain/bin/../usr/include/pthread.h:562 \t12# ? \t13# ? , backend=[172.20.50.7](http://172.20.50.7/)') ``` ```cpp auto tablet_schema = std::make_shared(); tablet_schema->copy_from(*tablet->tablet_schema()); if (!request.columns_desc.empty() && request.columns_desc[0].col_unique_id >= 0) { tablet_schema->clear_columns(); // TODO(lhy) handle variant for (const auto& column_desc : request.columns_desc) { tablet_schema->append_column(TabletColumn(column_desc)); } } RowsetSharedPtr rowset_to_add; // writes res = _convert_v2(tablet, &rowset_to_add, tablet_schema, push_type); if (!res.ok()) { LOG(WARNING) << "fail to convert tmp file when realtime push. res=" << res << ", failed to process realtime push." << ", tablet=" << tablet->tablet_id() << ", transaction_id=" << request.transaction_id; Status rollback_status = _engine.txn_manager()->rollback_txn(request.partition_id, *tablet, request.transaction_id); // has to check rollback status to ensure not delete a committed rowset if (rollback_status.ok()) { _engine.add_unused_rowset(rowset_to_add); } return res; } // add pending data to tablet if (push_type == PushType::PUSH_FOR_DELETE) { rowset_to_add->rowset_meta()->set_delete_predicate(std::move(del_preds.front())); del_preds.pop(); } ``` --- be/src/olap/delete_handler.cpp | 13 +++++++++---- be/src/olap/delete_handler.h | 3 +++ 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/be/src/olap/delete_handler.cpp b/be/src/olap/delete_handler.cpp index 4d5b1ce9add3e0e..80fc440ce36a6db 100644 --- a/be/src/olap/delete_handler.cpp +++ b/be/src/olap/delete_handler.cpp @@ -346,6 +346,8 @@ Status DeleteHandler::parse_condition(const std::string& condition_str, TConditi } template + requires(std::is_same_v or + std::is_same_v) Status DeleteHandler::_parse_column_pred(TabletSchemaSPtr complete_schema, TabletSchemaSPtr delete_pred_related_schema, const RepeatedPtrField& sub_pred_list, @@ -353,10 +355,13 @@ Status DeleteHandler::_parse_column_pred(TabletSchemaSPtr complete_schema, for (const auto& sub_predicate : sub_pred_list) { TCondition condition; RETURN_IF_ERROR(parse_condition(sub_predicate, &condition)); - int32_t col_unique_id; - if constexpr (std::is_same_v) { - col_unique_id = sub_predicate.col_unique_id; - } else { + int32_t col_unique_id = -1; + if constexpr (std::is_same_v) { + if (sub_predicate.has_column_unique_id()) [[likely]] { + col_unique_id = sub_predicate.column_unique_id(); + } + } + if (col_unique_id < 0) { const auto& column = *DORIS_TRY(delete_pred_related_schema->column(condition.column_name)); col_unique_id = column.unique_id(); diff --git a/be/src/olap/delete_handler.h b/be/src/olap/delete_handler.h index cc585c0abcf9f6b..77de62d31d988e1 100644 --- a/be/src/olap/delete_handler.h +++ b/be/src/olap/delete_handler.h @@ -21,6 +21,7 @@ #include #include +#include #include "common/factory_creator.h" #include "common/status.h" @@ -115,6 +116,8 @@ class DeleteHandler { private: template + requires(std::is_same_v or + std::is_same_v) Status _parse_column_pred( TabletSchemaSPtr complete_schema, TabletSchemaSPtr delete_pred_related_schema, const ::google::protobuf::RepeatedPtrField& sub_pred_list, From d1b448fd71b3feaa539dee973f8b230cef2d2b20 Mon Sep 17 00:00:00 2001 From: abmdocrt Date: Thu, 31 Oct 2024 19:41:05 +0800 Subject: [PATCH 15/82] [fix](rowset meta) Fix rowset meta size relation (#41022) Issue #40879 step 2 make rowset total size = rowset data size + rowset index size --- be/src/cloud/cloud_base_compaction.cpp | 23 ++-- be/src/cloud/cloud_cumulative_compaction.cpp | 25 ++-- .../cloud_cumulative_compaction_policy.cpp | 2 +- be/src/cloud/cloud_full_compaction.cpp | 26 ++-- be/src/cloud/cloud_meta_mgr.cpp | 123 +++++++++++++++++ be/src/cloud/cloud_meta_mgr.h | 3 + be/src/cloud/cloud_rowset_builder.cpp | 2 +- be/src/cloud/cloud_schema_change_job.cpp | 2 +- be/src/cloud/cloud_tablet.cpp | 2 +- be/src/common/config.cpp | 2 + be/src/common/config.h | 2 + be/src/olap/base_compaction.cpp | 2 +- be/src/olap/compaction.cpp | 29 ++-- be/src/olap/compaction.h | 5 +- be/src/olap/cumulative_compaction.cpp | 3 +- be/src/olap/rowset/beta_rowset.cpp | 2 +- be/src/olap/rowset/beta_rowset_writer.cpp | 9 +- be/src/olap/rowset/beta_rowset_writer.h | 1 - be/src/olap/rowset/rowset.h | 3 +- be/src/olap/rowset/rowset_meta.cpp | 1 + be/src/olap/rowset/segment_creator.cpp | 32 +++-- .../rowset/vertical_beta_rowset_writer.cpp | 1 - be/src/olap/single_replica_compaction.cpp | 13 +- be/src/olap/tablet.cpp | 126 +++++++++++++++++- be/src/olap/tablet.h | 4 + be/src/olap/tablet_meta.h | 6 +- be/src/olap/task/engine_checksum_task.cpp | 2 +- be/src/olap/task/index_builder.cpp | 13 +- be/src/runtime/load_stream_writer.cpp | 2 +- cloud/src/common/config.h | 1 - cloud/src/meta-service/meta_service_txn.cpp | 2 +- cloud/test/meta_service_http_test.cpp | 6 +- cloud/test/meta_service_test.cpp | 6 +- cloud/test/schema_kv_test.cpp | 6 +- .../pipeline/cloud_p0/conf/be_custom.conf | 1 + .../pipeline/cloud_p1/conf/be_custom.conf | 1 + regression-test/pipeline/p0/conf/be.conf | 1 + regression-test/pipeline/p1/conf/be.conf | 1 + 38 files changed, 398 insertions(+), 93 deletions(-) diff --git a/be/src/cloud/cloud_base_compaction.cpp b/be/src/cloud/cloud_base_compaction.cpp index f431eaf850bbd19..88d83000e95dfaa 100644 --- a/be/src/cloud/cloud_base_compaction.cpp +++ b/be/src/cloud/cloud_base_compaction.cpp @@ -124,7 +124,8 @@ Status CloudBaseCompaction::prepare_compact() { for (auto& rs : _input_rowsets) { _input_row_num += rs->num_rows(); _input_segments += rs->num_segments(); - _input_rowsets_size += rs->data_disk_size(); + _input_rowsets_data_size += rs->data_disk_size(); + _input_rowsets_total_size += rs->total_disk_size(); } LOG_INFO("start CloudBaseCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), _input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()) @@ -132,7 +133,9 @@ Status CloudBaseCompaction::prepare_compact() { .tag("input_rowsets", _input_rowsets.size()) .tag("input_rows", _input_row_num) .tag("input_segments", _input_segments) - .tag("input_data_size", _input_rowsets_size); + .tag("input_rowsets_data_size", _input_rowsets_data_size) + .tag("input_rowsets_index_size", _input_rowsets_index_size) + .tag("input_rowsets_total_size", _input_rowsets_total_size); return st; } @@ -270,17 +273,21 @@ Status CloudBaseCompaction::execute_compact() { .tag("input_rowsets", _input_rowsets.size()) .tag("input_rows", _input_row_num) .tag("input_segments", _input_segments) - .tag("input_data_size", _input_rowsets_size) + .tag("input_rowsets_data_size", _input_rowsets_data_size) + .tag("input_rowsets_index_size", _input_rowsets_index_size) + .tag("input_rowsets_total", _input_rowsets_total_size) .tag("output_rows", _output_rowset->num_rows()) .tag("output_segments", _output_rowset->num_segments()) - .tag("output_data_size", _output_rowset->data_disk_size()); + .tag("output_rowset_data_size", _output_rowset->data_disk_size()) + .tag("output_rowset_index_size", _output_rowset->index_disk_size()) + .tag("output_rowset_total_size", _output_rowset->total_disk_size()); //_compaction_succeed = true; _state = CompactionState::SUCCESS; DorisMetrics::instance()->base_compaction_deltas_total->increment(_input_rowsets.size()); - DorisMetrics::instance()->base_compaction_bytes_total->increment(_input_rowsets_size); - base_output_size << _output_rowset->data_disk_size(); + DorisMetrics::instance()->base_compaction_bytes_total->increment(_input_rowsets_total_size); + base_output_size << _output_rowset->total_disk_size(); return Status::OK(); } @@ -302,8 +309,8 @@ Status CloudBaseCompaction::modify_rowsets() { compaction_job->set_output_cumulative_point(cloud_tablet()->cumulative_layer_point()); compaction_job->set_num_input_rows(_input_row_num); compaction_job->set_num_output_rows(_output_rowset->num_rows()); - compaction_job->set_size_input_rowsets(_input_rowsets_size); - compaction_job->set_size_output_rowsets(_output_rowset->data_disk_size()); + compaction_job->set_size_input_rowsets(_input_rowsets_total_size); + compaction_job->set_size_output_rowsets(_output_rowset->total_disk_size()); compaction_job->set_num_input_segments(_input_segments); compaction_job->set_num_output_segments(_output_rowset->num_segments()); compaction_job->set_num_input_rowsets(_input_rowsets.size()); diff --git a/be/src/cloud/cloud_cumulative_compaction.cpp b/be/src/cloud/cloud_cumulative_compaction.cpp index 7910d94534e086b..8eb925776934874 100644 --- a/be/src/cloud/cloud_cumulative_compaction.cpp +++ b/be/src/cloud/cloud_cumulative_compaction.cpp @@ -164,7 +164,9 @@ Status CloudCumulativeCompaction::prepare_compact() { for (auto& rs : _input_rowsets) { _input_row_num += rs->num_rows(); _input_segments += rs->num_segments(); - _input_rowsets_size += rs->data_disk_size(); + _input_rowsets_data_size += rs->data_disk_size(); + _input_rowsets_index_size += rs->index_disk_size(); + _input_rowsets_total_size += rs->total_disk_size(); } LOG_INFO("start CloudCumulativeCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), _input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()) @@ -172,7 +174,9 @@ Status CloudCumulativeCompaction::prepare_compact() { .tag("input_rowsets", _input_rowsets.size()) .tag("input_rows", _input_row_num) .tag("input_segments", _input_segments) - .tag("input_data_size", _input_rowsets_size) + .tag("input_rowsets_data_size", _input_rowsets_data_size) + .tag("input_rowsets_index_size", _input_rowsets_index_size) + .tag("input_rowsets_total_size", _input_rowsets_total_size) .tag("tablet_max_version", cloud_tablet()->max_version_unlocked()) .tag("cumulative_point", cloud_tablet()->cumulative_layer_point()) .tag("num_rowsets", cloud_tablet()->fetch_add_approximate_num_rowsets(0)) @@ -201,10 +205,14 @@ Status CloudCumulativeCompaction::execute_compact() { .tag("input_rowsets", _input_rowsets.size()) .tag("input_rows", _input_row_num) .tag("input_segments", _input_segments) - .tag("input_data_size", _input_rowsets_size) + .tag("input_rowsets_data_size", _input_rowsets_data_size) + .tag("input_rowsets_index_size", _input_rowsets_index_size) + .tag("input_rowsets_total_size", _input_rowsets_total_size) .tag("output_rows", _output_rowset->num_rows()) .tag("output_segments", _output_rowset->num_segments()) - .tag("output_data_size", _output_rowset->data_disk_size()) + .tag("output_rowset_data_size", _output_rowset->data_disk_size()) + .tag("output_rowset_index_size", _output_rowset->index_disk_size()) + .tag("output_rowset_total_size", _output_rowset->total_disk_size()) .tag("tablet_max_version", _tablet->max_version_unlocked()) .tag("cumulative_point", cloud_tablet()->cumulative_layer_point()) .tag("num_rowsets", cloud_tablet()->fetch_add_approximate_num_rowsets(0)) @@ -213,8 +221,9 @@ Status CloudCumulativeCompaction::execute_compact() { _state = CompactionState::SUCCESS; DorisMetrics::instance()->cumulative_compaction_deltas_total->increment(_input_rowsets.size()); - DorisMetrics::instance()->cumulative_compaction_bytes_total->increment(_input_rowsets_size); - cumu_output_size << _output_rowset->data_disk_size(); + DorisMetrics::instance()->cumulative_compaction_bytes_total->increment( + _input_rowsets_total_size); + cumu_output_size << _output_rowset->total_disk_size(); return Status::OK(); } @@ -243,8 +252,8 @@ Status CloudCumulativeCompaction::modify_rowsets() { compaction_job->set_output_cumulative_point(new_cumulative_point); compaction_job->set_num_input_rows(_input_row_num); compaction_job->set_num_output_rows(_output_rowset->num_rows()); - compaction_job->set_size_input_rowsets(_input_rowsets_size); - compaction_job->set_size_output_rowsets(_output_rowset->data_disk_size()); + compaction_job->set_size_input_rowsets(_input_rowsets_total_size); + compaction_job->set_size_output_rowsets(_output_rowset->total_disk_size()); compaction_job->set_num_input_segments(_input_segments); compaction_job->set_num_output_segments(_output_rowset->num_segments()); compaction_job->set_num_input_rowsets(_input_rowsets.size()); diff --git a/be/src/cloud/cloud_cumulative_compaction_policy.cpp b/be/src/cloud/cloud_cumulative_compaction_policy.cpp index f9af469e56f60a1..5a9879387b23278 100644 --- a/be/src/cloud/cloud_cumulative_compaction_policy.cpp +++ b/be/src/cloud/cloud_cumulative_compaction_policy.cpp @@ -209,7 +209,7 @@ int64_t CloudSizeBasedCumulativeCompactionPolicy::new_cumulative_point( // if rowsets have no delete version, check output_rowset total disk size satisfies promotion size. return output_rowset->start_version() == last_cumulative_point && (last_delete_version.first != -1 || - output_rowset->data_disk_size() >= cloud_promotion_size(tablet) || + output_rowset->total_disk_size() >= cloud_promotion_size(tablet) || satisfy_promotion_version) ? output_rowset->end_version() + 1 : last_cumulative_point; diff --git a/be/src/cloud/cloud_full_compaction.cpp b/be/src/cloud/cloud_full_compaction.cpp index f22c449223c448b..c27b728c93d29b1 100644 --- a/be/src/cloud/cloud_full_compaction.cpp +++ b/be/src/cloud/cloud_full_compaction.cpp @@ -98,7 +98,9 @@ Status CloudFullCompaction::prepare_compact() { for (auto& rs : _input_rowsets) { _input_row_num += rs->num_rows(); _input_segments += rs->num_segments(); - _input_rowsets_size += rs->data_disk_size(); + _input_rowsets_data_size += rs->data_disk_size(); + _input_rowsets_index_size += rs->index_disk_size(); + _input_rowsets_total_size += rs->total_disk_size(); } LOG_INFO("start CloudFullCompaction, tablet_id={}, range=[{}-{}]", _tablet->tablet_id(), _input_rowsets.front()->start_version(), _input_rowsets.back()->end_version()) @@ -106,7 +108,9 @@ Status CloudFullCompaction::prepare_compact() { .tag("input_rowsets", _input_rowsets.size()) .tag("input_rows", _input_row_num) .tag("input_segments", _input_segments) - .tag("input_data_size", _input_rowsets_size); + .tag("input_rowsets_data_size", _input_rowsets_data_size) + .tag("input_rowsets_index_size", _input_rowsets_index_size) + .tag("input_rowsets_total_size", _input_rowsets_total_size); return st; } @@ -162,16 +166,20 @@ Status CloudFullCompaction::execute_compact() { .tag("input_rowsets", _input_rowsets.size()) .tag("input_rows", _input_row_num) .tag("input_segments", _input_segments) - .tag("input_data_size", _input_rowsets_size) + .tag("input_rowsets_data_size", _input_rowsets_data_size) + .tag("input_rowsets_index_size", _input_rowsets_index_size) + .tag("input_rowsets_total_size", _input_rowsets_total_size) .tag("output_rows", _output_rowset->num_rows()) .tag("output_segments", _output_rowset->num_segments()) - .tag("output_data_size", _output_rowset->data_disk_size()); + .tag("output_rowset_data_size", _output_rowset->data_disk_size()) + .tag("output_rowset_index_size", _output_rowset->index_disk_size()) + .tag("output_rowset_total_size", _output_rowset->total_disk_size()); _state = CompactionState::SUCCESS; DorisMetrics::instance()->full_compaction_deltas_total->increment(_input_rowsets.size()); - DorisMetrics::instance()->full_compaction_bytes_total->increment(_input_rowsets_size); - full_output_size << _output_rowset->data_disk_size(); + DorisMetrics::instance()->full_compaction_bytes_total->increment(_input_rowsets_total_size); + full_output_size << _output_rowset->total_disk_size(); return Status::OK(); } @@ -193,8 +201,8 @@ Status CloudFullCompaction::modify_rowsets() { compaction_job->set_output_cumulative_point(_output_rowset->end_version() + 1); compaction_job->set_num_input_rows(_input_row_num); compaction_job->set_num_output_rows(_output_rowset->num_rows()); - compaction_job->set_size_input_rowsets(_input_rowsets_size); - compaction_job->set_size_output_rowsets(_output_rowset->data_disk_size()); + compaction_job->set_size_input_rowsets(_input_rowsets_total_size); + compaction_job->set_size_output_rowsets(_output_rowset->total_disk_size()); DBUG_EXECUTE_IF("CloudFullCompaction::modify_rowsets.wrong_compaction_data_size", { compaction_job->set_size_input_rowsets(1); compaction_job->set_size_output_rowsets(10000001); @@ -345,7 +353,7 @@ Status CloudFullCompaction::_cloud_full_compaction_update_delete_bitmap(int64_t .tag("input_rowsets", _input_rowsets.size()) .tag("input_rows", _input_row_num) .tag("input_segments", _input_segments) - .tag("input_data_size", _input_rowsets_size) + .tag("input_rowsets_total_size", _input_rowsets_total_size) .tag("update_bitmap_size", delete_bitmap->delete_bitmap.size()); _tablet->tablet_meta()->delete_bitmap().merge(*delete_bitmap); return Status::OK(); diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index 7dc9a4f11a157c3..57f3c7f80098d84 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -51,6 +52,7 @@ #include "olap/olap_common.h" #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_factory.h" +#include "olap/rowset/rowset_fwd.h" #include "olap/storage_engine.h" #include "olap/tablet_meta.h" #include "runtime/client_cache.h" @@ -750,6 +752,7 @@ Status CloudMetaMgr::commit_rowset(const RowsetMeta& rs_meta, Status ret_st; TEST_INJECTION_POINT_RETURN_WITH_VALUE("CloudMetaMgr::commit_rowset", ret_st); } + check_table_size_correctness(rs_meta); CreateRowsetRequest req; CreateRowsetResponse resp; req.set_cloud_unique_id(config::cloud_unique_id); @@ -1125,4 +1128,124 @@ Status CloudMetaMgr::remove_old_version_delete_bitmap( return st; } +void CloudMetaMgr::check_table_size_correctness(const RowsetMeta& rs_meta) { + if (!config::enable_table_size_correctness_check) { + return; + } + int64_t total_segment_size = get_segment_file_size(rs_meta); + int64_t total_inverted_index_size = get_inverted_index_file_szie(rs_meta); + if (rs_meta.data_disk_size() != total_segment_size || + rs_meta.index_disk_size() != total_inverted_index_size || + rs_meta.data_disk_size() + rs_meta.index_disk_size() != rs_meta.total_disk_size()) { + LOG(WARNING) << "[Cloud table table size check failed]:" + << " tablet id: " << rs_meta.tablet_id() + << ", rowset id:" << rs_meta.rowset_id() + << ", rowset data disk size:" << rs_meta.data_disk_size() + << ", rowset real data disk size:" << total_segment_size + << ", rowset index disk size:" << rs_meta.index_disk_size() + << ", rowset real index disk size:" << total_inverted_index_size + << ", rowset total disk size:" << rs_meta.total_disk_size() + << ", rowset segment path:" + << StorageResource().remote_segment_path(rs_meta.tablet_id(), + rs_meta.rowset_id().to_string(), 0); + DCHECK(false); + } +} + +int64_t CloudMetaMgr::get_segment_file_size(const RowsetMeta& rs_meta) { + int64_t total_segment_size = 0; + const auto fs = const_cast(rs_meta).fs(); + if (!fs) { + LOG(WARNING) << "get fs failed, resource_id={}" << rs_meta.resource_id(); + } + for (int64_t seg_id = 0; seg_id < rs_meta.num_segments(); seg_id++) { + std::string segment_path = StorageResource().remote_segment_path( + rs_meta.tablet_id(), rs_meta.rowset_id().to_string(), seg_id); + int64_t segment_file_size = 0; + auto st = fs->file_size(segment_path, &segment_file_size); + if (!st.ok()) { + segment_file_size = 0; + if (st.is()) { + LOG(INFO) << "cloud table size correctness check get segment size 0 because " + "file not exist! msg:" + << st.msg() << ", segment path:" << segment_path; + } else { + LOG(WARNING) << "cloud table size correctness check get segment size failed! msg:" + << st.msg() << ", segment path:" << segment_path; + } + } + total_segment_size += segment_file_size; + } + return total_segment_size; +} + +int64_t CloudMetaMgr::get_inverted_index_file_szie(const RowsetMeta& rs_meta) { + int64_t total_inverted_index_size = 0; + const auto fs = const_cast(rs_meta).fs(); + if (!fs) { + LOG(WARNING) << "get fs failed, resource_id={}" << rs_meta.resource_id(); + } + if (rs_meta.tablet_schema()->get_inverted_index_storage_format() == + InvertedIndexStorageFormatPB::V1) { + auto indices = rs_meta.tablet_schema()->indexes(); + for (auto& index : indices) { + // only get file_size for inverted index + if (index.index_type() != IndexType::INVERTED) { + continue; + } + for (int seg_id = 0; seg_id < rs_meta.num_segments(); ++seg_id) { + std::string segment_path = StorageResource().remote_segment_path( + rs_meta.tablet_id(), rs_meta.rowset_id().to_string(), seg_id); + int64_t file_size = 0; + + std::string inverted_index_file_path = + InvertedIndexDescriptor::get_index_file_path_v1( + InvertedIndexDescriptor::get_index_file_path_prefix(segment_path), + index.index_id(), index.get_index_suffix()); + auto st = fs->file_size(inverted_index_file_path, &file_size); + if (!st.ok()) { + file_size = 0; + if (st.is()) { + LOG(INFO) << "cloud table size correctness check get inverted index v1 " + "0 because file not exist! msg:" + << st.msg() + << ", inverted index path:" << inverted_index_file_path; + } else { + LOG(WARNING) + << "cloud table size correctness check get inverted index v1 " + "size failed! msg:" + << st.msg() << ", inverted index path:" << inverted_index_file_path; + } + } + total_inverted_index_size += file_size; + } + } + } else { + for (int seg_id = 0; seg_id < rs_meta.num_segments(); ++seg_id) { + int64_t file_size = 0; + std::string segment_path = StorageResource().remote_segment_path( + rs_meta.tablet_id(), rs_meta.rowset_id().to_string(), seg_id); + + std::string inverted_index_file_path = InvertedIndexDescriptor::get_index_file_path_v2( + InvertedIndexDescriptor::get_index_file_path_prefix(segment_path)); + auto st = fs->file_size(inverted_index_file_path, &file_size); + if (!st.ok()) { + file_size = 0; + if (st.is()) { + LOG(INFO) << "cloud table size correctness check get inverted index v2 " + "0 because file not exist! msg:" + << st.msg() << ", inverted index path:" << inverted_index_file_path; + } else { + LOG(WARNING) << "cloud table size correctness check get inverted index v2 " + "size failed! msg:" + << st.msg() + << ", inverted index path:" << inverted_index_file_path; + } + } + total_inverted_index_size += file_size; + } + } + return total_inverted_index_size; +} + } // namespace doris::cloud diff --git a/be/src/cloud/cloud_meta_mgr.h b/be/src/cloud/cloud_meta_mgr.h index 79cdb3fd3d1f8c0..a48381f056e446f 100644 --- a/be/src/cloud/cloud_meta_mgr.h +++ b/be/src/cloud/cloud_meta_mgr.h @@ -113,6 +113,9 @@ class CloudMetaMgr { Status sync_tablet_delete_bitmap(CloudTablet* tablet, int64_t old_max_version, std::ranges::range auto&& rs_metas, const TabletStatsPB& stats, const TabletIndexPB& idx, DeleteBitmap* delete_bitmap); + void check_table_size_correctness(const RowsetMeta& rs_meta); + int64_t get_segment_file_size(const RowsetMeta& rs_meta); + int64_t get_inverted_index_file_szie(const RowsetMeta& rs_meta); }; } // namespace cloud diff --git a/be/src/cloud/cloud_rowset_builder.cpp b/be/src/cloud/cloud_rowset_builder.cpp index 192da0f17efa825..2e6764b33aa79cb 100644 --- a/be/src/cloud/cloud_rowset_builder.cpp +++ b/be/src/cloud/cloud_rowset_builder.cpp @@ -106,7 +106,7 @@ void CloudRowsetBuilder::update_tablet_stats() { tablet->fetch_add_approximate_num_rowsets(1); tablet->fetch_add_approximate_num_segments(_rowset->num_segments()); tablet->fetch_add_approximate_num_rows(_rowset->num_rows()); - tablet->fetch_add_approximate_data_size(_rowset->data_disk_size()); + tablet->fetch_add_approximate_data_size(_rowset->total_disk_size()); tablet->fetch_add_approximate_cumu_num_rowsets(1); tablet->fetch_add_approximate_cumu_num_deltas(_rowset->num_segments()); tablet->write_count.fetch_add(1, std::memory_order_relaxed); diff --git a/be/src/cloud/cloud_schema_change_job.cpp b/be/src/cloud/cloud_schema_change_job.cpp index b7e3be93e853bb9..896804578d7db9c 100644 --- a/be/src/cloud/cloud_schema_change_job.cpp +++ b/be/src/cloud/cloud_schema_change_job.cpp @@ -344,7 +344,7 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam sc_job->add_txn_ids(rs->txn_id()); sc_job->add_output_versions(rs->end_version()); num_output_rows += rs->num_rows(); - size_output_rowsets += rs->data_disk_size(); + size_output_rowsets += rs->total_disk_size(); num_output_segments += rs->num_segments(); } sc_job->set_num_output_rows(num_output_rows); diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index b944db87030c29b..d3b131d055d35cb 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -412,7 +412,7 @@ int CloudTablet::delete_expired_stale_rowsets() { void CloudTablet::update_base_size(const Rowset& rs) { // Define base rowset as the rowset of version [2-x] if (rs.start_version() == 2) { - _base_size = rs.data_disk_size(); + _base_size = rs.total_disk_size(); } } diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 2fdebbd09c2c802..d8308c7eb97d8ec 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1356,6 +1356,8 @@ DEFINE_mInt32(check_score_rounds_num, "1000"); DEFINE_Int32(query_cache_size, "512"); DEFINE_mBool(enable_delete_bitmap_merge_on_compaction, "false"); +// Enable validation to check the correctness of table size. +DEFINE_Bool(enable_table_size_correctness_check, "false"); // clang-format off #ifdef BE_TEST diff --git a/be/src/common/config.h b/be/src/common/config.h index 791ca0b5e1a3ab1..f827e0f7dae7ea7 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1443,6 +1443,8 @@ DECLARE_mInt32(check_score_rounds_num); DECLARE_Int32(query_cache_size); DECLARE_mBool(enable_delete_bitmap_merge_on_compaction); +// Enable validation to check the correctness of table size. +DECLARE_Bool(enable_table_size_correctness_check); #ifdef BE_TEST // test s3 diff --git a/be/src/olap/base_compaction.cpp b/be/src/olap/base_compaction.cpp index 8be29383c1e9b1e..8b9cbd75ed33b80 100644 --- a/be/src/olap/base_compaction.cpp +++ b/be/src/olap/base_compaction.cpp @@ -80,7 +80,7 @@ Status BaseCompaction::execute_compact() { tablet()->set_last_base_compaction_success_time(UnixMillis()); DorisMetrics::instance()->base_compaction_deltas_total->increment(_input_rowsets.size()); - DorisMetrics::instance()->base_compaction_bytes_total->increment(_input_rowsets_size); + DorisMetrics::instance()->base_compaction_bytes_total->increment(_input_rowsets_total_size); return Status::OK(); } diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 14769bc315ad505..a76a5d8679d74db 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -257,10 +257,10 @@ int64_t Compaction::get_avg_segment_rows() { if (meta->compaction_policy() == CUMULATIVE_TIME_SERIES_POLICY) { int64_t compaction_goal_size_mbytes = meta->time_series_compaction_goal_size_mbytes(); return (compaction_goal_size_mbytes * 1024 * 1024 * 2) / - (_input_rowsets_size / (_input_row_num + 1) + 1); + (_input_rowsets_data_size / (_input_row_num + 1) + 1); } return config::vertical_compaction_max_segment_size / - (_input_rowsets_size / (_input_row_num + 1) + 1); + (_input_rowsets_data_size / (_input_row_num + 1) + 1); } CompactionMixin::CompactionMixin(StorageEngine& engine, TabletSharedPtr tablet, @@ -305,9 +305,9 @@ Status CompactionMixin::do_compact_ordered_rowsets() { // build output rowset RowsetMetaSharedPtr rowset_meta = std::make_shared(); rowset_meta->set_num_rows(_input_row_num); - rowset_meta->set_total_disk_size(_input_rowsets_size); - rowset_meta->set_data_disk_size(_input_rowsets_size); - rowset_meta->set_index_disk_size(_input_index_size); + rowset_meta->set_total_disk_size(_input_rowsets_data_size + _input_rowsets_index_size); + rowset_meta->set_data_disk_size(_input_rowsets_data_size); + rowset_meta->set_index_disk_size(_input_rowsets_index_size); rowset_meta->set_empty(_input_row_num == 0); rowset_meta->set_num_segments(_input_num_segments); rowset_meta->set_segments_overlap(NONOVERLAPPING); @@ -320,12 +320,13 @@ Status CompactionMixin::do_compact_ordered_rowsets() { void CompactionMixin::build_basic_info() { for (auto& rowset : _input_rowsets) { - _input_rowsets_size += rowset->data_disk_size(); - _input_index_size += rowset->index_disk_size(); + _input_rowsets_data_size += rowset->data_disk_size(); + _input_rowsets_index_size += rowset->index_disk_size(); + _input_rowsets_total_size += rowset->total_disk_size(); _input_row_num += rowset->num_rows(); _input_num_segments += rowset->num_segments(); } - COUNTER_UPDATE(_input_rowsets_data_size_counter, _input_rowsets_size); + COUNTER_UPDATE(_input_rowsets_data_size_counter, _input_rowsets_data_size); COUNTER_UPDATE(_input_row_num_counter, _input_row_num); COUNTER_UPDATE(_input_segments_num_counter, _input_num_segments); @@ -444,8 +445,12 @@ Status CompactionMixin::execute_compact_impl(int64_t permits) { << ", disk=" << tablet()->data_dir()->path() << ", segments=" << _input_num_segments << ", input_row_num=" << _input_row_num << ", output_row_num=" << _output_rowset->num_rows() - << ", input_rowset_size=" << _input_rowsets_size - << ", output_rowset_size=" << _output_rowset->data_disk_size() + << ", input_rowsets_data_size=" << _input_rowsets_data_size + << ", input_rowsets_index_size=" << _input_rowsets_index_size + << ", input_rowsets_total_size=" << _input_rowsets_total_size + << ", output_rowset_data_size=" << _output_rowset->data_disk_size() + << ", output_rowset_index_size=" << _output_rowset->index_disk_size() + << ", output_rowset_total_size=" << _output_rowset->total_disk_size() << ". elapsed time=" << watch.get_elapse_second() << "s."; _state = CompactionState::SUCCESS; return Status::OK(); @@ -467,8 +472,8 @@ Status CompactionMixin::execute_compact_impl(int64_t permits) { << ". tablet=" << _tablet->tablet_id() << ", output_version=" << _output_version << ", current_max_version=" << tablet()->max_version().second << ", disk=" << tablet()->data_dir()->path() << ", segments=" << _input_num_segments - << ", input_rowset_size=" << _input_rowsets_size - << ", output_rowset_size=" << _output_rowset->data_disk_size() + << ", input_data_size=" << _input_rowsets_data_size + << ", output_rowset_size=" << _output_rowset->total_disk_size() << ", input_row_num=" << _input_row_num << ", output_row_num=" << _output_rowset->num_rows() << ", filtered_row_num=" << _stats.filtered_rows diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h index 13a37beca19b23d..06ef4268529247b 100644 --- a/be/src/olap/compaction.h +++ b/be/src/olap/compaction.h @@ -90,10 +90,11 @@ class Compaction { BaseTabletSPtr _tablet; std::vector _input_rowsets; - int64_t _input_rowsets_size {0}; + int64_t _input_rowsets_data_size {0}; + int64_t _input_rowsets_index_size {0}; + int64_t _input_rowsets_total_size {0}; int64_t _input_row_num {0}; int64_t _input_num_segments {0}; - int64_t _input_index_size {0}; Merger::Statistics _stats; diff --git a/be/src/olap/cumulative_compaction.cpp b/be/src/olap/cumulative_compaction.cpp index b762468b3455a47..b961c694ede4d0e 100644 --- a/be/src/olap/cumulative_compaction.cpp +++ b/be/src/olap/cumulative_compaction.cpp @@ -125,7 +125,8 @@ Status CumulativeCompaction::execute_compact() { tablet()->set_last_cumu_compaction_success_time(UnixMillis()); } DorisMetrics::instance()->cumulative_compaction_deltas_total->increment(_input_rowsets.size()); - DorisMetrics::instance()->cumulative_compaction_bytes_total->increment(_input_rowsets_size); + DorisMetrics::instance()->cumulative_compaction_bytes_total->increment( + _input_rowsets_total_size); return Status::OK(); } diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp index ee1605a3043daa0..4b51dcc3530476a 100644 --- a/be/src/olap/rowset/beta_rowset.cpp +++ b/be/src/olap/rowset/beta_rowset.cpp @@ -498,7 +498,7 @@ Status BetaRowset::upload_to(const StorageResource& dest_fs, const RowsetId& new auto st = dest_fs.fs->batch_upload(local_paths, dest_paths); if (st.ok()) { DorisMetrics::instance()->upload_rowset_count->increment(1); - DorisMetrics::instance()->upload_total_byte->increment(data_disk_size()); + DorisMetrics::instance()->upload_total_byte->increment(total_disk_size()); } else { DorisMetrics::instance()->upload_fail_count->increment(1); } diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp index 3f60e7c5674ae1d..548b1950b817625 100644 --- a/be/src/olap/rowset/beta_rowset_writer.cpp +++ b/be/src/olap/rowset/beta_rowset_writer.cpp @@ -81,7 +81,7 @@ void build_rowset_meta_with_spec_field(RowsetMeta& rowset_meta, const RowsetMeta& spec_rowset_meta) { rowset_meta.set_num_rows(spec_rowset_meta.num_rows()); rowset_meta.set_total_disk_size(spec_rowset_meta.total_disk_size()); - rowset_meta.set_data_disk_size(spec_rowset_meta.total_disk_size()); + rowset_meta.set_data_disk_size(spec_rowset_meta.data_disk_size()); rowset_meta.set_index_disk_size(spec_rowset_meta.index_disk_size()); // TODO write zonemap to meta rowset_meta.set_empty(spec_rowset_meta.num_rows() == 0); @@ -886,7 +886,8 @@ Status BaseBetaRowsetWriter::_build_rowset_meta(RowsetMeta* rowset_meta, bool ch rowset_meta->set_num_segments(segment_num); rowset_meta->set_num_rows(num_rows_written + _num_rows_written); - rowset_meta->set_total_disk_size(total_data_size + _total_data_size); + rowset_meta->set_total_disk_size(total_data_size + _total_data_size + total_index_size + + _total_index_size); rowset_meta->set_data_disk_size(total_data_size + _total_data_size); rowset_meta->set_index_disk_size(total_index_size + _total_index_size); rowset_meta->set_segments_key_bounds(segments_encoded_key_bounds); @@ -1089,8 +1090,8 @@ Status BetaRowsetWriter::flush_segment_writer_for_segcompaction( SegmentStatistics segstat; segstat.row_num = row_num; - segstat.data_size = segment_size + inverted_index_file_size; - segstat.index_size = index_size + inverted_index_file_size; + segstat.data_size = segment_size; + segstat.index_size = inverted_index_file_size; segstat.key_bounds = key_bounds; { std::lock_guard lock(_segid_statistics_map_mutex); diff --git a/be/src/olap/rowset/beta_rowset_writer.h b/be/src/olap/rowset/beta_rowset_writer.h index 47e12a531e96d5a..4539959fab506bc 100644 --- a/be/src/olap/rowset/beta_rowset_writer.h +++ b/be/src/olap/rowset/beta_rowset_writer.h @@ -223,7 +223,6 @@ class BaseBetaRowsetWriter : public RowsetWriter { RETURN_NOT_OK_STATUS_WITH_WARN(_idx_files.close(), "failed to close index file when build new rowset"); this->_total_index_size += _idx_files.get_total_index_size(); - this->_total_data_size += _idx_files.get_total_index_size(); return Status::OK(); } diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h index 24e660cd2f72101..e1a2347f6aeaa8b 100644 --- a/be/src/olap/rowset/rowset.h +++ b/be/src/olap/rowset/rowset.h @@ -149,7 +149,8 @@ class Rowset : public std::enable_shared_from_this { int64_t start_version() const { return rowset_meta()->version().first; } int64_t end_version() const { return rowset_meta()->version().second; } size_t index_disk_size() const { return rowset_meta()->index_disk_size(); } - size_t data_disk_size() const { return rowset_meta()->total_disk_size(); } + size_t data_disk_size() const { return rowset_meta()->data_disk_size(); } + size_t total_disk_size() const { return rowset_meta()->total_disk_size(); } bool empty() const { return rowset_meta()->empty(); } bool zero_num_rows() const { return rowset_meta()->num_rows() == 0; } size_t num_rows() const { return rowset_meta()->num_rows(); } diff --git a/be/src/olap/rowset/rowset_meta.cpp b/be/src/olap/rowset/rowset_meta.cpp index 1571105fa734713..6bed5e800ede4dd 100644 --- a/be/src/olap/rowset/rowset_meta.cpp +++ b/be/src/olap/rowset/rowset_meta.cpp @@ -226,6 +226,7 @@ void RowsetMeta::merge_rowset_meta(const RowsetMeta& other) { set_data_disk_size(data_disk_size() + other.data_disk_size()); set_total_disk_size(total_disk_size() + other.total_disk_size()); set_index_disk_size(index_disk_size() + other.index_disk_size()); + set_total_disk_size(data_disk_size() + index_disk_size()); for (auto&& key_bound : other.get_segments_key_bounds()) { add_segment_key_bounds(key_bound); } diff --git a/be/src/olap/rowset/segment_creator.cpp b/be/src/olap/rowset/segment_creator.cpp index 5f4a3dce7b807e7..e0eb7534123a860 100644 --- a/be/src/olap/rowset/segment_creator.cpp +++ b/be/src/olap/rowset/segment_creator.cpp @@ -225,9 +225,9 @@ Status SegmentFlusher::_flush_segment_writer( if (row_num == 0) { return Status::OK(); } - uint64_t segment_size; - uint64_t index_size; - Status s = writer->finalize(&segment_size, &index_size); + uint64_t segment_file_size; + uint64_t common_index_size; + Status s = writer->finalize(&segment_file_size, &common_index_size); if (!s.ok()) { return Status::Error(s.code(), "failed to finalize segment: {}", s.to_string()); } @@ -249,16 +249,20 @@ Status SegmentFlusher::_flush_segment_writer( uint32_t segment_id = writer->segment_id(); SegmentStatistics segstat; segstat.row_num = row_num; - segstat.data_size = segment_size + inverted_index_file_size; - segstat.index_size = index_size + inverted_index_file_size; + segstat.data_size = segment_file_size; + segstat.index_size = inverted_index_file_size; segstat.key_bounds = key_bounds; + LOG(INFO) << "tablet_id:" << _context.tablet_id + << ", flushing rowset_dir: " << _context.tablet_path + << ", rowset_id:" << _context.rowset_id << ", data size:" << segstat.data_size + << ", index size:" << segstat.index_size; writer.reset(); RETURN_IF_ERROR(_context.segment_collector->add(segment_id, segstat, flush_schema)); if (flush_size) { - *flush_size = segment_size + inverted_index_file_size; + *flush_size = segment_file_size; } return Status::OK(); } @@ -274,9 +278,9 @@ Status SegmentFlusher::_flush_segment_writer(std::unique_ptrfinalize(&segment_size, &index_size); + uint64_t segment_file_size; + uint64_t common_index_size; + Status s = writer->finalize(&segment_file_size, &common_index_size); if (!s.ok()) { return Status::Error(s.code(), "failed to finalize segment: {}", s.to_string()); } @@ -298,16 +302,20 @@ Status SegmentFlusher::_flush_segment_writer(std::unique_ptrget_segment_id(); SegmentStatistics segstat; segstat.row_num = row_num; - segstat.data_size = segment_size + inverted_index_file_size; - segstat.index_size = index_size + inverted_index_file_size; + segstat.data_size = segment_file_size; + segstat.index_size = inverted_index_file_size; segstat.key_bounds = key_bounds; + LOG(INFO) << "tablet_id:" << _context.tablet_id + << ", flushing rowset_dir: " << _context.tablet_path + << ", rowset_id:" << _context.rowset_id << ", data size:" << segstat.data_size + << ", index size:" << segstat.index_size; writer.reset(); RETURN_IF_ERROR(_context.segment_collector->add(segment_id, segstat, flush_schema)); if (flush_size) { - *flush_size = segment_size + inverted_index_file_size; + *flush_size = segment_file_size; } return Status::OK(); } diff --git a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp index fb8f66226858569..46070f8dccd7ce1 100644 --- a/be/src/olap/rowset/vertical_beta_rowset_writer.cpp +++ b/be/src/olap/rowset/vertical_beta_rowset_writer.cpp @@ -138,7 +138,6 @@ Status VerticalBetaRowsetWriter::_flush_columns(segment_v2::SegmentWriter* se this->_segment_num_rows.resize(_cur_writer_idx + 1); this->_segment_num_rows[_cur_writer_idx] = _segment_writers[_cur_writer_idx]->row_count(); } - this->_total_index_size += static_cast(index_size); return Status::OK(); } diff --git a/be/src/olap/single_replica_compaction.cpp b/be/src/olap/single_replica_compaction.cpp index ef93ab25caeac9f..7470afe0ef62c72 100644 --- a/be/src/olap/single_replica_compaction.cpp +++ b/be/src/olap/single_replica_compaction.cpp @@ -149,11 +149,15 @@ Status SingleReplicaCompaction::_do_single_replica_compaction_impl() { LOG(INFO) << "succeed to do single replica compaction" << ". tablet=" << _tablet->tablet_id() << ", output_version=" << _output_version << ", current_max_version=" << current_max_version - << ", input_rowset_size=" << _input_rowsets_size + << ", input_rowsets_data_size=" << _input_rowsets_data_size + << ", input_rowsets_index_size=" << _input_rowsets_index_size + << ", input_rowsets_total_size=" << _input_rowsets_total_size << ", input_row_num=" << _input_row_num << ", input_segments_num=" << _input_num_segments - << ", _input_index_size=" << _input_index_size + << ", _input_index_size=" << _input_rowsets_index_size << ", output_rowset_data_size=" << _output_rowset->data_disk_size() + << ", output_rowset_index_size=" << _output_rowset->index_disk_size() + << ", output_rowset_total_size=" << _output_rowset->total_disk_size() << ", output_row_num=" << _output_rowset->num_rows() << ", output_segments_num=" << _output_rowset->num_segments(); return Status::OK(); @@ -264,10 +268,11 @@ bool SingleReplicaCompaction::_find_rowset_to_fetch(const std::vector& return false; } for (auto& rowset : _input_rowsets) { - _input_rowsets_size += rowset->data_disk_size(); + _input_rowsets_data_size += rowset->data_disk_size(); _input_row_num += rowset->num_rows(); _input_num_segments += rowset->num_segments(); - _input_index_size += rowset->index_disk_size(); + _input_rowsets_index_size += rowset->index_disk_size(); + _input_rowsets_total_size += rowset->data_disk_size() + rowset->index_disk_size(); } _output_version = *proper_version; } diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 450f3d2cb8bab45..7c69ba54831ce9a 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -35,6 +36,7 @@ #include #include #include +#include #include #include "common/compiler_util.h" // IWYU pragma: keep @@ -86,6 +88,7 @@ #include "olap/rowset/beta_rowset.h" #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_factory.h" +#include "olap/rowset/rowset_fwd.h" #include "olap/rowset/rowset_meta.h" #include "olap/rowset/rowset_meta_manager.h" #include "olap/rowset/rowset_writer.h" @@ -329,6 +332,7 @@ Status Tablet::init() { // should save tablet meta to remote meta store // if it's a primary replica void Tablet::save_meta() { + check_table_size_correctness(); auto res = _tablet_meta->save_meta(_data_dir); CHECK_EQ(res, Status::OK()) << "fail to save tablet_meta. res=" << res << ", root=" << _data_dir->path(); @@ -2047,8 +2051,8 @@ Status Tablet::_cooldown_data(RowsetSharedPtr rowset) { LOG(INFO) << "Upload rowset " << old_rowset->version() << " " << new_rowset_id.to_string() << " to " << storage_resource.fs->root_path().native() << ", tablet_id=" << tablet_id() << ", duration=" << duration.count() - << ", capacity=" << old_rowset->data_disk_size() - << ", tp=" << old_rowset->data_disk_size() / duration.count() + << ", capacity=" << old_rowset->total_disk_size() + << ", tp=" << old_rowset->total_disk_size() / duration.count() << ", old rowset_id=" << old_rowset->rowset_id().to_string(); // gen a new rowset @@ -2427,7 +2431,7 @@ RowsetSharedPtr Tablet::need_cooldown(int64_t* cooldown_timestamp, size_t* file_ // current time or it's datatime is less than current time if (newest_cooldown_time != 0 && newest_cooldown_time < UnixSeconds()) { *cooldown_timestamp = newest_cooldown_time; - *file_size = rowset->data_disk_size(); + *file_size = rowset->total_disk_size(); VLOG_DEBUG << "tablet need cooldown, tablet id: " << tablet_id() << " file_size: " << *file_size; return rowset; @@ -2737,4 +2741,120 @@ void Tablet::clear_cache() { } } +void Tablet::check_table_size_correctness() { + if (!config::enable_table_size_correctness_check) { + return; + } + const std::vector& all_rs_metas = _tablet_meta->all_rs_metas(); + for (const auto& rs_meta : all_rs_metas) { + int64_t total_segment_size = get_segment_file_size(rs_meta); + int64_t total_inverted_index_size = get_inverted_index_file_szie(rs_meta); + if (rs_meta->data_disk_size() != total_segment_size || + rs_meta->index_disk_size() != total_inverted_index_size || + rs_meta->data_disk_size() + rs_meta->index_disk_size() != rs_meta->total_disk_size()) { + LOG(WARNING) << "[Local table table size check failed]:" + << " tablet id: " << rs_meta->tablet_id() + << ", rowset id:" << rs_meta->rowset_id() + << ", rowset data disk size:" << rs_meta->data_disk_size() + << ", rowset real data disk size:" << total_segment_size + << ", rowset index disk size:" << rs_meta->index_disk_size() + << ", rowset real index disk size:" << total_inverted_index_size + << ", rowset total disk size:" << rs_meta->total_disk_size() + << ", rowset segment path:" + << StorageResource().remote_segment_path( + rs_meta->tablet_id(), rs_meta->rowset_id().to_string(), 0); + DCHECK(false); + } + } +} + +std::string Tablet::get_segment_path(const RowsetMetaSharedPtr& rs_meta, int64_t seg_id) { + std::string segment_path; + if (rs_meta->is_local()) { + segment_path = local_segment_path(_tablet_path, rs_meta->rowset_id().to_string(), seg_id); + } else { + segment_path = rs_meta->remote_storage_resource().value()->remote_segment_path( + rs_meta->tablet_id(), rs_meta->rowset_id().to_string(), seg_id); + } + return segment_path; +} + +int64_t Tablet::get_segment_file_size(const RowsetMetaSharedPtr& rs_meta) { + const auto& fs = rs_meta->fs(); + if (!fs) { + LOG(WARNING) << "get fs failed, resource_id={}" << rs_meta->resource_id(); + } + int64_t total_segment_size = 0; + for (int64_t seg_id = 0; seg_id < rs_meta->num_segments(); seg_id++) { + std::string segment_path = get_segment_path(rs_meta, seg_id); + int64_t segment_file_size = 0; + auto st = fs->file_size(segment_path, &segment_file_size); + if (!st.ok()) { + segment_file_size = 0; + LOG(WARNING) << "table size correctness check get segment size failed! msg:" + << st.to_string() << ", segment path:" << segment_path; + } + total_segment_size += segment_file_size; + } + return total_segment_size; +} + +int64_t Tablet::get_inverted_index_file_szie(const RowsetMetaSharedPtr& rs_meta) { + const auto& fs = rs_meta->fs(); + if (!fs) { + LOG(WARNING) << "get fs failed, resource_id={}" << rs_meta->resource_id(); + } + int64_t total_inverted_index_size = 0; + + if (rs_meta->tablet_schema()->get_inverted_index_storage_format() == + InvertedIndexStorageFormatPB::V1) { + auto indices = rs_meta->tablet_schema()->indexes(); + for (auto& index : indices) { + // only get file_size for inverted index + if (index.index_type() != IndexType::INVERTED) { + continue; + } + for (int seg_id = 0; seg_id < rs_meta->num_segments(); ++seg_id) { + std::string segment_path = get_segment_path(rs_meta, seg_id); + int64_t file_size = 0; + + std::string inverted_index_file_path = + InvertedIndexDescriptor::get_index_file_path_v1( + InvertedIndexDescriptor::get_index_file_path_prefix(segment_path), + index.index_id(), index.get_index_suffix()); + auto st = fs->file_size(inverted_index_file_path, &file_size); + if (!st.ok()) { + file_size = 0; + LOG(WARNING) << " tablet id: " << get_tablet_info().tablet_id + << ", rowset id:" << rs_meta->rowset_id() + << ", table size correctness check get inverted index v1 " + "size failed! msg:" + << st.to_string() + << ", inverted index path:" << inverted_index_file_path; + } + total_inverted_index_size += file_size; + } + } + } else { + for (int seg_id = 0; seg_id < rs_meta->num_segments(); ++seg_id) { + int64_t file_size = 0; + std::string segment_path = get_segment_path(rs_meta, seg_id); + std::string inverted_index_file_path = InvertedIndexDescriptor::get_index_file_path_v2( + InvertedIndexDescriptor::get_index_file_path_prefix(segment_path)); + auto st = fs->file_size(inverted_index_file_path, &file_size); + if (!st.ok()) { + file_size = 0; + LOG(WARNING) << " tablet id: " << get_tablet_info().tablet_id + << ", rowset id:" << rs_meta->rowset_id() + << ", table size correctness check get inverted index v2 " + "size failed! msg:" + << st.to_string() + << ", inverted index path:" << inverted_index_file_path; + } + total_inverted_index_size += file_size; + } + } + return total_inverted_index_size; +} + } // namespace doris diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index 2d7975b0fc12d14..e181af3d4d3a3df 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -534,6 +534,10 @@ class Tablet final : public BaseTablet { //////////////////////////////////////////////////////////////////////////// void _clear_cache_by_rowset(const BetaRowsetSharedPtr& rowset); + void check_table_size_correctness(); + std::string get_segment_path(const RowsetMetaSharedPtr& rs_meta, int64_t seg_id); + int64_t get_segment_file_size(const RowsetMetaSharedPtr& rs_meta); + int64_t get_inverted_index_file_szie(const RowsetMetaSharedPtr& rs_meta); public: static const int64_t K_INVALID_CUMULATIVE_POINT = -1; diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h index 394aeb17b85183e..d56e529e42bf4b4 100644 --- a/be/src/olap/tablet_meta.h +++ b/be/src/olap/tablet_meta.h @@ -642,7 +642,7 @@ inline size_t TabletMeta::num_rows() const { inline size_t TabletMeta::tablet_footprint() const { size_t total_size = 0; for (auto& rs : _rs_metas) { - total_size += rs->data_disk_size(); + total_size += rs->total_disk_size(); } return total_size; } @@ -651,7 +651,7 @@ inline size_t TabletMeta::tablet_local_size() const { size_t total_size = 0; for (auto& rs : _rs_metas) { if (rs->is_local()) { - total_size += rs->data_disk_size(); + total_size += rs->total_disk_size(); } } return total_size; @@ -661,7 +661,7 @@ inline size_t TabletMeta::tablet_remote_size() const { size_t total_size = 0; for (auto& rs : _rs_metas) { if (!rs->is_local()) { - total_size += rs->data_disk_size(); + total_size += rs->total_disk_size(); } } return total_size; diff --git a/be/src/olap/task/engine_checksum_task.cpp b/be/src/olap/task/engine_checksum_task.cpp index d0c4b0e45f468ef..05ecfc0401b6d04 100644 --- a/be/src/olap/task/engine_checksum_task.cpp +++ b/be/src/olap/task/engine_checksum_task.cpp @@ -93,7 +93,7 @@ Status EngineChecksumTask::_compute_checksum() { } size_t input_size = 0; for (const auto& rowset : input_rowsets) { - input_size += rowset->data_disk_size(); + input_size += rowset->total_disk_size(); } auto res = reader.init(reader_params); diff --git a/be/src/olap/task/index_builder.cpp b/be/src/olap/task/index_builder.cpp index 2ecadfa53b09b78..09cbdeadb3f3c97 100644 --- a/be/src/olap/task/index_builder.cpp +++ b/be/src/olap/task/index_builder.cpp @@ -207,13 +207,12 @@ Status IndexBuilder::update_inverted_index_info() { InvertedIndexStorageFormatPB::V1) { if (_is_drop_op) { VLOG_DEBUG << "data_disk_size:" << input_rowset_meta->data_disk_size() - << " total_disk_size:" << input_rowset_meta->data_disk_size() + << " total_disk_size:" << input_rowset_meta->total_disk_size() << " index_disk_size:" << input_rowset_meta->index_disk_size() << " drop_index_size:" << drop_index_size; rowset_meta->set_total_disk_size(input_rowset_meta->total_disk_size() - drop_index_size); - rowset_meta->set_data_disk_size(input_rowset_meta->data_disk_size() - - drop_index_size); + rowset_meta->set_data_disk_size(input_rowset_meta->data_disk_size()); rowset_meta->set_index_disk_size(input_rowset_meta->index_disk_size() - drop_index_size); } else { @@ -238,7 +237,7 @@ Status IndexBuilder::update_inverted_index_info() { } rowset_meta->set_total_disk_size(input_rowset_meta->total_disk_size() - total_index_size); - rowset_meta->set_data_disk_size(input_rowset_meta->data_disk_size() - total_index_size); + rowset_meta->set_data_disk_size(input_rowset_meta->data_disk_size()); rowset_meta->set_index_disk_size(input_rowset_meta->index_disk_size() - total_index_size); } @@ -323,8 +322,7 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta inverted_index_size += inverted_index_writer->get_index_file_total_size(); } _inverted_index_file_writers.clear(); - output_rowset_meta->set_data_disk_size(output_rowset_meta->data_disk_size() + - inverted_index_size); + output_rowset_meta->set_data_disk_size(output_rowset_meta->data_disk_size()); output_rowset_meta->set_total_disk_size(output_rowset_meta->total_disk_size() + inverted_index_size); output_rowset_meta->set_index_disk_size(output_rowset_meta->index_disk_size() + @@ -489,8 +487,7 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta } _inverted_index_builders.clear(); _inverted_index_file_writers.clear(); - output_rowset_meta->set_data_disk_size(output_rowset_meta->data_disk_size() + - inverted_index_size); + output_rowset_meta->set_data_disk_size(output_rowset_meta->data_disk_size()); output_rowset_meta->set_total_disk_size(output_rowset_meta->total_disk_size() + inverted_index_size); output_rowset_meta->set_index_disk_size(output_rowset_meta->index_disk_size() + diff --git a/be/src/runtime/load_stream_writer.cpp b/be/src/runtime/load_stream_writer.cpp index 37243fab14bdb35..2e987edc7bd3bad 100644 --- a/be/src/runtime/load_stream_writer.cpp +++ b/be/src/runtime/load_stream_writer.cpp @@ -201,7 +201,7 @@ Status LoadStreamWriter::add_segment(uint32_t segid, const SegmentStatistics& st } DBUG_EXECUTE_IF("LoadStreamWriter.add_segment.size_not_match", { segment_file_size++; }); - if (segment_file_size + inverted_file_size != stat.data_size) { + if (segment_file_size != stat.data_size) { return Status::Corruption( "add_segment failed, segment stat {} does not match, file size={}, inverted file " "size={}, stat.data_size={}, tablet id={}", diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index 7caba826520fb3a..daeb5ddfee5d340 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -217,5 +217,4 @@ CONF_Int32(max_tablet_index_num_per_batch, "1000"); // Max aborted txn num for the same label name CONF_mInt64(max_num_aborted_txn, "100"); - } // namespace doris::cloud::config diff --git a/cloud/src/meta-service/meta_service_txn.cpp b/cloud/src/meta-service/meta_service_txn.cpp index cc333c428468a42..32f6b56f51af4cd 100644 --- a/cloud/src/meta-service/meta_service_txn.cpp +++ b/cloud/src/meta-service/meta_service_txn.cpp @@ -1167,7 +1167,7 @@ void commit_txn_immediately( // Accumulate affected rows auto& stats = tablet_stats[tablet_id]; - stats.data_size += i.data_disk_size(); + stats.data_size += i.total_disk_size(); stats.num_rows += i.num_rows(); ++stats.num_rowsets; stats.num_segs += i.num_segments(); diff --git a/cloud/test/meta_service_http_test.cpp b/cloud/test/meta_service_http_test.cpp index 20dee957126e4de..e49628fcb3a7839 100644 --- a/cloud/test/meta_service_http_test.cpp +++ b/cloud/test/meta_service_http_test.cpp @@ -320,6 +320,8 @@ static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, rowset.set_num_segments(1); rowset.set_num_rows(num_rows); rowset.set_data_disk_size(num_rows * 100); + rowset.set_index_disk_size(num_rows * 10); + rowset.set_total_disk_size(num_rows * 110); rowset.mutable_tablet_schema()->set_schema_version(0); rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK return rowset; @@ -1285,7 +1287,7 @@ TEST(MetaServiceHttpTest, GetTabletStatsTest) { stats_tablet_data_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, &data_size_key); ASSERT_EQ(txn->get(data_size_key, &data_size_val), TxnErrorCode::TXN_OK); - EXPECT_EQ(*(int64_t*)data_size_val.data(), 20000); + EXPECT_EQ(*(int64_t*)data_size_val.data(), 22000); std::string num_rows_key, num_rows_val; stats_tablet_num_rows_key({mock_instance, table_id, index_id, partition_id, tablet_id}, &num_rows_key); @@ -1306,7 +1308,7 @@ TEST(MetaServiceHttpTest, GetTabletStatsTest) { get_tablet_stats(meta_service.get(), table_id, index_id, partition_id, tablet_id, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); ASSERT_EQ(res.tablet_stats_size(), 1); - EXPECT_EQ(res.tablet_stats(0).data_size(), 40000); + EXPECT_EQ(res.tablet_stats(0).data_size(), 44000); EXPECT_EQ(res.tablet_stats(0).num_rows(), 400); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 5); EXPECT_EQ(res.tablet_stats(0).num_segments(), 4); diff --git a/cloud/test/meta_service_test.cpp b/cloud/test/meta_service_test.cpp index 3baec482710bc49..ee90e604e1c5f67 100644 --- a/cloud/test/meta_service_test.cpp +++ b/cloud/test/meta_service_test.cpp @@ -178,6 +178,8 @@ static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, rowset.set_num_segments(1); rowset.set_num_rows(num_rows); rowset.set_data_disk_size(num_rows * 100); + rowset.set_index_disk_size(num_rows * 10); + rowset.set_total_disk_size(num_rows * 110); rowset.mutable_tablet_schema()->set_schema_version(0); rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK return rowset; @@ -4429,7 +4431,7 @@ TEST(MetaServiceTest, GetTabletStatsTest) { stats_tablet_data_size_key({mock_instance, table_id, index_id, partition_id, tablet_id}, &data_size_key); ASSERT_EQ(txn->get(data_size_key, &data_size_val), TxnErrorCode::TXN_OK); - EXPECT_EQ(*(int64_t*)data_size_val.data(), 20000); + EXPECT_EQ(*(int64_t*)data_size_val.data(), 22000); std::string num_rows_key, num_rows_val; stats_tablet_num_rows_key({mock_instance, table_id, index_id, partition_id, tablet_id}, &num_rows_key); @@ -4450,7 +4452,7 @@ TEST(MetaServiceTest, GetTabletStatsTest) { get_tablet_stats(meta_service.get(), table_id, index_id, partition_id, tablet_id, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK); ASSERT_EQ(res.tablet_stats_size(), 1); - EXPECT_EQ(res.tablet_stats(0).data_size(), 40000); + EXPECT_EQ(res.tablet_stats(0).data_size(), 44000); EXPECT_EQ(res.tablet_stats(0).num_rows(), 400); EXPECT_EQ(res.tablet_stats(0).num_rowsets(), 5); EXPECT_EQ(res.tablet_stats(0).num_segments(), 4); diff --git a/cloud/test/schema_kv_test.cpp b/cloud/test/schema_kv_test.cpp index 69ee9aba4422098..07f658175c806f1 100644 --- a/cloud/test/schema_kv_test.cpp +++ b/cloud/test/schema_kv_test.cpp @@ -293,6 +293,8 @@ static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, rowset.set_num_rows(100); rowset.set_num_segments(1); rowset.set_data_disk_size(10000); + rowset.set_index_disk_size(1000); + rowset.set_total_disk_size(11000); if (version > 0) { rowset.set_start_version(version); rowset.set_end_version(version); @@ -478,7 +480,7 @@ TEST(DetachSchemaKVTest, RowsetTest) { EXPECT_EQ(get_rowset_res.stats().num_rows(), 100); EXPECT_EQ(get_rowset_res.stats().num_rowsets(), 2); EXPECT_EQ(get_rowset_res.stats().num_segments(), 1); - EXPECT_EQ(get_rowset_res.stats().data_size(), 10000); + EXPECT_EQ(get_rowset_res.stats().data_size(), 11000); } // new MS read rowsets committed by both old and new MS @@ -527,7 +529,7 @@ TEST(DetachSchemaKVTest, RowsetTest) { EXPECT_EQ(get_rowset_res->stats().num_rows(), 2500); EXPECT_EQ(get_rowset_res->stats().num_rowsets(), 26); EXPECT_EQ(get_rowset_res->stats().num_segments(), 25); - EXPECT_EQ(get_rowset_res->stats().data_size(), 250000); + EXPECT_EQ(get_rowset_res->stats().data_size(), 275000); if (schema != nullptr) { auto schema_version = get_rowset_res->rowset_meta(10).schema_version(); get_rowset_res->mutable_rowset_meta(10)->mutable_tablet_schema()->set_schema_version(3); diff --git a/regression-test/pipeline/cloud_p0/conf/be_custom.conf b/regression-test/pipeline/cloud_p0/conf/be_custom.conf index 377a02536c6d292..5fbf89fd75cc77e 100644 --- a/regression-test/pipeline/cloud_p0/conf/be_custom.conf +++ b/regression-test/pipeline/cloud_p0/conf/be_custom.conf @@ -37,3 +37,4 @@ enable_new_tablet_do_compaction = true arrow_flight_sql_port = 8181 pipeline_task_leakage_detect_period_sec=1 crash_in_memory_tracker_inaccurate = true +enable_table_size_correctness_check=true diff --git a/regression-test/pipeline/cloud_p1/conf/be_custom.conf b/regression-test/pipeline/cloud_p1/conf/be_custom.conf index 4310441a0ed66bd..0d3ae0c526d942c 100644 --- a/regression-test/pipeline/cloud_p1/conf/be_custom.conf +++ b/regression-test/pipeline/cloud_p1/conf/be_custom.conf @@ -33,3 +33,4 @@ arrow_flight_sql_port = 8181 pipeline_task_leakage_detect_period_sec=1 crash_in_memory_tracker_inaccurate = true enable_new_tablet_do_compaction = true +enable_table_size_correctness_check=true diff --git a/regression-test/pipeline/p0/conf/be.conf b/regression-test/pipeline/p0/conf/be.conf index 745515aed06cffa..760f813ffebcdc6 100644 --- a/regression-test/pipeline/p0/conf/be.conf +++ b/regression-test/pipeline/p0/conf/be.conf @@ -71,3 +71,4 @@ be_proc_monitor_interval_ms = 30000 webserver_num_workers = 128 pipeline_task_leakage_detect_period_sec=1 crash_in_memory_tracker_inaccurate = true +enable_table_size_correctness_check=true diff --git a/regression-test/pipeline/p1/conf/be.conf b/regression-test/pipeline/p1/conf/be.conf index fbb36c340f0d30f..1512bce7ac2388a 100644 --- a/regression-test/pipeline/p1/conf/be.conf +++ b/regression-test/pipeline/p1/conf/be.conf @@ -63,3 +63,4 @@ enable_missing_rows_correctness_check=true enable_jvm_monitor = true pipeline_task_leakage_detect_period_sec=1 crash_in_memory_tracker_inaccurate = true +enable_table_size_correctness_check=true From 1cb2a3fa665b6edd6fdbcd53aba6c2037d2400c0 Mon Sep 17 00:00:00 2001 From: walter Date: Thu, 31 Oct 2024 21:05:11 +0800 Subject: [PATCH 16/82] [feature](binlog) Allow BarrierLog to wrap another binlog (#42993) to make it work in the old Doris version without breaking compatibility --- .../org/apache/doris/persist/BarrierLog.java | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/BarrierLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/BarrierLog.java index ea849d217d740c1..2b4245b290c8506 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/BarrierLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/BarrierLog.java @@ -20,6 +20,7 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.thrift.TBinlogType; import com.google.gson.annotations.SerializedName; @@ -37,6 +38,11 @@ public class BarrierLog implements Writable { @SerializedName(value = "tableName") String tableName; + @SerializedName(value = "binlogType") + int binlogType; + @SerializedName(value = "binlog") + String binlog; + public BarrierLog() { } @@ -47,6 +53,28 @@ public BarrierLog(long dbId, String dbName, long tableId, String tableName) { this.tableName = tableName; } + // A trick: Wrap the binlog as part of the BarrierLog so that it can work in + // the old Doris version without breaking the compatibility. + public BarrierLog(long dbId, long tableId, TBinlogType binlogType, String binlog) { + this.dbId = dbId; + this.tableId = tableId; + this.binlogType = binlogType.getValue(); + this.binlog = binlog; + } + + public boolean hasBinlog() { + return binlog != null; + } + + public String getBinlog() { + return binlog; + } + + // null is returned if binlog is not set or binlogType is not recognized. + public TBinlogType getBinlogType() { + return binlog == null ? null : TBinlogType.findByValue(binlogType); + } + public long getDbId() { return dbId; } From cb47a94880dce3080b6cd967351f22dda67c6c5c Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Thu, 31 Oct 2024 21:13:32 +0800 Subject: [PATCH 17/82] [test](fix) add 'nonconcurrent' tag to avoid affecting other cases (#42689) ## Proposed changes Issue Number: close #xxx --- regression-test/suites/node_p0/test_frontend.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/node_p0/test_frontend.groovy b/regression-test/suites/node_p0/test_frontend.groovy index 4478a1d3709c28c..2ccc432460bd9c3 100644 --- a/regression-test/suites/node_p0/test_frontend.groovy +++ b/regression-test/suites/node_p0/test_frontend.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite("test_frontend") { +suite("test_frontend", "nonconcurrent") { def address = "127.0.0.1" def notExistPort = 12345 From 4d52f661626c56cbdc9fabc2837178325d283cf8 Mon Sep 17 00:00:00 2001 From: walter Date: Thu, 31 Oct 2024 21:13:57 +0800 Subject: [PATCH 18/82] [fix](editlog) add column rename binlog during the replay (#43012) This was introduced in #39782 --- fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index ebdc574cb111326..5f94c95a2495e45 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -323,6 +323,7 @@ public static void loadJournal(Env env, Long logId, JournalEntity journal) { case OperationType.OP_RENAME_COLUMN: { TableRenameColumnInfo info = (TableRenameColumnInfo) journal.getData(); env.replayRenameColumn(info); + env.getBinlogManager().addColumnRename(info, logId); break; } case OperationType.OP_BACKUP_JOB: { From 607adc4c855f234a62cbc68b6699dc484f1989e7 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Thu, 31 Oct 2024 21:24:54 +0800 Subject: [PATCH 19/82] [fix](build) add macro guard for hdfs.h to fix build error on Mac (#42995) --- be/src/io/hdfs_builder.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/be/src/io/hdfs_builder.cpp b/be/src/io/hdfs_builder.cpp index 9ecb8bcab348b45..59ca46e86944df8 100644 --- a/be/src/io/hdfs_builder.cpp +++ b/be/src/io/hdfs_builder.cpp @@ -27,7 +27,9 @@ #include "common/config.h" #include "common/logging.h" +#ifdef USE_HADOOP_HDFS #include "hadoop_hdfs/hdfs.h" +#endif #include "io/fs/hdfs.h" #include "util/string_util.h" From 7e3a976d20b61fdbc84a250c5e08f4f59f69a766 Mon Sep 17 00:00:00 2001 From: Yongqiang YANG <98214048+dataroaring@users.noreply.github.com> Date: Thu, 31 Oct 2024 21:26:28 +0800 Subject: [PATCH 20/82] [fix](exit) stop storage engine in exec_env before exit (#42015) --- be/src/runtime/exec_env_init.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index ff6205bf55e5d03..9e1536d5186cfd8 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -676,7 +676,7 @@ void ExecEnv::destroy() { SAFE_STOP(_write_cooldown_meta_executors); // StorageEngine must be destoried before _page_no_cache_mem_tracker.reset and _cache_manager destory - // shouldn't use SAFE_STOP. otherwise will lead to twice stop. + SAFE_STOP(_storage_engine); _storage_engine.reset(); SAFE_STOP(_spill_stream_mgr); From c1c94208e77e8df1984c1e31b9067a1397475ecc Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Thu, 31 Oct 2024 21:32:41 +0800 Subject: [PATCH 21/82] [fix](build) Fix Mac compilation error caused by namespace conflict in find_symbols.h (#43004) From #42488 --- be/src/vec/functions/url/find_symbols.h | 56 +++++++++++++------------ 1 file changed, 29 insertions(+), 27 deletions(-) diff --git a/be/src/vec/functions/url/find_symbols.h b/be/src/vec/functions/url/find_symbols.h index 0fa0588e65689ec..4eafea893f878f1 100644 --- a/be/src/vec/functions/url/find_symbols.h +++ b/be/src/vec/functions/url/find_symbols.h @@ -362,8 +362,8 @@ inline const char* find_first_symbols_dispatch(const std::string_view haystack, template inline const char* find_first_symbols(const char* begin, const char* end) { - return detail::find_first_symbols_dispatch(begin, - end); + return ::detail::find_first_symbols_dispatch(begin, + end); } /// Returning non const result for non const arguments. @@ -371,93 +371,95 @@ inline const char* find_first_symbols(const char* begin, const char* end) { template inline char* find_first_symbols(char* begin, char* end) { return const_cast( - detail::find_first_symbols_dispatch(begin, - end)); + ::detail::find_first_symbols_dispatch( + begin, end)); } inline const char* find_first_symbols(std::string_view haystack, const SearchSymbols& symbols) { - return detail::find_first_symbols_dispatch(haystack, symbols); + return ::detail::find_first_symbols_dispatch(haystack, + symbols); } template inline const char* find_first_not_symbols(const char* begin, const char* end) { - return detail::find_first_symbols_dispatch(begin, - end); + return ::detail::find_first_symbols_dispatch( + begin, end); } template inline char* find_first_not_symbols(char* begin, char* end) { return const_cast( - detail::find_first_symbols_dispatch(begin, - end)); + ::detail::find_first_symbols_dispatch( + begin, end)); } inline const char* find_first_not_symbols(std::string_view haystack, const SearchSymbols& symbols) { - return detail::find_first_symbols_dispatch(haystack, symbols); + return ::detail::find_first_symbols_dispatch(haystack, + symbols); } template inline const char* find_first_symbols_or_null(const char* begin, const char* end) { - return detail::find_first_symbols_dispatch(begin, - end); + return ::detail::find_first_symbols_dispatch( + begin, end); } template inline char* find_first_symbols_or_null(char* begin, char* end) { return const_cast( - detail::find_first_symbols_dispatch( + ::detail::find_first_symbols_dispatch( begin, end)); } inline const char* find_first_symbols_or_null(std::string_view haystack, const SearchSymbols& symbols) { - return detail::find_first_symbols_dispatch(haystack, - symbols); + return ::detail::find_first_symbols_dispatch(haystack, + symbols); } template inline const char* find_first_not_symbols_or_null(const char* begin, const char* end) { - return detail::find_first_symbols_dispatch( + return ::detail::find_first_symbols_dispatch( begin, end); } template inline char* find_first_not_symbols_or_null(char* begin, char* end) { return const_cast( - detail::find_first_symbols_dispatch( + ::detail::find_first_symbols_dispatch( begin, end)); } inline const char* find_first_not_symbols_or_null(std::string_view haystack, const SearchSymbols& symbols) { - return detail::find_first_symbols_dispatch(haystack, - symbols); + return ::detail::find_first_symbols_dispatch(haystack, + symbols); } template inline const char* find_last_symbols_or_null(const char* begin, const char* end) { - return detail::find_last_symbols_sse2(begin, - end); + return ::detail::find_last_symbols_sse2(begin, + end); } template inline char* find_last_symbols_or_null(char* begin, char* end) { return const_cast( - detail::find_last_symbols_sse2(begin, - end)); + ::detail::find_last_symbols_sse2(begin, + end)); } template inline const char* find_last_not_symbols_or_null(const char* begin, const char* end) { - return detail::find_last_symbols_sse2(begin, - end); + return ::detail::find_last_symbols_sse2(begin, + end); } template inline char* find_last_not_symbols_or_null(char* begin, char* end) { return const_cast( - detail::find_last_symbols_sse2(begin, - end)); + ::detail::find_last_symbols_sse2( + begin, end)); } /// Slightly resembles boost::split. The drawback of boost::split is that it fires a false positive in clang static analyzer. From 9ed48346c238ccf422e2cea5eb6600b4e0b668e5 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Thu, 31 Oct 2024 21:33:16 +0800 Subject: [PATCH 22/82] [refactor](metrics) Complete metrics for some operators (#42992) --- .../exec/group_commit_block_sink_operator.cpp | 52 +- .../exec/group_commit_block_sink_operator.h | 7 +- .../exec/group_commit_scan_operator.cpp | 1 + be/src/pipeline/exec/hashjoin_build_sink.cpp | 17 +- be/src/pipeline/exec/hashjoin_build_sink.h | 4 +- .../pipeline/exec/hashjoin_probe_operator.cpp | 7 +- .../pipeline/exec/hashjoin_probe_operator.h | 4 +- .../exec/jdbc_table_sink_operator.cpp | 1 + .../exec/join/process_hash_table_probe.h | 7 +- .../exec/join/process_hash_table_probe_impl.h | 16 +- .../exec/join_build_sink_operator.cpp | 8 +- .../pipeline/exec/join_build_sink_operator.h | 2 - be/src/pipeline/exec/join_probe_operator.cpp | 3 +- be/src/pipeline/exec/join_probe_operator.h | 2 +- .../exec/memory_scratch_sink_operator.cpp | 24 +- .../exec/memory_scratch_sink_operator.h | 3 + .../exec/multi_cast_data_stream_source.cpp | 13 +- .../exec/multi_cast_data_stream_source.h | 3 + .../exec/nested_loop_join_build_operator.cpp | 1 - .../exec/nested_loop_join_probe_operator.cpp | 41 +- .../exec/nested_loop_join_probe_operator.h | 77 ++- be/src/pipeline/exec/repeat_operator.cpp | 75 +- be/src/pipeline/exec/repeat_operator.h | 5 + be/src/util/arrow/row_batch.cpp | 28 +- be/src/util/arrow/row_batch.h | 8 +- .../serde/data_type_serde_arrow_test.cpp | 654 ------------------ 26 files changed, 229 insertions(+), 834 deletions(-) delete mode 100644 be/test/vec/data_types/serde/data_type_serde_arrow_test.cpp diff --git a/be/src/pipeline/exec/group_commit_block_sink_operator.cpp b/be/src/pipeline/exec/group_commit_block_sink_operator.cpp index e0171b41ab1ee8b..9f99d55d3ea9893 100644 --- a/be/src/pipeline/exec/group_commit_block_sink_operator.cpp +++ b/be/src/pipeline/exec/group_commit_block_sink_operator.cpp @@ -64,6 +64,7 @@ Status GroupCommitBlockSinkLocalState::open(RuntimeState* state) { } Status GroupCommitBlockSinkLocalState::_initialize_load_queue() { + SCOPED_TIMER(_init_load_queue_timer); auto& p = _parent->cast(); if (_state->exec_env()->wal_mgr()->is_running()) { RETURN_IF_ERROR(_state->exec_env()->group_commit_mgr()->get_first_block_load_queue( @@ -238,6 +239,17 @@ Status GroupCommitBlockSinkLocalState::_add_blocks(RuntimeState* state, return Status::OK(); } +Status GroupCommitBlockSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_init_timer); + _init_load_queue_timer = ADD_TIMER(_profile, "InitLoadQueueTime"); + _valid_and_convert_block_timer = ADD_TIMER(_profile, "ValidAndConvertBlockTime"); + _find_partition_timer = ADD_TIMER(_profile, "FindPartitionTime"); + _append_blocks_timer = ADD_TIMER(_profile, "AppendBlocksTime"); + return Status::OK(); +} + Status GroupCommitBlockSinkOperatorX::init(const TDataSink& t_sink) { RETURN_IF_ERROR(Base::init(t_sink)); DCHECK(t_sink.__isset.olap_table_sink); @@ -318,10 +330,15 @@ Status GroupCommitBlockSinkOperatorX::sink(RuntimeState* state, vectorized::Bloc std::shared_ptr block; bool has_filtered_rows = false; - RETURN_IF_ERROR(local_state._block_convertor->validate_and_convert_block( - state, input_block, block, local_state._output_vexpr_ctxs, rows, has_filtered_rows)); + { + SCOPED_TIMER(local_state._valid_and_convert_block_timer); + RETURN_IF_ERROR(local_state._block_convertor->validate_and_convert_block( + state, input_block, block, local_state._output_vexpr_ctxs, rows, + has_filtered_rows)); + } local_state._has_filtered_rows = false; if (!local_state._vpartition->is_auto_partition()) { + SCOPED_TIMER(local_state._find_partition_timer); //reuse vars for find_partition local_state._partitions.assign(rows, nullptr); local_state._filter_bitmap.Reset(rows); @@ -351,23 +368,26 @@ Status GroupCommitBlockSinkOperatorX::sink(RuntimeState* state, vectorized::Bloc } } } - - if (local_state._block_convertor->num_filtered_rows() > 0 || local_state._has_filtered_rows) { - auto cloneBlock = block->clone_without_columns(); - auto res_block = vectorized::MutableBlock::build_mutable_block(&cloneBlock); - for (int i = 0; i < rows; ++i) { - if (local_state._block_convertor->filter_map()[i]) { - continue; - } - if (local_state._filter_bitmap.Get(i)) { - continue; + { + SCOPED_TIMER(local_state._append_blocks_timer); + if (local_state._block_convertor->num_filtered_rows() > 0 || + local_state._has_filtered_rows) { + auto cloneBlock = block->clone_without_columns(); + auto res_block = vectorized::MutableBlock::build_mutable_block(&cloneBlock); + for (int i = 0; i < rows; ++i) { + if (local_state._block_convertor->filter_map()[i]) { + continue; + } + if (local_state._filter_bitmap.Get(i)) { + continue; + } + res_block.add_row(block.get(), i); } - res_block.add_row(block.get(), i); + block->swap(res_block.to_block()); } - block->swap(res_block.to_block()); + // add block into block queue + RETURN_IF_ERROR(local_state._add_block(state, block)); } - // add block into block queue - RETURN_IF_ERROR(local_state._add_block(state, block)); return wind_up(); } diff --git a/be/src/pipeline/exec/group_commit_block_sink_operator.h b/be/src/pipeline/exec/group_commit_block_sink_operator.h index 32ca0613652ae47..e469aee8df595c0 100644 --- a/be/src/pipeline/exec/group_commit_block_sink_operator.h +++ b/be/src/pipeline/exec/group_commit_block_sink_operator.h @@ -42,8 +42,8 @@ class GroupCommitBlockSinkLocalState final : public PipelineXSinkLocalState dependencies() const override { @@ -79,6 +79,11 @@ class GroupCommitBlockSinkLocalState final : public PipelineXSinkLocalState _finish_dependency; std::shared_ptr _create_plan_dependency = nullptr; std::shared_ptr _put_block_dependency = nullptr; + + RuntimeProfile::Counter* _init_load_queue_timer = nullptr; + RuntimeProfile::Counter* _valid_and_convert_block_timer = nullptr; + RuntimeProfile::Counter* _find_partition_timer = nullptr; + RuntimeProfile::Counter* _append_blocks_timer = nullptr; }; class GroupCommitBlockSinkOperatorX final diff --git a/be/src/pipeline/exec/group_commit_scan_operator.cpp b/be/src/pipeline/exec/group_commit_scan_operator.cpp index 9577639813a7604..141a5e7bf770c56 100644 --- a/be/src/pipeline/exec/group_commit_scan_operator.cpp +++ b/be/src/pipeline/exec/group_commit_scan_operator.cpp @@ -31,6 +31,7 @@ GroupCommitOperatorX::GroupCommitOperatorX(ObjectPool* pool, const TPlanNode& tn Status GroupCommitOperatorX::get_block(RuntimeState* state, vectorized::Block* block, bool* eos) { auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state.exec_time_counter()); bool find_node = false; while (!find_node && !*eos) { RETURN_IF_ERROR(local_state.load_block_queue->get_block(state, block, &find_node, eos, diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index cbe9209eeb424dd..37de9ac93d839f5 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -51,19 +51,19 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo _shared_state->build_exprs_size = _build_expr_ctxs.size(); _should_build_hash_table = true; + profile()->add_info_string("BroadcastJoin", std::to_string(p._is_broadcast_join)); if (p._is_broadcast_join) { - profile()->add_info_string("BroadcastJoin", "true"); if (state->enable_share_hash_table_for_broadcast_join()) { _should_build_hash_table = info.task_idx == 0; if (_should_build_hash_table) { - profile()->add_info_string("ShareHashTableEnabled", "true"); p._shared_hashtable_controller->set_builder_and_consumers( state->fragment_instance_id(), p.node_id()); } - } else { - profile()->add_info_string("ShareHashTableEnabled", "false"); } } + profile()->add_info_string("BuildShareHashTable", std::to_string(_should_build_hash_table)); + profile()->add_info_string("ShareHashTableEnabled", + std::to_string(state->enable_share_hash_table_for_broadcast_join())); if (!_should_build_hash_table) { _dependency->block(); _finish_dependency->block(); @@ -72,6 +72,7 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo _finish_dependency->shared_from_this()); } + _runtime_filter_init_timer = ADD_TIMER(profile(), "RuntimeFilterInitTime"); _build_blocks_memory_usage = ADD_COUNTER_WITH_LEVEL(profile(), "MemoryUsageBuildBlocks", TUnit::BYTES, 1); _hash_table_memory_usage = @@ -81,13 +82,10 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo // Build phase auto* record_profile = _should_build_hash_table ? profile() : faker_runtime_profile(); - _build_table_timer = ADD_TIMER(profile(), "BuildTableTime"); - _build_side_merge_block_timer = ADD_TIMER(profile(), "BuildSideMergeBlockTime"); + _build_table_timer = ADD_TIMER(profile(), "BuildHashTableTime"); + _build_side_merge_block_timer = ADD_TIMER(profile(), "MergeBuildBlockTime"); _build_table_insert_timer = ADD_TIMER(record_profile, "BuildTableInsertTime"); _build_expr_call_timer = ADD_TIMER(record_profile, "BuildExprCallTime"); - _build_side_compute_hash_timer = ADD_TIMER(record_profile, "BuildSideHashComputingTime"); - - _allocate_resource_timer = ADD_TIMER(profile(), "AllocateResourceTime"); // Hash Table Init RETURN_IF_ERROR(_hash_table_init(state)); @@ -256,7 +254,6 @@ Status HashJoinBuildSinkLocalState::process_build_block(RuntimeState* state, if (UNLIKELY(rows == 0)) { return Status::OK(); } - COUNTER_UPDATE(_build_rows_counter, rows); block.replace_if_overflow(); vectorized::ColumnRawPtrs raw_ptrs(_build_expr_ctxs.size()); diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index 9f1cf486fe50271..45aa1e8c8a262dc 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -94,14 +94,12 @@ class HashJoinBuildSinkLocalState final RuntimeProfile::Counter* _build_table_timer = nullptr; RuntimeProfile::Counter* _build_expr_call_timer = nullptr; RuntimeProfile::Counter* _build_table_insert_timer = nullptr; - RuntimeProfile::Counter* _build_side_compute_hash_timer = nullptr; RuntimeProfile::Counter* _build_side_merge_block_timer = nullptr; - RuntimeProfile::Counter* _allocate_resource_timer = nullptr; - RuntimeProfile::Counter* _build_blocks_memory_usage = nullptr; RuntimeProfile::Counter* _hash_table_memory_usage = nullptr; RuntimeProfile::Counter* _build_arena_memory_usage = nullptr; + RuntimeProfile::Counter* _runtime_filter_init_timer = nullptr; }; class HashJoinBuildSinkOperatorX final diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index e7b784d4d77ab39..426bfcb219dc042 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -57,13 +57,11 @@ Status HashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) _probe_arena_memory_usage = profile()->AddHighWaterMarkCounter("MemoryUsageProbeKeyArena", TUnit::BYTES, "", 1); // Probe phase - _probe_next_timer = ADD_TIMER(profile(), "ProbeFindNextTime"); _probe_expr_call_timer = ADD_TIMER(profile(), "ProbeExprCallTime"); _search_hashtable_timer = ADD_TIMER(profile(), "ProbeWhenSearchHashTableTime"); _build_side_output_timer = ADD_TIMER(profile(), "ProbeWhenBuildSideOutputTime"); _probe_side_output_timer = ADD_TIMER(profile(), "ProbeWhenProbeSideOutputTime"); - _probe_process_hashtable_timer = ADD_TIMER(profile(), "ProbeWhenProcessHashTableTime"); - _process_other_join_conjunct_timer = ADD_TIMER(profile(), "OtherJoinConjunctTime"); + _non_equal_join_conjuncts_timer = ADD_TIMER(profile(), "NonEqualJoinConjunctEvaluationTime"); _init_probe_side_timer = ADD_TIMER(profile(), "InitProbeSideTime"); return Status::OK(); } @@ -229,7 +227,6 @@ HashJoinProbeOperatorX::HashJoinProbeOperatorX(ObjectPool* pool, const TPlanNode Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) const { auto& local_state = get_local_state(state); - SCOPED_TIMER(local_state._probe_timer); if (local_state._shared_state->short_circuit_for_probe) { // If we use a short-circuit strategy, should return empty block directly. *eos = true; @@ -320,7 +317,7 @@ Status HashJoinProbeOperatorX::pull(doris::RuntimeState* state, vectorized::Bloc if constexpr (!std::is_same_v) { using HashTableCtxType = std::decay_t; if constexpr (!std::is_same_v) { - st = process_hashtable_ctx.process_data_in_hashtable( + st = process_hashtable_ctx.finish_probing( arg, mutable_join_block, &temp_block, eos, _is_mark_join); } else { st = Status::InternalError("uninited hash table"); diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.h b/be/src/pipeline/exec/hashjoin_probe_operator.h index 63673edc410fc15..1bdb9d13347d09e 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.h +++ b/be/src/pipeline/exec/hashjoin_probe_operator.h @@ -117,14 +117,12 @@ class HashJoinProbeLocalState final std::make_unique(); RuntimeProfile::Counter* _probe_expr_call_timer = nullptr; - RuntimeProfile::Counter* _probe_next_timer = nullptr; RuntimeProfile::Counter* _probe_side_output_timer = nullptr; - RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; RuntimeProfile::HighWaterMarkCounter* _probe_arena_memory_usage = nullptr; RuntimeProfile::Counter* _search_hashtable_timer = nullptr; RuntimeProfile::Counter* _init_probe_side_timer = nullptr; RuntimeProfile::Counter* _build_side_output_timer = nullptr; - RuntimeProfile::Counter* _process_other_join_conjunct_timer = nullptr; + RuntimeProfile::Counter* _non_equal_join_conjuncts_timer = nullptr; }; class HashJoinProbeOperatorX final : public JoinProbeOperatorX { diff --git a/be/src/pipeline/exec/jdbc_table_sink_operator.cpp b/be/src/pipeline/exec/jdbc_table_sink_operator.cpp index 10fd0d8e40bf253..29c881d1c281000 100644 --- a/be/src/pipeline/exec/jdbc_table_sink_operator.cpp +++ b/be/src/pipeline/exec/jdbc_table_sink_operator.cpp @@ -47,6 +47,7 @@ Status JdbcTableSinkOperatorX::open(RuntimeState* state) { Status JdbcTableSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, bool eos) { auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); + COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); RETURN_IF_ERROR(local_state.sink(state, block, eos)); return Status::OK(); } diff --git a/be/src/pipeline/exec/join/process_hash_table_probe.h b/be/src/pipeline/exec/join/process_hash_table_probe.h index 739783eb1fd1ecb..14e0edd977f57bc 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe.h @@ -87,9 +87,8 @@ struct ProcessHashTableProbe { // Process full outer join/ right join / right semi/anti join to output the join result // in hash table template - Status process_data_in_hashtable(HashTableType& hash_table_ctx, - vectorized::MutableBlock& mutable_block, - vectorized::Block* output_block, bool* eos, bool is_mark_join); + Status finish_probing(HashTableType& hash_table_ctx, vectorized::MutableBlock& mutable_block, + vectorized::Block* output_block, bool* eos, bool is_mark_join); /// For null aware join with other conjuncts, if the probe key of one row on left side is null, /// we should make this row match with all rows in build side. @@ -136,7 +135,7 @@ struct ProcessHashTableProbe { RuntimeProfile::Counter* _init_probe_side_timer = nullptr; RuntimeProfile::Counter* _build_side_output_timer = nullptr; RuntimeProfile::Counter* _probe_side_output_timer = nullptr; - RuntimeProfile::Counter* _probe_process_hashtable_timer = nullptr; + RuntimeProfile::Counter* _finish_probe_phase_timer = nullptr; size_t _right_col_idx; size_t _right_col_len; diff --git a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h index bf4325ccece042b..231c231c81326ee 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h @@ -56,7 +56,7 @@ ProcessHashTableProbe::ProcessHashTableProbe(HashJoinProbeLocalState _init_probe_side_timer(parent->_init_probe_side_timer), _build_side_output_timer(parent->_build_side_output_timer), _probe_side_output_timer(parent->_probe_side_output_timer), - _probe_process_hashtable_timer(parent->_probe_process_hashtable_timer), + _finish_probe_phase_timer(parent->_finish_probe_phase_timer), _right_col_idx((_is_right_semi_anti && !_have_other_join_conjunct) ? 0 : _parent->left_table_data_types().size()), @@ -502,7 +502,7 @@ Status ProcessHashTableProbe::do_other_join_conjuncts(vectorized::Bl return Status::OK(); } - SCOPED_TIMER(_parent->_process_other_join_conjunct_timer); + SCOPED_TIMER(_parent->_non_equal_join_conjuncts_timer); size_t orig_columns = output_block->columns(); vectorized::IColumn::Filter other_conjunct_filter(row_count, 1); { @@ -617,10 +617,11 @@ Status ProcessHashTableProbe::do_other_join_conjuncts(vectorized::Bl template template -Status ProcessHashTableProbe::process_data_in_hashtable( - HashTableType& hash_table_ctx, vectorized::MutableBlock& mutable_block, - vectorized::Block* output_block, bool* eos, bool is_mark_join) { - SCOPED_TIMER(_probe_process_hashtable_timer); +Status ProcessHashTableProbe::finish_probing(HashTableType& hash_table_ctx, + vectorized::MutableBlock& mutable_block, + vectorized::Block* output_block, bool* eos, + bool is_mark_join) { + SCOPED_TIMER(_finish_probe_phase_timer); auto& mcol = mutable_block.mutable_columns(); if (is_mark_join) { std::unique_ptr mark_column = @@ -709,8 +710,7 @@ struct ExtractType { ExtractType::Type & hash_table_ctx, vectorized::ConstNullMapPtr null_map, \ vectorized::MutableBlock & mutable_block, vectorized::Block * output_block, \ uint32_t probe_rows, bool is_mark_join, bool have_other_join_conjunct); \ - template Status \ - ProcessHashTableProbe::process_data_in_hashtable::Type>( \ + template Status ProcessHashTableProbe::finish_probing::Type>( \ ExtractType::Type & hash_table_ctx, vectorized::MutableBlock & mutable_block, \ vectorized::Block * output_block, bool* eos, bool is_mark_join); diff --git a/be/src/pipeline/exec/join_build_sink_operator.cpp b/be/src/pipeline/exec/join_build_sink_operator.cpp index fc0d3b8746077ba..8b3f5cd98ff7c06 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.cpp +++ b/be/src/pipeline/exec/join_build_sink_operator.cpp @@ -33,15 +33,11 @@ Status JoinBuildSinkLocalState::init(RuntimeState* stat PipelineXSinkLocalState::profile()->add_info_string("JoinType", to_string(p._join_op)); - _build_rows_counter = ADD_COUNTER(PipelineXSinkLocalState::profile(), - "BuildRows", TUnit::UNIT); _publish_runtime_filter_timer = ADD_TIMER(PipelineXSinkLocalState::profile(), "PublishRuntimeFilterTime"); - _runtime_filter_compute_timer = ADD_TIMER(PipelineXSinkLocalState::profile(), - "RuntimeFilterComputeTime"); - _runtime_filter_init_timer = - ADD_TIMER(PipelineXSinkLocalState::profile(), "RuntimeFilterInitTime"); + _runtime_filter_compute_timer = + ADD_TIMER(PipelineXSinkLocalState::profile(), "BuildRuntimeFilterTime"); return Status::OK(); } diff --git a/be/src/pipeline/exec/join_build_sink_operator.h b/be/src/pipeline/exec/join_build_sink_operator.h index 714e0c341906781..9d79a97397ff776 100644 --- a/be/src/pipeline/exec/join_build_sink_operator.h +++ b/be/src/pipeline/exec/join_build_sink_operator.h @@ -39,10 +39,8 @@ class JoinBuildSinkLocalState : public PipelineXSinkLocalState template friend class JoinBuildSinkOperatorX; - RuntimeProfile::Counter* _build_rows_counter = nullptr; RuntimeProfile::Counter* _publish_runtime_filter_timer = nullptr; RuntimeProfile::Counter* _runtime_filter_compute_timer = nullptr; - RuntimeProfile::Counter* _runtime_filter_init_timer = nullptr; std::vector> _runtime_filters; }; diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 76dc75a90d8f3c4..11b5b29c8b556b1 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -29,11 +29,10 @@ Status JoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); - _probe_timer = ADD_TIMER(Base::profile(), "ProbeTime"); _join_filter_timer = ADD_TIMER(Base::profile(), "JoinFilterTimer"); _build_output_block_timer = ADD_TIMER(Base::profile(), "BuildOutputBlock"); _probe_rows_counter = ADD_COUNTER_WITH_LEVEL(Base::profile(), "ProbeRows", TUnit::UNIT, 1); - + _finish_probe_phase_timer = ADD_TIMER(Base::profile(), "FinishProbePhaseTime"); return Status::OK(); } diff --git a/be/src/pipeline/exec/join_probe_operator.h b/be/src/pipeline/exec/join_probe_operator.h index 3f68c73d04b1612..078806cea4fc5ac 100644 --- a/be/src/pipeline/exec/join_probe_operator.h +++ b/be/src/pipeline/exec/join_probe_operator.h @@ -49,10 +49,10 @@ class JoinProbeLocalState : public PipelineXLocalState { size_t _mark_column_id = -1; - RuntimeProfile::Counter* _probe_timer = nullptr; RuntimeProfile::Counter* _probe_rows_counter = nullptr; RuntimeProfile::Counter* _join_filter_timer = nullptr; RuntimeProfile::Counter* _build_output_block_timer = nullptr; + RuntimeProfile::Counter* _finish_probe_phase_timer = nullptr; std::unique_ptr _child_block = nullptr; bool _child_eos = false; diff --git a/be/src/pipeline/exec/memory_scratch_sink_operator.cpp b/be/src/pipeline/exec/memory_scratch_sink_operator.cpp index 1d022f9304fd0de..2c69c0e2b2ba9fc 100644 --- a/be/src/pipeline/exec/memory_scratch_sink_operator.cpp +++ b/be/src/pipeline/exec/memory_scratch_sink_operator.cpp @@ -33,6 +33,9 @@ Status MemoryScratchSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _get_arrow_schema_timer = ADD_TIMER(_profile, "GetArrowSchemaTime"); + _convert_block_to_arrow_batch_timer = ADD_TIMER(_profile, "ConvertBlockToArrowBatchTime"); + _evaluation_timer = ADD_TIMER(_profile, "EvaluationTime"); // create queue state->exec_env()->result_queue_mgr()->create_queue(state->fragment_instance_id(), &_queue); @@ -92,13 +95,22 @@ Status MemoryScratchSinkOperatorX::sink(RuntimeState* state, vectorized::Block* // Exec vectorized expr here to speed up, block.rows() == 0 means expr exec // failed, just return the error status vectorized::Block block; - RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( - local_state._output_vexpr_ctxs, *input_block, &block)); + { + SCOPED_TIMER(local_state._evaluation_timer); + RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( + local_state._output_vexpr_ctxs, *input_block, &block)); + } std::shared_ptr block_arrow_schema; - // After expr executed, use recaculated schema as final schema - RETURN_IF_ERROR(convert_block_arrow_schema(block, &block_arrow_schema, state->timezone())); - RETURN_IF_ERROR(convert_to_arrow_batch(block, block_arrow_schema, arrow::default_memory_pool(), - &result, _timezone_obj)); + { + SCOPED_TIMER(local_state._get_arrow_schema_timer); + // After expr executed, use recaculated schema as final schema + RETURN_IF_ERROR(get_arrow_schema(block, &block_arrow_schema, state->timezone())); + } + { + SCOPED_TIMER(local_state._convert_block_to_arrow_batch_timer); + RETURN_IF_ERROR(convert_to_arrow_batch( + block, block_arrow_schema, arrow::default_memory_pool(), &result, _timezone_obj)); + } local_state._queue->blocking_put(result); if (local_state._queue->size() > config::max_memory_sink_batch_count) { local_state._queue_dependency->block(); diff --git a/be/src/pipeline/exec/memory_scratch_sink_operator.h b/be/src/pipeline/exec/memory_scratch_sink_operator.h index 69c0fa14042ef28..c74659d15b96f29 100644 --- a/be/src/pipeline/exec/memory_scratch_sink_operator.h +++ b/be/src/pipeline/exec/memory_scratch_sink_operator.h @@ -45,6 +45,9 @@ class MemoryScratchSinkLocalState final : public PipelineXSinkLocalState _queue_dependency = nullptr; + RuntimeProfile::Counter* _get_arrow_schema_timer = nullptr; + RuntimeProfile::Counter* _convert_block_to_arrow_batch_timer = nullptr; + RuntimeProfile::Counter* _evaluation_timer = nullptr; }; class MemoryScratchSinkOperatorX final : public DataSinkOperatorX { diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp index 71204f1285ce7b1..e45e59d17e27b37 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp @@ -40,6 +40,9 @@ Status MultiCastDataStreamSourceLocalState::init(RuntimeState* state, LocalState auto& p = _parent->cast(); _shared_state->multi_cast_data_streamer->set_dep_by_sender_idx(p._consumer_id, _dependency); _wait_for_rf_timer = ADD_TIMER(_runtime_profile, "WaitForRuntimeFilter"); + _filter_timer = ADD_TIMER(_runtime_profile, "FilterTime"); + _get_data_timer = ADD_TIMER(_runtime_profile, "GetDataTime"); + _materialize_data_timer = ADD_TIMER(_runtime_profile, "MaterializeDataTime"); // init profile for runtime filter RuntimeFilterConsumer::_init_profile(profile()); init_runtime_filter_dependency(_filter_dependencies, p.operator_id(), p.node_id(), @@ -86,15 +89,19 @@ Status MultiCastDataStreamerSourceOperatorX::get_block(RuntimeState* state, if (!local_state._output_expr_contexts.empty()) { output_block = &tmp_block; } - RETURN_IF_ERROR(local_state._shared_state->multi_cast_data_streamer->pull(_consumer_id, - output_block, eos)); - + { + SCOPED_TIMER(local_state._get_data_timer); + RETURN_IF_ERROR(local_state._shared_state->multi_cast_data_streamer->pull( + _consumer_id, output_block, eos)); + } if (!local_state._conjuncts.empty()) { + SCOPED_TIMER(local_state._filter_timer); RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, output_block->columns())); } if (!local_state._output_expr_contexts.empty() && output_block->rows() > 0) { + SCOPED_TIMER(local_state._materialize_data_timer); RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs( local_state._output_expr_contexts, *output_block, block, true)); vectorized::materialize_block_inplace(*block); diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.h b/be/src/pipeline/exec/multi_cast_data_stream_source.h index 2059f706cad3f50..57410bf8d9568a5 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -68,6 +68,9 @@ class MultiCastDataStreamSourceLocalState final : public PipelineXLocalState> _filter_dependencies; RuntimeProfile::Counter* _wait_for_rf_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; + RuntimeProfile::Counter* _get_data_timer = nullptr; + RuntimeProfile::Counter* _materialize_data_timer = nullptr; }; class MultiCastDataStreamerSourceOperatorX final diff --git a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp index 59020a5df437bde..83b378e792c3fa3 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.cpp @@ -139,7 +139,6 @@ Status NestedLoopJoinBuildSinkOperatorX::sink(doris::RuntimeState* state, vector } if (eos) { - COUNTER_UPDATE(local_state._build_rows_counter, local_state._build_rows); RuntimeFilterBuild rf_ctx(&local_state); RETURN_IF_ERROR(rf_ctx(state)); diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp index d0fb4ee19a58249..afa1a2e59b798ce 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp @@ -43,6 +43,10 @@ Status NestedLoopJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); _loop_join_timer = ADD_TIMER(profile(), "LoopGenerateJoin"); + _output_temp_blocks_timer = ADD_TIMER(profile(), "OutputTempBlocksTime"); + _update_visited_flags_timer = ADD_TIMER(profile(), "UpdateVisitedFlagsTime"); + _join_conjuncts_evaluation_timer = ADD_TIMER(profile(), "JoinConjunctsEvaluationTime"); + _filtered_by_join_conjuncts_timer = ADD_TIMER(profile(), "FilteredByJoinConjunctsTime"); return Status::OK(); } @@ -168,23 +172,26 @@ Status NestedLoopJoinProbeLocalState::generate_join_block_data(RuntimeState* sta _process_left_child_block(_join_block, now_process_build_block); } - if constexpr (set_probe_side_flag) { - RETURN_IF_ERROR( - (_do_filtering_and_update_visited_flags( - &_join_block, !p._is_left_semi_anti))); - _update_additional_flags(&_join_block); - // If this join operation is left outer join or full outer join, when - // `_left_side_process_count`, means all rows from build - // side have been joined with _left_side_process_count, we should output current - // probe row with null from build side. - if (_left_side_process_count) { - _finalize_current_phase( - _join_block, state->batch_size()); + { + SCOPED_TIMER(_finish_probe_phase_timer); + if constexpr (set_probe_side_flag) { + RETURN_IF_ERROR( + (_do_filtering_and_update_visited_flags( + &_join_block, !p._is_left_semi_anti))); + _update_additional_flags(&_join_block); + // If this join operation is left outer join or full outer join, when + // `_left_side_process_count`, means all rows from build + // side have been joined with _left_side_process_count, we should output current + // probe row with null from build side. + if (_left_side_process_count) { + _finalize_current_phase( + _join_block, state->batch_size()); + } + } else if (_left_side_process_count && p._is_mark_join && + _shared_state->build_blocks.empty()) { + _append_left_data_with_null(_join_block); } - } else if (_left_side_process_count && p._is_mark_join && - _shared_state->build_blocks.empty()) { - _append_left_data_with_null(_join_block); } } @@ -377,6 +384,7 @@ void NestedLoopJoinProbeLocalState::_append_left_data_with_null(vectorized::Bloc void NestedLoopJoinProbeLocalState::_process_left_child_block( vectorized::Block& block, const vectorized::Block& now_process_build_block) const { + SCOPED_TIMER(_output_temp_blocks_timer); auto& p = _parent->cast(); auto dst_columns = block.mutate_columns(); const size_t max_added_rows = now_process_build_block.rows(); @@ -485,6 +493,7 @@ Status NestedLoopJoinProbeOperatorX::push(doris::RuntimeState* state, vectorized set_build_side_flag, set_probe_side_flag>( state, join_op_variants); }; + SCOPED_TIMER(local_state._loop_join_timer); RETURN_IF_ERROR( std::visit(func, local_state._shared_state->join_op_variants, vectorized::make_bool_variant(_match_all_build || _is_right_semi_anti), diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.h b/be/src/pipeline/exec/nested_loop_join_probe_operator.h index 5b0fec159e28bf1..c744e6acdc507e3 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.h @@ -68,42 +68,48 @@ class NestedLoopJoinProbeLocalState final size_t build_block_idx, size_t processed_blocks_num, bool materialize, Filter& filter) { - if constexpr (SetBuildSideFlag) { - for (size_t i = 0; i < processed_blocks_num; i++) { - auto& build_side_flag = - assert_cast( - _shared_state->build_side_visited_flags[build_block_idx].get()) - ->get_data(); - auto* __restrict build_side_flag_data = build_side_flag.data(); - auto cur_sz = build_side_flag.size(); - const size_t offset = _build_offset_stack.top(); - _build_offset_stack.pop(); - for (size_t j = 0; j < cur_sz; j++) { - build_side_flag_data[j] |= filter[offset + j]; + { + SCOPED_TIMER(_update_visited_flags_timer); + if constexpr (SetBuildSideFlag) { + for (size_t i = 0; i < processed_blocks_num; i++) { + auto& build_side_flag = + assert_cast( + _shared_state->build_side_visited_flags[build_block_idx].get()) + ->get_data(); + auto* __restrict build_side_flag_data = build_side_flag.data(); + auto cur_sz = build_side_flag.size(); + const size_t offset = _build_offset_stack.top(); + _build_offset_stack.pop(); + for (size_t j = 0; j < cur_sz; j++) { + build_side_flag_data[j] |= filter[offset + j]; + } + build_block_idx = build_block_idx == 0 ? _shared_state->build_blocks.size() - 1 + : build_block_idx - 1; } - build_block_idx = build_block_idx == 0 ? _shared_state->build_blocks.size() - 1 - : build_block_idx - 1; } - } - if constexpr (SetProbeSideFlag) { - int64_t end = filter.size(); - for (int i = _left_block_pos == _child_block->rows() ? _left_block_pos - 1 - : _left_block_pos; - i >= _left_block_start_pos; i--) { - int64_t offset = 0; - if (!_probe_offset_stack.empty()) { - offset = _probe_offset_stack.top(); - _probe_offset_stack.pop(); - } - if (!_cur_probe_row_visited_flags[i]) { - _cur_probe_row_visited_flags[i] = - simd::contain_byte(filter.data() + offset, end - offset, 1) ? 1 - : 0; + if constexpr (SetProbeSideFlag) { + int64_t end = filter.size(); + for (int i = _left_block_pos == _child_block->rows() ? _left_block_pos - 1 + : _left_block_pos; + i >= _left_block_start_pos; i--) { + int64_t offset = 0; + if (!_probe_offset_stack.empty()) { + offset = _probe_offset_stack.top(); + _probe_offset_stack.pop(); + } + if (!_cur_probe_row_visited_flags[i]) { + _cur_probe_row_visited_flags[i] = + simd::contain_byte(filter.data() + offset, end - offset, 1) + ? 1 + : 0; + } + end = offset; } - end = offset; } } + if (materialize) { + SCOPED_TIMER(_filtered_by_join_conjuncts_timer); vectorized::Block::filter_block_internal(block, filter, column_to_keep); } else { CLEAR_BLOCK @@ -125,8 +131,11 @@ class NestedLoopJoinProbeLocalState final if (LIKELY(!_join_conjuncts.empty() && block->rows() > 0)) { vectorized::IColumn::Filter filter(block->rows(), 1); bool can_filter_all = false; - RETURN_IF_ERROR(vectorized::VExprContext::execute_conjuncts( - _join_conjuncts, nullptr, IgnoreNull, block, &filter, &can_filter_all)); + { + SCOPED_TIMER(_join_conjuncts_evaluation_timer); + RETURN_IF_ERROR(vectorized::VExprContext::execute_conjuncts( + _join_conjuncts, nullptr, IgnoreNull, block, &filter, &can_filter_all)); + } if (can_filter_all) { CLEAR_BLOCK @@ -185,6 +194,10 @@ class NestedLoopJoinProbeLocalState final vectorized::VExprContextSPtrs _join_conjuncts; RuntimeProfile::Counter* _loop_join_timer = nullptr; + RuntimeProfile::Counter* _output_temp_blocks_timer = nullptr; + RuntimeProfile::Counter* _update_visited_flags_timer = nullptr; + RuntimeProfile::Counter* _join_conjuncts_evaluation_timer = nullptr; + RuntimeProfile::Counter* _filtered_by_join_conjuncts_timer = nullptr; }; class NestedLoopJoinProbeOperatorX final diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp index dba4f27af7c385d..5c94d43f0d1e05d 100644 --- a/be/src/pipeline/exec/repeat_operator.cpp +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -46,6 +46,16 @@ Status RepeatLocalState::open(RuntimeState* state) { return Status::OK(); } +Status RepeatLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_init_timer); + _evaluate_input_timer = ADD_TIMER(profile(), "EvaluateInputDataTime"); + _get_repeat_data_timer = ADD_TIMER(profile(), "GetRepeatDataTime"); + _filter_timer = ADD_TIMER(profile(), "FilterTime"); + return Status::OK(); +} + Status RepeatOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(OperatorXBase::init(tnode, state)); RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(tnode.repeat_node.exprs, _expr_ctxs)); @@ -166,23 +176,24 @@ Status RepeatLocalState::add_grouping_id_column(std::size_t rows, std::size_t& c Status RepeatOperatorX::push(RuntimeState* state, vectorized::Block* input_block, bool eos) const { auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state._evaluate_input_timer); local_state._child_eos = eos; - auto& _intermediate_block = local_state._intermediate_block; - auto& _expr_ctxs = local_state._expr_ctxs; - DCHECK(!_intermediate_block || _intermediate_block->rows() == 0); + auto& intermediate_block = local_state._intermediate_block; + auto& expr_ctxs = local_state._expr_ctxs; + DCHECK(!intermediate_block || intermediate_block->rows() == 0); if (input_block->rows() > 0) { - _intermediate_block = vectorized::Block::create_unique(); + intermediate_block = vectorized::Block::create_unique(); - for (auto& expr : _expr_ctxs) { + for (auto& expr : expr_ctxs) { int result_column_id = -1; RETURN_IF_ERROR(expr->execute(input_block, &result_column_id)); DCHECK(result_column_id != -1); input_block->get_by_position(result_column_id).column = input_block->get_by_position(result_column_id) .column->convert_to_full_column_if_const(); - _intermediate_block->insert(input_block->get_by_position(result_column_id)); + intermediate_block->insert(input_block->get_by_position(result_column_id)); } - DCHECK_EQ(_expr_ctxs.size(), _intermediate_block->columns()); + DCHECK_EQ(expr_ctxs.size(), intermediate_block->columns()); } return Status::OK(); @@ -202,33 +213,39 @@ Status RepeatOperatorX::pull(doris::RuntimeState* state, vectorized::Block* outp } DCHECK(output_block->rows() == 0); - if (_intermediate_block && _intermediate_block->rows() > 0) { - RETURN_IF_ERROR(local_state.get_repeated_block(_intermediate_block.get(), _repeat_id_idx, - output_block)); + { + SCOPED_TIMER(local_state._get_repeat_data_timer); + if (_intermediate_block && _intermediate_block->rows() > 0) { + RETURN_IF_ERROR(local_state.get_repeated_block(_intermediate_block.get(), + _repeat_id_idx, output_block)); - _repeat_id_idx++; + _repeat_id_idx++; - int size = _repeat_id_list.size(); - if (_repeat_id_idx >= size) { - _intermediate_block->clear(); + int size = _repeat_id_list.size(); + if (_repeat_id_idx >= size) { + _intermediate_block->clear(); + _child_block.clear_column_data(_child->row_desc().num_materialized_slots()); + _repeat_id_idx = 0; + } + } else if (local_state._expr_ctxs.empty()) { + auto m_block = vectorized::VectorizedUtils::build_mutable_mem_reuse_block( + output_block, _output_slots); + auto rows = _child_block.rows(); + auto& columns = m_block.mutable_columns(); + + for (int repeat_id_idx = 0; repeat_id_idx < _repeat_id_list.size(); repeat_id_idx++) { + std::size_t cur_col = 0; + RETURN_IF_ERROR( + local_state.add_grouping_id_column(rows, cur_col, columns, repeat_id_idx)); + } _child_block.clear_column_data(_child->row_desc().num_materialized_slots()); - _repeat_id_idx = 0; } - } else if (local_state._expr_ctxs.empty()) { - auto m_block = vectorized::VectorizedUtils::build_mutable_mem_reuse_block(output_block, - _output_slots); - auto rows = _child_block.rows(); - auto& columns = m_block.mutable_columns(); - - for (int repeat_id_idx = 0; repeat_id_idx < _repeat_id_list.size(); repeat_id_idx++) { - std::size_t cur_col = 0; - RETURN_IF_ERROR( - local_state.add_grouping_id_column(rows, cur_col, columns, repeat_id_idx)); - } - _child_block.clear_column_data(_child->row_desc().num_materialized_slots()); } - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, - output_block->columns())); + { + SCOPED_TIMER(local_state._filter_timer); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, output_block, + output_block->columns())); + } *eos = _child_eos && _child_block.rows() == 0; local_state.reached_limit(output_block, eos); return Status::OK(); diff --git a/be/src/pipeline/exec/repeat_operator.h b/be/src/pipeline/exec/repeat_operator.h index 22398df372ae654..31f88f37231aaae 100644 --- a/be/src/pipeline/exec/repeat_operator.h +++ b/be/src/pipeline/exec/repeat_operator.h @@ -36,6 +36,7 @@ class RepeatLocalState final : public PipelineXLocalState { using Base = PipelineXLocalState; RepeatLocalState(RuntimeState* state, OperatorXBase* parent); + Status init(RuntimeState* state, LocalStateInfo& info) override; Status open(RuntimeState* state) override; Status get_repeated_block(vectorized::Block* child_block, int repeat_id_idx, @@ -53,6 +54,10 @@ class RepeatLocalState final : public PipelineXLocalState { int _repeat_id_idx; std::unique_ptr _intermediate_block; vectorized::VExprContextSPtrs _expr_ctxs; + + RuntimeProfile::Counter* _evaluate_input_timer = nullptr; + RuntimeProfile::Counter* _get_repeat_data_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; }; class RepeatOperatorX final : public StatefulOperatorX { diff --git a/be/src/util/arrow/row_batch.cpp b/be/src/util/arrow/row_batch.cpp index 2c6ed52ddde65fa..0cbb6bcd0c8916d 100644 --- a/be/src/util/arrow/row_batch.cpp +++ b/be/src/util/arrow/row_batch.cpp @@ -157,17 +157,8 @@ Status convert_to_arrow_type(const TypeDescriptor& type, std::shared_ptr* field, - const std::string& timezone) { - std::shared_ptr type; - RETURN_IF_ERROR(convert_to_arrow_type(desc->type(), &type, timezone)); - *field = arrow::field(desc->col_name(), type, desc->is_nullable()); - return Status::OK(); -} - -Status convert_block_arrow_schema(const vectorized::Block& block, - std::shared_ptr* result, - const std::string& timezone) { +Status get_arrow_schema(const vectorized::Block& block, std::shared_ptr* result, + const std::string& timezone) { std::vector> fields; for (const auto& type_and_name : block) { std::shared_ptr arrow_type; @@ -180,21 +171,6 @@ Status convert_block_arrow_schema(const vectorized::Block& block, return Status::OK(); } -Status convert_to_arrow_schema(const RowDescriptor& row_desc, - std::shared_ptr* result, - const std::string& timezone) { - std::vector> fields; - for (auto tuple_desc : row_desc.tuple_descriptors()) { - for (auto desc : tuple_desc->slots()) { - std::shared_ptr field; - RETURN_IF_ERROR(convert_to_arrow_field(desc, &field, timezone)); - fields.push_back(field); - } - } - *result = arrow::schema(std::move(fields)); - return Status::OK(); -} - Status convert_expr_ctxs_arrow_schema(const vectorized::VExprContextSPtrs& output_vexpr_ctxs, std::shared_ptr* result, const std::string& timezone) { diff --git a/be/src/util/arrow/row_batch.h b/be/src/util/arrow/row_batch.h index 9a33719a1cfbcce..3993003baf6e95c 100644 --- a/be/src/util/arrow/row_batch.h +++ b/be/src/util/arrow/row_batch.h @@ -45,12 +45,8 @@ Status convert_to_arrow_type(const TypeDescriptor& type, std::shared_ptr* result, const std::string& timezone); - -Status convert_block_arrow_schema(const vectorized::Block& block, - std::shared_ptr* result, - const std::string& timezone); +Status get_arrow_schema(const vectorized::Block& block, std::shared_ptr* result, + const std::string& timezone); Status convert_expr_ctxs_arrow_schema(const vectorized::VExprContextSPtrs& output_vexpr_ctxs, std::shared_ptr* result, diff --git a/be/test/vec/data_types/serde/data_type_serde_arrow_test.cpp b/be/test/vec/data_types/serde/data_type_serde_arrow_test.cpp deleted file mode 100644 index fc692b8f67569e4..000000000000000 --- a/be/test/vec/data_types/serde/data_type_serde_arrow_test.cpp +++ /dev/null @@ -1,654 +0,0 @@ - -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -#include "gtest/gtest_pred_impl.h" -#include "olap/hll.h" -#include "runtime/descriptors.cpp" -#include "runtime/descriptors.h" -#include "util/arrow/block_convertor.h" -#include "util/arrow/row_batch.h" -#include "util/bitmap_value.h" -#include "util/quantile_state.h" -#include "util/string_parser.hpp" -#include "vec/columns/column.h" -#include "vec/columns/column_array.h" -#include "vec/columns/column_complex.h" -#include "vec/columns/column_decimal.h" -#include "vec/columns/column_map.h" -#include "vec/columns/column_nullable.h" -#include "vec/columns/column_string.h" -#include "vec/columns/column_vector.h" -#include "vec/core/block.h" -#include "vec/core/field.h" -#include "vec/core/types.h" -#include "vec/data_types/data_type.h" -#include "vec/data_types/data_type_array.h" -#include "vec/data_types/data_type_bitmap.h" -#include "vec/data_types/data_type_date.h" -#include "vec/data_types/data_type_date_time.h" -#include "vec/data_types/data_type_decimal.h" -#include "vec/data_types/data_type_hll.h" -#include "vec/data_types/data_type_ipv4.h" -#include "vec/data_types/data_type_ipv6.h" -#include "vec/data_types/data_type_map.h" -#include "vec/data_types/data_type_nullable.h" -#include "vec/data_types/data_type_number.h" -#include "vec/data_types/data_type_quantilestate.h" -#include "vec/data_types/data_type_string.h" -#include "vec/data_types/data_type_struct.h" -#include "vec/data_types/data_type_time_v2.h" -#include "vec/io/io_helper.h" -#include "vec/runtime/vdatetime_value.h" -#include "vec/utils/arrow_column_to_doris_column.h" - -namespace doris::vectorized { - -template -void serialize_and_deserialize_arrow_test() { - vectorized::Block block; - std::vector> cols; - if constexpr (is_scalar) { - cols = { - {"k1", FieldType::OLAP_FIELD_TYPE_INT, 1, TYPE_INT, false}, - {"k7", FieldType::OLAP_FIELD_TYPE_INT, 7, TYPE_INT, true}, - {"k2", FieldType::OLAP_FIELD_TYPE_STRING, 2, TYPE_STRING, false}, - {"k3", FieldType::OLAP_FIELD_TYPE_DECIMAL128I, 3, TYPE_DECIMAL128I, false}, - {"k11", FieldType::OLAP_FIELD_TYPE_DATETIME, 11, TYPE_DATETIME, false}, - {"k4", FieldType::OLAP_FIELD_TYPE_BOOL, 4, TYPE_BOOLEAN, false}, - {"k5", FieldType::OLAP_FIELD_TYPE_DECIMAL32, 5, TYPE_DECIMAL32, false}, - {"k6", FieldType::OLAP_FIELD_TYPE_DECIMAL64, 6, TYPE_DECIMAL64, false}, - {"k12", FieldType::OLAP_FIELD_TYPE_DATETIMEV2, 12, TYPE_DATETIMEV2, false}, - {"k8", FieldType::OLAP_FIELD_TYPE_IPV4, 8, TYPE_IPV4, false}, - {"k9", FieldType::OLAP_FIELD_TYPE_IPV6, 9, TYPE_IPV6, false}, - }; - } else { - cols = {{"a", FieldType::OLAP_FIELD_TYPE_ARRAY, 6, TYPE_ARRAY, true}, - {"m", FieldType::OLAP_FIELD_TYPE_MAP, 8, TYPE_MAP, true}, - {"s", FieldType::OLAP_FIELD_TYPE_STRUCT, 5, TYPE_STRUCT, true}}; - } - - int row_num = 7; - // make desc and generate block - TupleDescriptor tuple_desc(PTupleDescriptor(), true); - for (auto t : cols) { - TSlotDescriptor tslot; - std::string col_name = std::get<0>(t); - tslot.__set_colName(col_name); - TypeDescriptor type_desc(std::get<3>(t)); - bool is_nullable(std::get<4>(t)); - switch (std::get<3>(t)) { - case TYPE_BOOLEAN: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto vec = vectorized::ColumnVector::create(); - auto& data = vec->get_data(); - for (int i = 0; i < row_num; ++i) { - data.push_back(i % 2); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, - col_name); - block.insert(std::move(type_and_name)); - } - break; - case TYPE_INT: - tslot.__set_slotType(type_desc.to_thrift()); - if (is_nullable) { - { - auto column_vector_int32 = vectorized::ColumnVector::create(); - auto column_nullable_vector = - vectorized::make_nullable(std::move(column_vector_int32)); - auto mutable_nullable_vector = std::move(*column_nullable_vector).mutate(); - for (int i = 0; i < row_num; i++) { - if (i % 2 == 0) { - mutable_nullable_vector->insert_default(); - } else { - mutable_nullable_vector->insert(int32(i)); - } - } - auto data_type = vectorized::make_nullable( - std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name( - mutable_nullable_vector->get_ptr(), data_type, col_name); - block.insert(type_and_name); - } - } else { - auto vec = vectorized::ColumnVector::create(); - auto& data = vec->get_data(); - for (int i = 0; i < row_num; ++i) { - data.push_back(i); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, - col_name); - block.insert(std::move(type_and_name)); - } - break; - case TYPE_DECIMAL32: - type_desc.precision = 9; - type_desc.scale = 2; - tslot.__set_slotType(type_desc.to_thrift()); - { - vectorized::DataTypePtr decimal_data_type = - std::make_shared>(type_desc.precision, - type_desc.scale); - auto decimal_column = decimal_data_type->create_column(); - auto& data = ((vectorized::ColumnDecimal>*) - decimal_column.get()) - ->get_data(); - for (int i = 0; i < row_num; ++i) { - if (i == 0) { - data.push_back(Int32(0)); - continue; - } - Int32 val; - StringParser::ParseResult result = StringParser::PARSE_SUCCESS; - i % 2 == 0 ? val = StringParser::string_to_decimal( - "1234567.56", 11, type_desc.precision, type_desc.scale, - &result) - : val = StringParser::string_to_decimal( - "-1234567.56", 12, type_desc.precision, type_desc.scale, - &result); - EXPECT_TRUE(result == StringParser::PARSE_SUCCESS); - data.push_back(val); - } - - vectorized::ColumnWithTypeAndName type_and_name(decimal_column->get_ptr(), - decimal_data_type, col_name); - block.insert(type_and_name); - } - break; - case TYPE_DECIMAL64: - type_desc.precision = 18; - type_desc.scale = 6; - tslot.__set_slotType(type_desc.to_thrift()); - { - vectorized::DataTypePtr decimal_data_type = - std::make_shared>(type_desc.precision, - type_desc.scale); - auto decimal_column = decimal_data_type->create_column(); - auto& data = ((vectorized::ColumnDecimal>*) - decimal_column.get()) - ->get_data(); - for (int i = 0; i < row_num; ++i) { - if (i == 0) { - data.push_back(Int64(0)); - continue; - } - Int64 val; - StringParser::ParseResult result = StringParser::PARSE_SUCCESS; - std::string decimal_string = - i % 2 == 0 ? "-123456789012.123456" : "123456789012.123456"; - val = StringParser::string_to_decimal( - decimal_string.c_str(), decimal_string.size(), type_desc.precision, - type_desc.scale, &result); - EXPECT_TRUE(result == StringParser::PARSE_SUCCESS); - data.push_back(val); - } - vectorized::ColumnWithTypeAndName type_and_name(decimal_column->get_ptr(), - decimal_data_type, col_name); - block.insert(type_and_name); - } - break; - case TYPE_DECIMAL128I: - type_desc.precision = 27; - type_desc.scale = 9; - tslot.__set_slotType(type_desc.to_thrift()); - { - vectorized::DataTypePtr decimal_data_type( - doris::vectorized::create_decimal(27, 9, true)); - auto decimal_column = decimal_data_type->create_column(); - auto& data = ((vectorized::ColumnDecimal>*) - decimal_column.get()) - ->get_data(); - for (int i = 0; i < row_num; ++i) { - __int128_t value = __int128_t(i * pow(10, 9) + i * pow(10, 8)); - data.push_back(value); - } - vectorized::ColumnWithTypeAndName type_and_name(decimal_column->get_ptr(), - decimal_data_type, col_name); - block.insert(type_and_name); - } - break; - case TYPE_STRING: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto strcol = vectorized::ColumnString::create(); - for (int i = 0; i < row_num; ++i) { - std::string is = std::to_string(i); - strcol->insert_data(is.c_str(), is.size()); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(strcol->get_ptr(), data_type, - col_name); - block.insert(type_and_name); - } - break; - case TYPE_HLL: - tslot.__set_slotType(type_desc.to_thrift()); - { - vectorized::DataTypePtr hll_data_type(std::make_shared()); - auto hll_column = hll_data_type->create_column(); - std::vector& container = - ((vectorized::ColumnHLL*)hll_column.get())->get_data(); - for (int i = 0; i < row_num; ++i) { - HyperLogLog hll; - hll.update(i); - container.push_back(hll); - } - vectorized::ColumnWithTypeAndName type_and_name(hll_column->get_ptr(), - hll_data_type, col_name); - - block.insert(type_and_name); - } - break; - case TYPE_DATEV2: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto column_vector_date_v2 = vectorized::ColumnVector::create(); - auto& date_v2_data = column_vector_date_v2->get_data(); - for (int i = 0; i < row_num; ++i) { - DateV2Value value; - value.from_date((uint32_t)((2022 << 9) | (6 << 5) | 6)); - date_v2_data.push_back(*reinterpret_cast(&value)); - } - vectorized::DataTypePtr date_v2_type( - std::make_shared()); - vectorized::ColumnWithTypeAndName test_date_v2(column_vector_date_v2->get_ptr(), - date_v2_type, col_name); - block.insert(test_date_v2); - } - break; - case TYPE_DATE: // int64 - tslot.__set_slotType(type_desc.to_thrift()); - { - auto column_vector_date = vectorized::ColumnVector::create(); - auto& date_data = column_vector_date->get_data(); - for (int i = 0; i < row_num; ++i) { - VecDateTimeValue value; - value.from_date_int64(20210501); - date_data.push_back(*reinterpret_cast(&value)); - } - vectorized::DataTypePtr date_type(std::make_shared()); - vectorized::ColumnWithTypeAndName test_date(column_vector_date->get_ptr(), - date_type, col_name); - block.insert(test_date); - } - break; - case TYPE_DATETIME: // int64 - tslot.__set_slotType(type_desc.to_thrift()); - { - auto column_vector_datetime = vectorized::ColumnVector::create(); - auto& datetime_data = column_vector_datetime->get_data(); - for (int i = 0; i < row_num; ++i) { - VecDateTimeValue value; - value.from_date_int64(20210501080910); - datetime_data.push_back(*reinterpret_cast(&value)); - } - vectorized::DataTypePtr datetime_type( - std::make_shared()); - vectorized::ColumnWithTypeAndName test_datetime(column_vector_datetime->get_ptr(), - datetime_type, col_name); - block.insert(test_datetime); - } - break; - case TYPE_DATETIMEV2: // uint64 - tslot.__set_slotType(type_desc.to_thrift()); - { - // 2022-01-01 11:11:11.111 - auto column_vector_datetimev2 = - vectorized::ColumnVector::create(); - // auto& datetimev2_data = column_vector_datetimev2->get_data(); - DateV2Value value; - string date_literal = "2022-01-01 11:11:11.111"; - value.from_date_str(date_literal.c_str(), date_literal.size()); - char to[64] = {}; - std::cout << "value: " << value.to_string(to) << std::endl; - for (int i = 0; i < row_num; ++i) { - column_vector_datetimev2->insert(value.to_date_int_val()); - } - vectorized::DataTypePtr datetimev2_type( - std::make_shared()); - vectorized::ColumnWithTypeAndName test_datetimev2( - column_vector_datetimev2->get_ptr(), datetimev2_type, col_name); - block.insert(test_datetimev2); - } - break; - case TYPE_ARRAY: // array - type_desc.add_sub_type(TYPE_STRING, true); - tslot.__set_slotType(type_desc.to_thrift()); - { - DataTypePtr s = - std::make_shared(std::make_shared()); - DataTypePtr au = std::make_shared(s); - Array a1, a2; - a1.push_back(String("sss")); - a1.push_back(Null()); - a1.push_back(String("clever amory")); - a2.push_back(String("hello amory")); - a2.push_back(Null()); - a2.push_back(String("cute amory")); - a2.push_back(String("sf")); - MutableColumnPtr array_column = au->create_column(); - array_column->reserve(2); - array_column->insert(a1); - array_column->insert(a2); - vectorized::ColumnWithTypeAndName type_and_name(array_column->get_ptr(), au, - col_name); - block.insert(type_and_name); - } - break; - case TYPE_MAP: - type_desc.add_sub_type(TYPE_STRING, true); - type_desc.add_sub_type(TYPE_STRING, true); - tslot.__set_slotType(type_desc.to_thrift()); - { - DataTypePtr s = - std::make_shared(std::make_shared()); - ; - DataTypePtr d = - std::make_shared(std::make_shared()); - DataTypePtr m = std::make_shared(s, d); - Array k1, k2, v1, v2; - k1.push_back("null"); - k1.push_back("doris"); - k1.push_back("clever amory"); - v1.push_back("ss"); - v1.push_back(Null()); - v1.push_back("NULL"); - k2.push_back("hello amory"); - k2.push_back("NULL"); - k2.push_back("cute amory"); - k2.push_back("doris"); - v2.push_back("s"); - v2.push_back("0"); - v2.push_back("sf"); - v2.push_back(Null()); - Map m1, m2; - m1.push_back(k1); - m1.push_back(v1); - m2.push_back(k2); - m2.push_back(v2); - MutableColumnPtr map_column = m->create_column(); - map_column->reserve(2); - map_column->insert(m1); - map_column->insert(m2); - vectorized::ColumnWithTypeAndName type_and_name(map_column->get_ptr(), m, col_name); - block.insert(type_and_name); - } - break; - case TYPE_STRUCT: - type_desc.add_sub_type(TYPE_STRING, "name", true); - type_desc.add_sub_type(TYPE_LARGEINT, "age", true); - type_desc.add_sub_type(TYPE_BOOLEAN, "is", true); - tslot.__set_slotType(type_desc.to_thrift()); - { - DataTypePtr s = - std::make_shared(std::make_shared()); - DataTypePtr d = - std::make_shared(std::make_shared()); - DataTypePtr m = - std::make_shared(std::make_shared()); - DataTypePtr st = - std::make_shared(std::vector {s, d, m}); - Tuple t1, t2; - t1.push_back(String("amory cute")); - t1.push_back(__int128_t(37)); - t1.push_back(true); - t2.push_back("null"); - t2.push_back(__int128_t(26)); - t2.push_back(false); - MutableColumnPtr struct_column = st->create_column(); - struct_column->reserve(2); - struct_column->insert(t1); - struct_column->insert(t2); - vectorized::ColumnWithTypeAndName type_and_name(struct_column->get_ptr(), st, - col_name); - block.insert(type_and_name); - } - break; - case TYPE_IPV4: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto vec = vectorized::ColumnIPv4::create(); - auto& data = vec->get_data(); - for (int i = 0; i < row_num; ++i) { - data.push_back(i); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, - col_name); - block.insert(std::move(type_and_name)); - } - break; - case TYPE_IPV6: - tslot.__set_slotType(type_desc.to_thrift()); - { - auto vec = vectorized::ColumnIPv6::create(); - auto& data = vec->get_data(); - for (int i = 0; i < row_num; ++i) { - data.push_back(i); - } - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(vec->get_ptr(), data_type, - col_name); - block.insert(std::move(type_and_name)); - } - break; - default: - break; - } - - tslot.__set_col_unique_id(std::get<2>(t)); - SlotDescriptor* slot = new SlotDescriptor(tslot); - tuple_desc.add_slot(slot); - } - - RowDescriptor row_desc(&tuple_desc, true); - // arrow schema - std::shared_ptr _arrow_schema; - EXPECT_EQ(convert_to_arrow_schema(row_desc, &_arrow_schema, "UTC"), Status::OK()); - - // serialize - std::shared_ptr result; - std::cout << "block data: " << block.dump_data(0, row_num) << std::endl; - std::cout << "_arrow_schema: " << _arrow_schema->ToString(true) << std::endl; - - cctz::time_zone timezone_obj; - TimezoneUtils::find_cctz_time_zone(TimezoneUtils::default_time_zone, timezone_obj); - static_cast(convert_to_arrow_batch(block, _arrow_schema, arrow::default_memory_pool(), - &result, timezone_obj)); - Block new_block = block.clone_empty(); - EXPECT_TRUE(result != nullptr); - std::cout << "result: " << result->ToString() << std::endl; - // deserialize - for (auto t : cols) { - std::string real_column_name = std::get<0>(t); - auto* array = result->GetColumnByName(real_column_name).get(); - auto& column_with_type_and_name = new_block.get_by_name(real_column_name); - if (std::get<3>(t) == PrimitiveType::TYPE_DATE || - std::get<3>(t) == PrimitiveType::TYPE_DATETIME) { - { - auto strcol = vectorized::ColumnString::create(); - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(strcol->get_ptr(), data_type, - real_column_name); - static_cast(arrow_column_to_doris_column( - array, 0, type_and_name.column, type_and_name.type, block.rows(), "UTC")); - { - auto& col = column_with_type_and_name.column.get()->assume_mutable_ref(); - auto& date_data = static_cast&>(col).get_data(); - for (int i = 0; i < strcol->size(); ++i) { - StringRef str = strcol->get_data_at(i); - VecDateTimeValue value; - value.from_date_str(str.data, str.size); - date_data.push_back(*reinterpret_cast(&value)); - } - } - } - continue; - } else if (std::get<3>(t) == PrimitiveType::TYPE_DATEV2) { - auto strcol = vectorized::ColumnString::create(); - vectorized::DataTypePtr data_type(std::make_shared()); - vectorized::ColumnWithTypeAndName type_and_name(strcol->get_ptr(), data_type, - real_column_name); - static_cast(arrow_column_to_doris_column( - array, 0, type_and_name.column, type_and_name.type, block.rows(), "UTC")); - { - auto& col = column_with_type_and_name.column.get()->assume_mutable_ref(); - auto& date_data = static_cast&>(col).get_data(); - for (int i = 0; i < strcol->size(); ++i) { - StringRef str = strcol->get_data_at(i); - DateV2Value value; - value.from_date_str(str.data, str.size); - date_data.push_back(*reinterpret_cast(&value)); - } - } - continue; - } else if (std::get<3>(t) == PrimitiveType::TYPE_DATETIMEV2) { - // now we only support read doris datetimev2 to arrow - block.erase(real_column_name); - new_block.erase(real_column_name); - continue; - } - static_cast(arrow_column_to_doris_column(array, 0, column_with_type_and_name.column, - column_with_type_and_name.type, block.rows(), - "UTC")); - } - - std::cout << block.dump_data() << std::endl; - std::cout << new_block.dump_data() << std::endl; - EXPECT_EQ(block.dump_data(), new_block.dump_data()); -} - -TEST(DataTypeSerDeArrowTest, DataTypeScalaSerDeTest) { - serialize_and_deserialize_arrow_test(); -} - -TEST(DataTypeSerDeArrowTest, DataTypeCollectionSerDeTest) { - serialize_and_deserialize_arrow_test(); -} - -TEST(DataTypeSerDeArrowTest, DataTypeMapNullKeySerDeTest) { - TupleDescriptor tuple_desc(PTupleDescriptor(), true); - TSlotDescriptor tslot; - std::string col_name = "map_null_key"; - tslot.__set_colName(col_name); - TypeDescriptor type_desc(TYPE_MAP); - type_desc.add_sub_type(TYPE_STRING, true); - type_desc.add_sub_type(TYPE_INT, true); - tslot.__set_slotType(type_desc.to_thrift()); - vectorized::Block block; - { - DataTypePtr s = std::make_shared(std::make_shared()); - ; - DataTypePtr d = std::make_shared(std::make_shared()); - DataTypePtr m = std::make_shared(s, d); - Array k1, k2, v1, v2, k3, v3; - k1.push_back("doris"); - k1.push_back("clever amory"); - v1.push_back(Null()); - v1.push_back(30); - k2.push_back("hello amory"); - k2.push_back("NULL"); - k2.push_back("cute amory"); - k2.push_back("doris"); - v2.push_back(26); - v2.push_back(Null()); - v2.push_back(6); - v2.push_back(7); - k3.push_back("test"); - v3.push_back(11); - Map m1, m2, m3; - m1.push_back(k1); - m1.push_back(v1); - m2.push_back(k2); - m2.push_back(v2); - m3.push_back(k3); - m3.push_back(v3); - MutableColumnPtr map_column = m->create_column(); - map_column->reserve(3); - map_column->insert(m1); - map_column->insert(m2); - map_column->insert(m3); - vectorized::ColumnWithTypeAndName type_and_name(map_column->get_ptr(), m, col_name); - block.insert(type_and_name); - } - - tslot.__set_col_unique_id(1); - SlotDescriptor* slot = new SlotDescriptor(tslot); - tuple_desc.add_slot(slot); - RowDescriptor row_desc(&tuple_desc, true); - // arrow schema - std::shared_ptr _arrow_schema; - EXPECT_EQ(convert_to_arrow_schema(row_desc, &_arrow_schema, "UTC"), Status::OK()); - - // serialize - std::shared_ptr result; - std::cout << "block structure: " << block.dump_structure() << std::endl; - std::cout << "_arrow_schema: " << _arrow_schema->ToString(true) << std::endl; - - cctz::time_zone timezone_obj; - TimezoneUtils::find_cctz_time_zone(TimezoneUtils::default_time_zone, timezone_obj); - static_cast(convert_to_arrow_batch(block, _arrow_schema, arrow::default_memory_pool(), - &result, timezone_obj)); - Block new_block = block.clone_empty(); - EXPECT_TRUE(result != nullptr); - std::cout << "result: " << result->ToString() << std::endl; - // deserialize - auto* array = result->GetColumnByName(col_name).get(); - auto& column_with_type_and_name = new_block.get_by_name(col_name); - static_cast(arrow_column_to_doris_column(array, 0, column_with_type_and_name.column, - column_with_type_and_name.type, block.rows(), - "UTC")); - std::cout << block.dump_data() << std::endl; - std::cout << new_block.dump_data() << std::endl; - // new block row_index 0, 2 which row has key null will be filter - EXPECT_EQ(new_block.dump_one_line(0, 1), "{\"doris\":null, \"clever amory\":30}"); - EXPECT_EQ(new_block.dump_one_line(2, 1), "{\"test\":11}"); - EXPECT_EQ(block.dump_data(1, 1), new_block.dump_data(1, 1)); -} - -} // namespace doris::vectorized From 2b4aca93f08a79d95bcaa3ca596ffe8e21d5ca6c Mon Sep 17 00:00:00 2001 From: Jibing-Li <64681310+Jibing-Li@users.noreply.github.com> Date: Thu, 31 Oct 2024 21:55:27 +0800 Subject: [PATCH 23/82] [improvement](statistics)Improve statisitcs insert into audit log. (#42983) The audit log for statistics insert sql was like: Stmt=OriginStatement{originStmt='INSERT INTO internal.__internal_schema.column_statistics VALUES...} Remove the "OriginStatement{originStmt" part, and change it to: Stmt=INSERT INTO internal.__internal_schema.column_statistics VALUES.. --- .../src/main/java/org/apache/doris/statistics/AnalysisJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java index acfd457d8a21082..0ab1fe004f69145 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/AnalysisJob.java @@ -154,7 +154,7 @@ protected void executeWithExceptionOnFail(StmtExecutor stmtExecutor) throws Exce + queryState.getErrorMessage()); } } finally { - AuditLogHelper.logAuditLog(stmtExecutor.getContext(), stmtExecutor.getOriginStmt().toString(), + AuditLogHelper.logAuditLog(stmtExecutor.getContext(), stmtExecutor.getOriginStmt().originStmt, stmtExecutor.getParsedStmt(), stmtExecutor.getQueryStatisticsForAuditLog(), true); } From adbc003567ff6c8b93de364fb607b8a99a025360 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Thu, 31 Oct 2024 22:25:29 +0800 Subject: [PATCH 24/82] Revert "[improvement](jdbc catalog) Add catalog property to enable jdbc connection pool" (#42969) Reverts apache/doris#41992 --- be/src/runtime/descriptors.cpp | 10 +- be/src/runtime/descriptors.h | 2 - be/src/vec/exec/scan/new_jdbc_scanner.cpp | 1 - be/src/vec/exec/vjdbc_connector.cpp | 1 - be/src/vec/exec/vjdbc_connector.h | 1 - be/src/vec/sink/writer/vjdbc_table_writer.cpp | 1 - .../apache/doris/jdbc/BaseJdbcExecutor.java | 110 +++++++---------- .../doris/jdbc/JdbcDataSourceConfig.java | 10 -- .../apache/doris/catalog/JdbcResource.java | 5 +- .../org/apache/doris/catalog/JdbcTable.java | 8 -- .../datasource/jdbc/JdbcExternalCatalog.java | 32 +---- .../datasource/jdbc/client/JdbcClient.java | 64 +--------- .../jdbc/client/JdbcClientConfig.java | 13 -- gensrc/thrift/Descriptors.thrift | 1 - gensrc/thrift/Types.thrift | 1 - ...test_clickhouse_jdbc_catalog_pool_test.out | Bin 8815 -> 0 bytes .../test_mysql_jdbc_catalog_pool_test.out | 85 -------------- .../test_oracle_jdbc_catalog_pool_test.out | 109 ----------------- .../test_pg_jdbc_catalog_pool_test.out | 69 ----------- .../test_sqlserver_jdbc_catalog_pool_test.out | 81 ------------- ...t_clickhouse_jdbc_catalog_pool_test.groovy | 71 ----------- .../test_mysql_jdbc_catalog_pool_test.groovy | 73 ------------ .../test_oracle_jdbc_catalog_pool_test.groovy | 111 ------------------ .../test_pg_jdbc_catalog_pool_test.groovy | 71 ----------- ...st_sqlserver_jdbc_catalog_pool_test.groovy | 70 ----------- .../manager/test_manager_interface_3.groovy | 8 +- 26 files changed, 61 insertions(+), 947 deletions(-) delete mode 100644 regression-test/data/external_table_p0/jdbc/connection_pool_test/test_clickhouse_jdbc_catalog_pool_test.out delete mode 100644 regression-test/data/external_table_p0/jdbc/connection_pool_test/test_mysql_jdbc_catalog_pool_test.out delete mode 100644 regression-test/data/external_table_p0/jdbc/connection_pool_test/test_oracle_jdbc_catalog_pool_test.out delete mode 100644 regression-test/data/external_table_p0/jdbc/connection_pool_test/test_pg_jdbc_catalog_pool_test.out delete mode 100644 regression-test/data/external_table_p0/jdbc/connection_pool_test/test_sqlserver_jdbc_catalog_pool_test.out delete mode 100644 regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_clickhouse_jdbc_catalog_pool_test.groovy delete mode 100644 regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_mysql_jdbc_catalog_pool_test.groovy delete mode 100644 regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_oracle_jdbc_catalog_pool_test.groovy delete mode 100644 regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_pg_jdbc_catalog_pool_test.groovy delete mode 100644 regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_sqlserver_jdbc_catalog_pool_test.groovy diff --git a/be/src/runtime/descriptors.cpp b/be/src/runtime/descriptors.cpp index cc6f9050ac39153..bea11feb916f108 100644 --- a/be/src/runtime/descriptors.cpp +++ b/be/src/runtime/descriptors.cpp @@ -286,8 +286,7 @@ JdbcTableDescriptor::JdbcTableDescriptor(const TTableDescriptor& tdesc) _connection_pool_max_size(tdesc.jdbcTable.connection_pool_max_size), _connection_pool_max_wait_time(tdesc.jdbcTable.connection_pool_max_wait_time), _connection_pool_max_life_time(tdesc.jdbcTable.connection_pool_max_life_time), - _connection_pool_keep_alive(tdesc.jdbcTable.connection_pool_keep_alive), - _enable_connection_pool(tdesc.jdbcTable.enable_connection_pool) {} + _connection_pool_keep_alive(tdesc.jdbcTable.connection_pool_keep_alive) {} std::string JdbcTableDescriptor::debug_string() const { fmt::memory_buffer buf; @@ -295,14 +294,13 @@ std::string JdbcTableDescriptor::debug_string() const { buf, "JDBCTable({} ,_jdbc_catalog_id = {}, _jdbc_resource_name={} ,_jdbc_driver_url={} " ",_jdbc_driver_class={} ,_jdbc_driver_checksum={} ,_jdbc_url={} " - ",_jdbc_table_name={} ,_jdbc_user={} ,_jdbc_passwd={} " - ",_enable_connection_pool={},_connection_pool_min_size={} " + ",_jdbc_table_name={} ,_jdbc_user={} ,_jdbc_passwd={} ,_connection_pool_min_size={} " ",_connection_pool_max_size={} ,_connection_pool_max_wait_time={} " ",_connection_pool_max_life_time={} ,_connection_pool_keep_alive={})", TableDescriptor::debug_string(), _jdbc_catalog_id, _jdbc_resource_name, _jdbc_driver_url, _jdbc_driver_class, _jdbc_driver_checksum, _jdbc_url, - _jdbc_table_name, _jdbc_user, _jdbc_passwd, _enable_connection_pool, - _connection_pool_min_size, _connection_pool_max_size, _connection_pool_max_wait_time, + _jdbc_table_name, _jdbc_user, _jdbc_passwd, _connection_pool_min_size, + _connection_pool_max_size, _connection_pool_max_wait_time, _connection_pool_max_life_time, _connection_pool_keep_alive); return fmt::to_string(buf); } diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index b5797b0d016d751..b807c5675430383 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -319,7 +319,6 @@ class JdbcTableDescriptor : public TableDescriptor { int32_t connection_pool_max_wait_time() const { return _connection_pool_max_wait_time; } int32_t connection_pool_max_life_time() const { return _connection_pool_max_life_time; } bool connection_pool_keep_alive() const { return _connection_pool_keep_alive; } - bool enable_connection_pool() const { return _enable_connection_pool; } private: int64_t _jdbc_catalog_id; @@ -336,7 +335,6 @@ class JdbcTableDescriptor : public TableDescriptor { int32_t _connection_pool_max_wait_time; int32_t _connection_pool_max_life_time; bool _connection_pool_keep_alive; - bool _enable_connection_pool; }; class TupleDescriptor { diff --git a/be/src/vec/exec/scan/new_jdbc_scanner.cpp b/be/src/vec/exec/scan/new_jdbc_scanner.cpp index a23e83e2426c078..7eaa9ab3eab7881 100644 --- a/be/src/vec/exec/scan/new_jdbc_scanner.cpp +++ b/be/src/vec/exec/scan/new_jdbc_scanner.cpp @@ -89,7 +89,6 @@ Status NewJdbcScanner::prepare(RuntimeState* state, const VExprContextSPtrs& con _jdbc_param.connection_pool_max_life_time = jdbc_table->connection_pool_max_life_time(); _jdbc_param.connection_pool_max_wait_time = jdbc_table->connection_pool_max_wait_time(); _jdbc_param.connection_pool_keep_alive = jdbc_table->connection_pool_keep_alive(); - _jdbc_param.enable_connection_pool = jdbc_table->enable_connection_pool(); _local_state->scanner_profile()->add_info_string("JdbcDriverClass", _jdbc_param.driver_class); _local_state->scanner_profile()->add_info_string("JdbcDriverUrl", _jdbc_param.driver_path); diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp index 14263cf4c08b5b9..0fa33bfaad917d8 100644 --- a/be/src/vec/exec/vjdbc_connector.cpp +++ b/be/src/vec/exec/vjdbc_connector.cpp @@ -152,7 +152,6 @@ Status JdbcConnector::open(RuntimeState* state, bool read) { } ctor_params.__set_op(read ? TJdbcOperation::READ : TJdbcOperation::WRITE); ctor_params.__set_table_type(_conn_param.table_type); - ctor_params.__set_enable_connection_pool(_conn_param.enable_connection_pool); ctor_params.__set_connection_pool_min_size(_conn_param.connection_pool_min_size); ctor_params.__set_connection_pool_max_size(_conn_param.connection_pool_max_size); ctor_params.__set_connection_pool_max_wait_time(_conn_param.connection_pool_max_wait_time); diff --git a/be/src/vec/exec/vjdbc_connector.h b/be/src/vec/exec/vjdbc_connector.h index 066a95de554444b..954b0abfa78f0ce 100644 --- a/be/src/vec/exec/vjdbc_connector.h +++ b/be/src/vec/exec/vjdbc_connector.h @@ -61,7 +61,6 @@ struct JdbcConnectorParam { int32_t connection_pool_max_wait_time = -1; int32_t connection_pool_max_life_time = -1; bool connection_pool_keep_alive = false; - bool enable_connection_pool; const TupleDescriptor* tuple_desc = nullptr; }; diff --git a/be/src/vec/sink/writer/vjdbc_table_writer.cpp b/be/src/vec/sink/writer/vjdbc_table_writer.cpp index d54768e58fe3c6e..8c24f4746adf83d 100644 --- a/be/src/vec/sink/writer/vjdbc_table_writer.cpp +++ b/be/src/vec/sink/writer/vjdbc_table_writer.cpp @@ -52,7 +52,6 @@ JdbcConnectorParam VJdbcTableWriter::create_connect_param(const doris::TDataSink jdbc_param.connection_pool_max_wait_time = t_jdbc_sink.jdbc_table.connection_pool_max_wait_time; jdbc_param.connection_pool_max_life_time = t_jdbc_sink.jdbc_table.connection_pool_max_life_time; jdbc_param.connection_pool_keep_alive = t_jdbc_sink.jdbc_table.connection_pool_keep_alive; - jdbc_param.enable_connection_pool = t_jdbc_sink.jdbc_table.enable_connection_pool; return jdbc_param; } diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java index 61382e6c2532f3d..03e5ca1fa7c5d4f 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java +++ b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/BaseJdbcExecutor.java @@ -41,7 +41,6 @@ import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.Date; -import java.sql.Driver; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.ResultSetMetaData; @@ -52,7 +51,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.function.Function; public abstract class BaseJdbcExecutor implements JdbcExecutor { @@ -94,8 +92,7 @@ public BaseJdbcExecutor(byte[] thriftParams) throws Exception { .setConnectionPoolMaxSize(request.connection_pool_max_size) .setConnectionPoolMaxWaitTime(request.connection_pool_max_wait_time) .setConnectionPoolMaxLifeTime(request.connection_pool_max_life_time) - .setConnectionPoolKeepAlive(request.connection_pool_keep_alive) - .setEnableConnectionPool(request.enable_connection_pool); + .setConnectionPoolKeepAlive(request.connection_pool_keep_alive); JdbcDataSource.getDataSource().setCleanupInterval(request.connection_pool_cache_clear_time); System.setProperty("com.zaxxer.hikari.useWeakReferences", "true"); init(config, request.statement); @@ -120,12 +117,10 @@ public void close() throws Exception { } } finally { closeResources(resultSet, stmt, conn); - if (config.isEnableConnectionPool()) { - if (config.getConnectionPoolMinSize() == 0 && hikariDataSource != null) { - hikariDataSource.close(); - JdbcDataSource.getDataSource().getSourcesMap().remove(config.createCacheKey()); - hikariDataSource = null; - } + if (config.getConnectionPoolMinSize() == 0 && hikariDataSource != null) { + hikariDataSource.close(); + JdbcDataSource.getDataSource().getSourcesMap().remove(config.createCacheKey()); + hikariDataSource = null; } } } @@ -147,12 +142,10 @@ protected void abortReadConnection(Connection connection, ResultSet resultSet) } public void cleanDataSource() { - if (config.isEnableConnectionPool()) { - if (hikariDataSource != null) { - hikariDataSource.close(); - JdbcDataSource.getDataSource().getSourcesMap().remove(config.createCacheKey()); - hikariDataSource = null; - } + if (hikariDataSource != null) { + hikariDataSource.close(); + JdbcDataSource.getDataSource().getSourcesMap().remove(config.createCacheKey()); + hikariDataSource = null; } } @@ -294,64 +287,51 @@ public boolean hasNext() throws JdbcExecutorException { private void init(JdbcDataSourceConfig config, String sql) throws JdbcExecutorException { ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader(); + String hikariDataSourceKey = config.createCacheKey(); try { ClassLoader parent = getClass().getClassLoader(); ClassLoader classLoader = UdfUtils.getClassLoader(config.getJdbcDriverUrl(), parent); Thread.currentThread().setContextClassLoader(classLoader); - if (config.isEnableConnectionPool()) { - String hikariDataSourceKey = config.createCacheKey(); - hikariDataSource = JdbcDataSource.getDataSource().getSource(hikariDataSourceKey); - if (hikariDataSource == null) { - synchronized (hikariDataSourceLock) { - hikariDataSource = JdbcDataSource.getDataSource().getSource(hikariDataSourceKey); - if (hikariDataSource == null) { - long start = System.currentTimeMillis(); - HikariDataSource ds = new HikariDataSource(); - ds.setDriverClassName(config.getJdbcDriverClass()); - ds.setJdbcUrl(SecurityChecker.getInstance().getSafeJdbcUrl(config.getJdbcUrl())); - ds.setUsername(config.getJdbcUser()); - ds.setPassword(config.getJdbcPassword()); - ds.setMinimumIdle(config.getConnectionPoolMinSize()); // default 1 - ds.setMaximumPoolSize(config.getConnectionPoolMaxSize()); // default 10 - ds.setConnectionTimeout(config.getConnectionPoolMaxWaitTime()); // default 5000 - ds.setMaxLifetime(config.getConnectionPoolMaxLifeTime()); // default 30 min - ds.setIdleTimeout(config.getConnectionPoolMaxLifeTime() / 2L); // default 15 min - setValidationQuery(ds); - if (config.isConnectionPoolKeepAlive()) { - ds.setKeepaliveTime(config.getConnectionPoolMaxLifeTime() / 5L); // default 6 min - } - hikariDataSource = ds; - JdbcDataSource.getDataSource().putSource(hikariDataSourceKey, hikariDataSource); - LOG.info("JdbcClient set" - + " ConnectionPoolMinSize = " + config.getConnectionPoolMinSize() - + ", ConnectionPoolMaxSize = " + config.getConnectionPoolMaxSize() - + ", ConnectionPoolMaxWaitTime = " + config.getConnectionPoolMaxWaitTime() - + ", ConnectionPoolMaxLifeTime = " + config.getConnectionPoolMaxLifeTime() - + ", ConnectionPoolKeepAlive = " + config.isConnectionPoolKeepAlive()); - LOG.info("init datasource [" + (config.getJdbcUrl() + config.getJdbcUser()) + "] cost: " + ( - System.currentTimeMillis() - start) + " ms"); + hikariDataSource = JdbcDataSource.getDataSource().getSource(hikariDataSourceKey); + if (hikariDataSource == null) { + synchronized (hikariDataSourceLock) { + hikariDataSource = JdbcDataSource.getDataSource().getSource(hikariDataSourceKey); + if (hikariDataSource == null) { + long start = System.currentTimeMillis(); + HikariDataSource ds = new HikariDataSource(); + ds.setDriverClassName(config.getJdbcDriverClass()); + ds.setJdbcUrl(SecurityChecker.getInstance().getSafeJdbcUrl(config.getJdbcUrl())); + ds.setUsername(config.getJdbcUser()); + ds.setPassword(config.getJdbcPassword()); + ds.setMinimumIdle(config.getConnectionPoolMinSize()); // default 1 + ds.setMaximumPoolSize(config.getConnectionPoolMaxSize()); // default 10 + ds.setConnectionTimeout(config.getConnectionPoolMaxWaitTime()); // default 5000 + ds.setMaxLifetime(config.getConnectionPoolMaxLifeTime()); // default 30 min + ds.setIdleTimeout(config.getConnectionPoolMaxLifeTime() / 2L); // default 15 min + setValidationQuery(ds); + if (config.isConnectionPoolKeepAlive()) { + ds.setKeepaliveTime(config.getConnectionPoolMaxLifeTime() / 5L); // default 6 min } + hikariDataSource = ds; + JdbcDataSource.getDataSource().putSource(hikariDataSourceKey, hikariDataSource); + LOG.info("JdbcClient set" + + " ConnectionPoolMinSize = " + config.getConnectionPoolMinSize() + + ", ConnectionPoolMaxSize = " + config.getConnectionPoolMaxSize() + + ", ConnectionPoolMaxWaitTime = " + config.getConnectionPoolMaxWaitTime() + + ", ConnectionPoolMaxLifeTime = " + config.getConnectionPoolMaxLifeTime() + + ", ConnectionPoolKeepAlive = " + config.isConnectionPoolKeepAlive()); + LOG.info("init datasource [" + (config.getJdbcUrl() + config.getJdbcUser()) + "] cost: " + ( + System.currentTimeMillis() - start) + " ms"); } } - conn = hikariDataSource.getConnection(); - } else { - Class driverClass = Class.forName(config.getJdbcDriverClass(), true, classLoader); - Driver driverInstance = (Driver) driverClass.getDeclaredConstructor().newInstance(); - - Properties info = new Properties(); - info.put("user", config.getJdbcUser()); - info.put("password", config.getJdbcPassword()); - - conn = driverInstance.connect(SecurityChecker.getInstance().getSafeJdbcUrl(config.getJdbcUrl()), info); - if (conn == null) { - throw new SQLException("Failed to establish a connection. The JDBC driver returned null. " - + "Please check if the JDBC URL is correct: " - + config.getJdbcUrl() - + ". Ensure that the URL format and parameters are valid for the driver: " - + driverInstance.getClass().getName()); - } } + long start = System.currentTimeMillis(); + conn = hikariDataSource.getConnection(); + LOG.info("get connection [" + (config.getJdbcUrl() + config.getJdbcUser()) + "] cost: " + ( + System.currentTimeMillis() - start) + + " ms"); + initializeStatement(conn, config, sql); } catch (MalformedURLException e) { diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/JdbcDataSourceConfig.java b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/JdbcDataSourceConfig.java index 30e94ddd37f49dd..a99377add2532db 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/JdbcDataSourceConfig.java +++ b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/JdbcDataSourceConfig.java @@ -35,7 +35,6 @@ public class JdbcDataSourceConfig { private int connectionPoolMaxWaitTime = 5000; private int connectionPoolMaxLifeTime = 1800000; private boolean connectionPoolKeepAlive = false; - private boolean enableConnectionPool = false; public String createCacheKey() { return catalogId + jdbcUrl + jdbcUser + jdbcPassword + jdbcDriverUrl + jdbcDriverClass @@ -168,13 +167,4 @@ public JdbcDataSourceConfig setConnectionPoolKeepAlive(boolean connectionPoolKee this.connectionPoolKeepAlive = connectionPoolKeepAlive; return this; } - - public boolean isEnableConnectionPool() { - return enableConnectionPool; - } - - public JdbcDataSourceConfig setEnableConnectionPool(boolean enableConnectionPool) { - this.enableConnectionPool = enableConnectionPool; - return this; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java index b7db351f49a4d0a..28d58b35297ac3e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java @@ -108,7 +108,6 @@ public class JdbcResource extends Resource { public static final String CHECK_SUM = "checksum"; public static final String CREATE_TIME = "create_time"; public static final String TEST_CONNECTION = "test_connection"; - public static final String ENABLE_CONNECTION_POOL = "enable_connection_pool"; private static final ImmutableList ALL_PROPERTIES = new ImmutableList.Builder().add( JDBC_URL, @@ -129,8 +128,7 @@ public class JdbcResource extends Resource { CONNECTION_POOL_MAX_WAIT_TIME, CONNECTION_POOL_KEEP_ALIVE, TEST_CONNECTION, - ExternalCatalog.USE_META_CACHE, - ENABLE_CONNECTION_POOL + ExternalCatalog.USE_META_CACHE ).build(); // The default value of optional properties @@ -151,7 +149,6 @@ public class JdbcResource extends Resource { OPTIONAL_PROPERTIES_DEFAULT_VALUE.put(TEST_CONNECTION, "true"); OPTIONAL_PROPERTIES_DEFAULT_VALUE.put(ExternalCatalog.USE_META_CACHE, String.valueOf(ExternalCatalog.DEFAULT_USE_META_CACHE)); - OPTIONAL_PROPERTIES_DEFAULT_VALUE.put(ENABLE_CONNECTION_POOL, "false"); } // timeout for both connection and read. 10 seconds is long enough. diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcTable.java index 7c3678c8ed8c6cf..6dce40a2684fbca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcTable.java @@ -97,7 +97,6 @@ public class JdbcTable extends Table { @SerializedName("cid") private long catalogId = -1; - private boolean enableConnectionPool; private int connectionPoolMinSize; private int connectionPoolMaxSize; private int connectionPoolMaxWaitTime; @@ -191,11 +190,6 @@ public long getCatalogId() { return catalogId; } - public boolean isEnableConnectionPool() { - return Boolean.parseBoolean(getFromJdbcResourceOrDefault(JdbcResource.ENABLE_CONNECTION_POOL, - String.valueOf(enableConnectionPool))); - } - public int getConnectionPoolMinSize() { return Integer.parseInt(getFromJdbcResourceOrDefault(JdbcResource.CONNECTION_POOL_MIN_SIZE, String.valueOf(connectionPoolMinSize))); @@ -244,7 +238,6 @@ public TTableDescriptor toThrift() { tJdbcTable.setJdbcDriverUrl(getDriverUrl()); tJdbcTable.setJdbcResourceName(resourceName); tJdbcTable.setJdbcDriverChecksum(checkSum); - tJdbcTable.setEnableConnectionPool(isEnableConnectionPool()); tJdbcTable.setConnectionPoolMinSize(getConnectionPoolMinSize()); tJdbcTable.setConnectionPoolMaxSize(getConnectionPoolMaxSize()); tJdbcTable.setConnectionPoolMaxWaitTime(getConnectionPoolMaxWaitTime()); @@ -401,7 +394,6 @@ private void validate(Map properties) throws DdlException { driverClass = jdbcResource.getProperty(DRIVER_CLASS); driverUrl = jdbcResource.getProperty(DRIVER_URL); checkSum = jdbcResource.getProperty(CHECK_SUM); - enableConnectionPool = Boolean.parseBoolean(jdbcResource.getProperty(JdbcResource.ENABLE_CONNECTION_POOL)); connectionPoolMinSize = Integer.parseInt(jdbcResource.getProperty(JdbcResource.CONNECTION_POOL_MIN_SIZE)); connectionPoolMaxSize = Integer.parseInt(jdbcResource.getProperty(JdbcResource.CONNECTION_POOL_MAX_SIZE)); connectionPoolMaxWaitTime = Integer.parseInt( diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java index e7e7634cff0207a..fb26265d19fe93c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/JdbcExternalCatalog.java @@ -93,8 +93,6 @@ public void checkProperties() throws DdlException { JdbcResource.checkBooleanProperty(JdbcResource.CONNECTION_POOL_KEEP_ALIVE, String.valueOf(isConnectionPoolKeepAlive())); JdbcResource.checkBooleanProperty(JdbcResource.TEST_CONNECTION, String.valueOf(isTestConnection())); - JdbcResource.checkBooleanProperty(JdbcResource.ENABLE_CONNECTION_POOL, - String.valueOf(isEnableConnectionPool())); JdbcResource.checkDatabaseListProperties(getOnlySpecifiedDatabase(), getIncludeDatabaseMap(), getExcludeDatabaseMap()); JdbcResource.checkConnectionPoolProperties(getConnectionPoolMinSize(), getConnectionPoolMaxSize(), @@ -114,27 +112,6 @@ public void setDefaultPropsIfMissing(boolean isReplay) { throw new IllegalArgumentException("Jdbc catalog property lower_case_table_names is not supported," + " please use lower_case_meta_names instead."); } - if (catalogProperty.getOrDefault(JdbcResource.ENABLE_CONNECTION_POOL, "").isEmpty()) { - // If not setting enable_connection_pool in replay logic, - // set default value true to be compatible with older version. - catalogProperty.addProperty(JdbcResource.ENABLE_CONNECTION_POOL, - isReplay ? "true" : String.valueOf(JdbcResource - .getDefaultPropertyValue(JdbcResource.ENABLE_CONNECTION_POOL))); - } - } - - @Override - public void tryModifyCatalogProps(Map props) { - // It is forbidden to modify the enable_connection_pool attribute and driver_url attribute of jdbc catalog - if (props.containsKey(JdbcResource.ENABLE_CONNECTION_POOL)) { - throw new IllegalArgumentException("Can not modify enable_connection_pool property of jdbc catalog," - + "please re-create the catalog"); - } - if (props.containsKey(JdbcResource.DRIVER_URL)) { - throw new IllegalArgumentException("Can not modify driver_url property of jdbc catalog" - + "please re-create the catalog"); - } - super.tryModifyCatalogProps(props); } @Override @@ -245,11 +222,6 @@ public boolean isTestConnection() { .getDefaultPropertyValue(JdbcResource.TEST_CONNECTION))); } - public boolean isEnableConnectionPool() { - return Boolean.parseBoolean(catalogProperty.getOrDefault(JdbcResource.ENABLE_CONNECTION_POOL, JdbcResource - .getDefaultPropertyValue(JdbcResource.ENABLE_CONNECTION_POOL))); - } - @Override protected void initLocalObjectsImpl() { JdbcClientConfig jdbcClientConfig = new JdbcClientConfig() @@ -268,8 +240,7 @@ protected void initLocalObjectsImpl() { .setConnectionPoolMaxSize(getConnectionPoolMaxSize()) .setConnectionPoolMaxLifeTime(getConnectionPoolMaxLifeTime()) .setConnectionPoolMaxWaitTime(getConnectionPoolMaxWaitTime()) - .setConnectionPoolKeepAlive(isConnectionPoolKeepAlive()) - .setEnableConnectionPool(isEnableConnectionPool()); + .setConnectionPoolKeepAlive(isConnectionPoolKeepAlive()); jdbcClient = JdbcClient.createJdbcClient(jdbcClientConfig); } @@ -349,7 +320,6 @@ public void configureJdbcTable(JdbcTable jdbcTable, String tableName) { jdbcTable.setConnectionPoolMaxLifeTime(this.getConnectionPoolMaxLifeTime()); jdbcTable.setConnectionPoolMaxWaitTime(this.getConnectionPoolMaxWaitTime()); jdbcTable.setConnectionPoolKeepAlive(this.isConnectionPoolKeepAlive()); - jdbcTable.setEnableConnectionPool(this.isEnableConnectionPool()); } private void testJdbcConnection() throws DdlException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java index e863a42c122a9be..8c4ada017745592 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java @@ -39,7 +39,6 @@ import java.net.URLClassLoader; import java.sql.Connection; import java.sql.DatabaseMetaData; -import java.sql.Driver; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.ResultSetMetaData; @@ -49,7 +48,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Properties; import java.util.Set; import java.util.function.Consumer; @@ -62,11 +60,7 @@ public abstract class JdbcClient { private String catalogName; protected String dbType; protected String jdbcUser; - protected String jdbcUrl; - protected String jdbcPassword; - protected String jdbcDriverClass; protected ClassLoader classLoader = null; - protected boolean enableConnectionPool; protected HikariDataSource dataSource = null; protected boolean isOnlySpecifiedDatabase; protected boolean isLowerCaseMetaNames; @@ -109,9 +103,6 @@ protected JdbcClient(JdbcClientConfig jdbcClientConfig) { System.setProperty("com.zaxxer.hikari.useWeakReferences", "true"); this.catalogName = jdbcClientConfig.getCatalog(); this.jdbcUser = jdbcClientConfig.getUser(); - this.jdbcPassword = jdbcClientConfig.getPassword(); - this.jdbcUrl = jdbcClientConfig.getJdbcUrl(); - this.jdbcDriverClass = jdbcClientConfig.getDriverClass(); this.isOnlySpecifiedDatabase = Boolean.parseBoolean(jdbcClientConfig.getOnlySpecifiedDatabase()); this.isLowerCaseMetaNames = Boolean.parseBoolean(jdbcClientConfig.getIsLowerCaseMetaNames()); this.metaNamesMapping = jdbcClientConfig.getMetaNamesMapping(); @@ -119,12 +110,10 @@ protected JdbcClient(JdbcClientConfig jdbcClientConfig) { Optional.ofNullable(jdbcClientConfig.getIncludeDatabaseMap()).orElse(Collections.emptyMap()); this.excludeDatabaseMap = Optional.ofNullable(jdbcClientConfig.getExcludeDatabaseMap()).orElse(Collections.emptyMap()); - this.enableConnectionPool = jdbcClientConfig.isEnableConnectionPool(); + String jdbcUrl = jdbcClientConfig.getJdbcUrl(); this.dbType = parseDbType(jdbcUrl); initializeClassLoader(jdbcClientConfig); - if (enableConnectionPool) { - initializeDataSource(jdbcClientConfig); - } + initializeDataSource(jdbcClientConfig); this.jdbcLowerCaseMetaMatching = new JdbcIdentifierMapping(isLowerCaseMetaNames, metaNamesMapping, this); } @@ -179,57 +168,15 @@ public static String parseDbType(String jdbcUrl) { } public void closeClient() { - if (enableConnectionPool && dataSource != null) { - dataSource.close(); - } + dataSource.close(); } public Connection getConnection() throws JdbcClientException { - if (enableConnectionPool) { - return getConnectionWithPool(); - } else { - return getConnectionWithoutPool(); - } - } - - private Connection getConnectionWithoutPool() throws JdbcClientException { - ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader(); - try { - Thread.currentThread().setContextClassLoader(this.classLoader); - - Class driverClass = Class.forName(jdbcDriverClass, true, this.classLoader); - Driver driverInstance = (Driver) driverClass.getDeclaredConstructor().newInstance(); - - Properties info = new Properties(); - info.put("user", jdbcUser); - info.put("password", jdbcPassword); - - Connection connection = driverInstance.connect(SecurityChecker.getInstance().getSafeJdbcUrl(jdbcUrl), info); - - if (connection == null) { - throw new SQLException("Failed to establish a connection. The JDBC driver returned null. " - + "Please check if the JDBC URL is correct: " - + jdbcUrl - + ". Ensure that the URL format and parameters are valid for the driver: " - + driverInstance.getClass().getName()); - } - - return connection; - } catch (Exception e) { - String errorMessage = String.format("Can not connect to jdbc due to error: %s, Catalog name: %s", - e.getMessage(), this.getCatalogName()); - throw new JdbcClientException(errorMessage, e); - } finally { - Thread.currentThread().setContextClassLoader(oldClassLoader); - } - } - - - private Connection getConnectionWithPool() throws JdbcClientException { ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader(); + Connection conn; try { Thread.currentThread().setContextClassLoader(this.classLoader); - return dataSource.getConnection(); + conn = dataSource.getConnection(); } catch (Exception e) { String errorMessage = String.format( "Catalog `%s` can not connect to jdbc due to error: %s", @@ -238,6 +185,7 @@ private Connection getConnectionWithPool() throws JdbcClientException { } finally { Thread.currentThread().setContextClassLoader(oldClassLoader); } + return conn; } public void close(AutoCloseable... closeables) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClientConfig.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClientConfig.java index f3ab9953e050af9..85f3bd8f256d8b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClientConfig.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClientConfig.java @@ -39,7 +39,6 @@ public class JdbcClientConfig implements Cloneable { private int connectionPoolMaxWaitTime; private int connectionPoolMaxLifeTime; private boolean connectionPoolKeepAlive; - private boolean enableConnectionPool; private Map includeDatabaseMap; private Map excludeDatabaseMap; @@ -59,8 +58,6 @@ public JdbcClientConfig() { JdbcResource.getDefaultPropertyValue(JdbcResource.CONNECTION_POOL_MAX_LIFE_TIME)); this.connectionPoolKeepAlive = Boolean.parseBoolean( JdbcResource.getDefaultPropertyValue(JdbcResource.CONNECTION_POOL_KEEP_ALIVE)); - this.enableConnectionPool = Boolean.parseBoolean( - JdbcResource.getDefaultPropertyValue(JdbcResource.ENABLE_CONNECTION_POOL)); this.includeDatabaseMap = Maps.newHashMap(); this.excludeDatabaseMap = Maps.newHashMap(); this.customizedProperties = Maps.newHashMap(); @@ -76,7 +73,6 @@ public JdbcClientConfig clone() { cloned.connectionPoolMaxLifeTime = connectionPoolMaxLifeTime; cloned.connectionPoolMaxWaitTime = connectionPoolMaxWaitTime; cloned.connectionPoolKeepAlive = connectionPoolKeepAlive; - cloned.enableConnectionPool = enableConnectionPool; cloned.includeDatabaseMap = Maps.newHashMap(includeDatabaseMap); cloned.excludeDatabaseMap = Maps.newHashMap(excludeDatabaseMap); cloned.customizedProperties = Maps.newHashMap(customizedProperties); @@ -212,15 +208,6 @@ public JdbcClientConfig setConnectionPoolKeepAlive(boolean connectionPoolKeepAli return this; } - public boolean isEnableConnectionPool() { - return enableConnectionPool; - } - - public JdbcClientConfig setEnableConnectionPool(boolean enableConnectionPool) { - this.enableConnectionPool = enableConnectionPool; - return this; - } - public Map getIncludeDatabaseMap() { return includeDatabaseMap; } diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index 7d74c1773fec88e..dd6ebf2248e083f 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -352,7 +352,6 @@ struct TJdbcTable { 12: optional i32 connection_pool_max_life_time 13: optional bool connection_pool_keep_alive 14: optional i64 catalog_id - 15: optional bool enable_connection_pool } struct TMCTable { diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 7fabe64e26e2929..235c1cb28378e24 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -454,7 +454,6 @@ struct TJdbcExecutorCtorParams { 14: optional i32 connection_pool_cache_clear_time 15: optional bool connection_pool_keep_alive 16: optional i64 catalog_id - 17: optional bool enable_connection_pool } struct TJavaUdfExecutorCtorParams { diff --git a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_clickhouse_jdbc_catalog_pool_test.out b/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_clickhouse_jdbc_catalog_pool_test.out deleted file mode 100644 index 247a342cb7eedfd87ffbc1717625f2bf52dcc109..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8815 zcmeHM-A=vw(%V{}{DmX_8T8l~!je^$Znm2)S0Ftuq^Fq+dx-_f@6ozwJK`=0id3iK3}+G=Xc zvD>RU(BP@jHp=+gj4^r>PA`TsmX*r2hDeZ-fnXpnceel=faKYv;BmrGzbjnES?5jf z8>fj71(&OwFM*3ZCsKjjCw@U2-7tvFWrs?SDGAcvA>2$XH2~mkJj!3`m>GIOnf*p!-FJc)~>=9znbiFU-^a_Qz?Ory;+PU&ycP@eAX`dB%x|8{&rfH0INI RPyO}cJ&hmHPR`3tz5=5nC*S}8 diff --git a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_mysql_jdbc_catalog_pool_test.out b/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_mysql_jdbc_catalog_pool_test.out deleted file mode 100644 index cb9ca3b52312481..000000000000000 --- a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_mysql_jdbc_catalog_pool_test.out +++ /dev/null @@ -1,85 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types_refresh -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types_refresh -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types_refresh -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types_refresh -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types_refresh -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - --- !mysql_all_types -- -\N 302 \N 502 602 4.14159 \N 6.14159 \N -124 -302 2013 -402 -502 -602 \N 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 \N -7.1400 row2 \N 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 \N \N 0x2F \N 0x88656C6C9F Value3 -201 301 401 501 601 3.14159 4.1415926 5.14159 1 -123 -301 2012 -401 -501 -601 2012-10-30 2012-10-25T12:05:36.345700 2012-10-25T08:08:08 -4.14145 -5.1400000001 -6.1400 row1 line1 09:09:09.5678 text1 0x48656C6C6F20576F726C64 {"age": 30, "city": "London", "name": "Alice"} Option1,Option3 0x2A 0x48656C6C6F00000000000000 0x48656C6C6F Value2 -202 302 402 502 602 4.14159 5.1415926 6.14159 0 -124 -302 2013 -402 -502 -602 2012-11-01 2012-10-26T02:08:39.345700 2013-10-26T08:09:18 -5.14145 -6.1400000001 -7.1400 row2 line2 09:11:09.5678 text2 0xE86F6C6C6F20576F726C67 {"age": 18, "city": "ChongQing", "name": "Gaoxin"} Option1,Option2 0x2F 0x58676C6C6F00000000000000 0x88656C6C9F Value3 -203 303 403 503 603 7.14159 8.1415926 9.14159 0 \N -402 2017 -602 -902 -1102 2012-11-02 \N 2013-10-27T08:11:18 -5.14145 -6.1400000000001 -7.1400 row3 line3 09:11:09.5678 text3 0xE86F6C6C6F20576F726C67 {"age": 24, "city": "ChongQing", "name": "ChenQi"} Option2 0x2F 0x58676C6C6F00000000000000 \N Value1 - diff --git a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_oracle_jdbc_catalog_pool_test.out b/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_oracle_jdbc_catalog_pool_test.out deleted file mode 100644 index fe8440d8a659da4..000000000000000 --- a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_oracle_jdbc_catalog_pool_test.out +++ /dev/null @@ -1,109 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !query_ojdbc6_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457000 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !query_ojdbc6_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457000 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !query_ojdbc6_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457000 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !query_ojdbc6_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457000 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !query_ojdbc6_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457000 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !query_ojdbc6_all_types_refresh -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457000 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !query_ojdbc6_all_types -- -1 111 123 7456123.89 573 34 673.43 34.1264 60.0 23.231 99 9999 999999999 999999999999999999 999 99999 9999999999 9999999999999999999 1 china beijing alice abcdefghrjkmnopq 123.45 12300 0.0012345 2022-01-21T05:23:01 2019-11-12T20:33:57.999 2019-11-12T20:33:57.999998 2019-11-12T20:33:57.999996 2019-11-12T20:33:57.999997 223-9 12 10:23:1.123457000 -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - diff --git a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_pg_jdbc_catalog_pool_test.out b/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_pg_jdbc_catalog_pool_test.out deleted file mode 100644 index b063d2e7919e4c8..000000000000000 --- a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_pg_jdbc_catalog_pool_test.out +++ /dev/null @@ -1,69 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - --- !select_all_types_refresh -- -1 abc def 2022-10-11 1 2 3 2022-10-22T10:59:59 34.123 false 12.123456 10.16.10.14/32 10.16.10.14 ff:ff:ff:ff:ff:aa 1010101010 01010 1 {"id": 1} (1.0,1.0) {1.0,1.0,1.0} [(1.0,1.0),(2.0,2.0)] (2.0,2.0),(1.0,1.0) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) ((1.0,1.0),(2.0,2.0),(2.0,1.0)) <(0.0,0.0),1.0> -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N 2 \N \N \N \N \N \N \N \N - diff --git a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_sqlserver_jdbc_catalog_pool_test.out b/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_sqlserver_jdbc_catalog_pool_test.out deleted file mode 100644 index 87fe3a17dd78133..000000000000000 --- a/regression-test/data/external_table_p0/jdbc/connection_pool_test/test_sqlserver_jdbc_catalog_pool_test.out +++ /dev/null @@ -1,81 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - --- !all_type_refresh -- -1 doris 18 0 1 1 123.123 123.123 123.123 12345678901234567890123456789012345678 12345678901234567890123456789012345678 1234567890123456789012345678.0123456789 1234567890123456789012345678.0123456789 Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! Make Doris Great! 2023-01-17 16:49:05.123 2023-01-17T16:49:05 2023-01-17T16:49:05.123456 2023-01-17T16:49 2023-01-17 16:49:05 +08:00 Make Doris Great! Make Doris Great! 922337203685477.5807 214748.3647 false -2 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N - diff --git a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_clickhouse_jdbc_catalog_pool_test.groovy b/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_clickhouse_jdbc_catalog_pool_test.groovy deleted file mode 100644 index 4115b8321f73cbd..000000000000000 --- a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_clickhouse_jdbc_catalog_pool_test.groovy +++ /dev/null @@ -1,71 +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. - -suite("test_clickhouse_jdbc_catalog_pool_test", "p0,external,clickhouse,external_docker,external_docker_clickhouse") { - String enabled = context.config.otherConfigs.get("enableJdbcTest") - if (enabled != null && enabled.equalsIgnoreCase("true")) { - String ex_db_name = "doris_test"; - String clickhouse_port = context.config.otherConfigs.get("clickhouse_22_port"); - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - String s3_endpoint = getS3Endpoint() - String bucket = getS3BucketName() - String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/clickhouse-jdbc-0.4.2-all.jar" - - def poolOptions = [true, false] - - poolOptions.each { poolEnabled -> - String poolState = poolEnabled ? "true" : "false" - String catalog_name = "clickhouse_catalog_pool_${poolState}"; - - sql """ drop catalog if exists ${catalog_name} """ - sql """ create catalog if not exists ${catalog_name} properties( - "type"="jdbc", - "user"="default", - "password"="123456", - "jdbc_url" = "jdbc:clickhouse://${externalEnvIp}:${clickhouse_port}/doris_test", - "driver_url" = "${driver_url}", - "driver_class" = "com.clickhouse.jdbc.ClickHouseDriver", - "enable_connection_pool" = "${poolState}" - );""" - - def tasks = (1..5).collect { - Thread.start { - sql """ switch ${catalog_name} """ - sql """ use ${ex_db_name} """ - order_qt_type """ select * from type order by k1; """ - } - } - - tasks*.join() - - sql """refresh catalog ${catalog_name}""" - - def refreshTasks = (1..5).collect { - Thread.start { - sql """ switch ${catalog_name} """ - sql """ use ${ex_db_name} """ - order_qt_type_refresh """ select * from type order by k1; """ - } - } - - refreshTasks*.join() - - sql """ drop catalog if exists ${catalog_name} """ - } - } -} - diff --git a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_mysql_jdbc_catalog_pool_test.groovy b/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_mysql_jdbc_catalog_pool_test.groovy deleted file mode 100644 index 3b1150b3e8adc5c..000000000000000 --- a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_mysql_jdbc_catalog_pool_test.groovy +++ /dev/null @@ -1,73 +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. - -suite("test_mysql_jdbc_catalog_pool_test", "p0,external,mysql,external_docker,external_docker_mysql") { - String enabled = context.config.otherConfigs.get("enableJdbcTest") - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - String s3_endpoint = getS3Endpoint() - String bucket = getS3BucketName() - String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.3.0.jar" - - if (enabled != null && enabled.equalsIgnoreCase("true")) { - String ex_db_name = "doris_test"; - String mysql_port = context.config.otherConfigs.get("mysql_57_port"); - - def poolOptions = [true, false] - - poolOptions.each { poolEnabled -> - String poolState = poolEnabled ? "pool_true" : "pool_false" - String catalog_name = "mysql_jdbc_catalog_${poolState}"; - - sql """ drop catalog if exists ${catalog_name} """ - sql """ create catalog if not exists ${catalog_name} properties( - "type"="jdbc", - "user"="root", - "password"="123456", - "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}/doris_test?useSSL=false&zeroDateTimeBehavior=convertToNull", - "driver_url" = "${driver_url}", - "driver_class" = "com.mysql.cj.jdbc.Driver", - "enable_connection_pool" = "${poolEnabled}" - );""" - - def tasks = (1..5).collect { - Thread.start { - sql """ switch ${catalog_name} """ - sql """ use doris_test; """ - qt_mysql_all_types """ select * from all_types order by tinyint_u; """ - } - } - - tasks*.join() - - sql """ refresh catalog ${catalog_name} """ - - def refreshTasks = (1..5).collect { - Thread.start { - sql """ switch ${catalog_name} """ - sql """ use doris_test; """ - qt_mysql_all_types_refresh """ select * from all_types order by tinyint_u; """ - } - } - - refreshTasks*.join() - - sql """ drop catalog if exists ${catalog_name} """ - } - } -} - - diff --git a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_oracle_jdbc_catalog_pool_test.groovy b/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_oracle_jdbc_catalog_pool_test.groovy deleted file mode 100644 index 8ec0da5c0ea7390..000000000000000 --- a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_oracle_jdbc_catalog_pool_test.groovy +++ /dev/null @@ -1,111 +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. - -suite("test_oracle_jdbc_catalog_pool_test", "p0,external,oracle,external_docker,external_docker_oracle") { - String enabled = context.config.otherConfigs.get("enableJdbcTest"); - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - String s3_endpoint = getS3Endpoint() - String bucket = getS3BucketName() - String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/ojdbc8.jar" - String driver6_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/ojdbc6.jar" - - if (enabled != null && enabled.equalsIgnoreCase("true")) { - String ex_db_name = "DORIS_TEST"; - String oracle_port = context.config.otherConfigs.get("oracle_11_port"); - String SID = "XE"; - String test_all_types = "TEST_ALL_TYPES"; - - def poolOptions = [true, false] - - poolOptions.each { poolEnabled -> - String poolState = poolEnabled ? "pool_true" : "pool_false" - String catalog_name = "oracle_catalog_${poolState}"; - - sql """drop catalog if exists ${catalog_name} """ - - sql """create catalog if not exists ${catalog_name} properties( - "type"="jdbc", - "user"="doris_test", - "password"="123456", - "jdbc_url" = "jdbc:oracle:thin:@${externalEnvIp}:${oracle_port}:${SID}", - "driver_url" = "${driver_url}", - "driver_class" = "oracle.jdbc.driver.OracleDriver", - "enable_connection_pool" = "${poolEnabled}" - );""" - - def tasks = (1..5).collect { - Thread.start { - sql """switch ${catalog_name}""" - sql """ use ${ex_db_name}""" - order_qt_select_all_types """select * from ${test_all_types}; """ - } - } - tasks*.join() - - sql """refresh catalog ${catalog_name}""" - - def refreshTasks = (1..5).collect { - Thread.start { - sql """switch ${catalog_name}""" - sql """ use ${ex_db_name}""" - order_qt_select_all_types_refresh """select * from ${test_all_types}; """ - } - } - refreshTasks*.join() - - sql """drop catalog if exists ${catalog_name} """ - } - - poolOptions.each { poolEnabled -> - String poolState = poolEnabled ? "pool_true" : "pool_false" - String catalog_name = "oracle_ojdbc6_${poolState}"; - - sql """drop catalog if exists ${catalog_name} """ - - sql """create catalog if not exists ${catalog_name} properties( - "type"="jdbc", - "user"="doris_test", - "password"="123456", - "jdbc_url" = "jdbc:oracle:thin:@${externalEnvIp}:${oracle_port}:${SID}", - "driver_url" = "${driver6_url}", - "driver_class" = "oracle.jdbc.OracleDriver", - "enable_connection_pool" = "${poolEnabled}" - );""" - - - def tasks6 = (1..5).collect { - Thread.start { - sql """ use ${catalog_name}.DORIS_TEST; """ - qt_query_ojdbc6_all_types """ select * from ${catalog_name}.DORIS_TEST.TEST_ALL_TYPES order by 1; """ - } - } - tasks6*.join() - - sql """refresh catalog ${catalog_name};""" - - def refreshTasks6 = (1..5).collect { - Thread.start { - sql """ use ${catalog_name}.DORIS_TEST; """ - qt_query_ojdbc6_all_types_refresh """ select * from ${catalog_name}.DORIS_TEST.TEST_ALL_TYPES order by 1; """ - } - } - refreshTasks6*.join() - - sql """drop catalog if exists ${catalog_name}; """ - } - } -} diff --git a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_pg_jdbc_catalog_pool_test.groovy b/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_pg_jdbc_catalog_pool_test.groovy deleted file mode 100644 index 91f5c61b0baf9c2..000000000000000 --- a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_pg_jdbc_catalog_pool_test.groovy +++ /dev/null @@ -1,71 +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. - -suite("test_pg_jdbc_catalog_pool_test", "p0,external,pg,external_docker,external_docker_pg") { - String enabled = context.config.otherConfigs.get("enableJdbcTest") - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - String s3_endpoint = getS3Endpoint() - String bucket = getS3BucketName() - String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/postgresql-42.5.0.jar" - - if (enabled != null && enabled.equalsIgnoreCase("true")) { - String ex_schema_name = "catalog_pg_test"; - String pg_port = context.config.otherConfigs.get("pg_14_port"); - String test_all_types = "test_all_types"; - - def poolOptions = [true, false] - - poolOptions.each { poolEnabled -> - String poolState = poolEnabled ? "pool_true" : "pool_false" - String catalog_name = "pg_jdbc_catalog_${poolState}"; - - sql """drop catalog if exists ${catalog_name} """ - - sql """create catalog if not exists ${catalog_name} properties( - "type"="jdbc", - "user"="postgres", - "password"="123456", - "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/postgres?currentSchema=doris_test&useSSL=false", - "driver_url" = "${driver_url}", - "driver_class" = "org.postgresql.Driver", - "enable_connection_pool" = "${poolEnabled}" - );""" - - def tasks = (1..5).collect { - Thread.start { - sql """switch ${catalog_name}""" - sql """ use ${ex_schema_name}""" - order_qt_select_all_types """select * from ${test_all_types}; """ - } - } - tasks*.join() - - sql """refresh catalog ${catalog_name}""" - - def refreshTasks = (1..5).collect { - Thread.start { - sql """switch ${catalog_name}""" - sql """ use ${ex_schema_name}""" - order_qt_select_all_types_refresh """select * from ${test_all_types}; """ - } - } - refreshTasks*.join() - - sql """ drop catalog if exists ${catalog_name} """ - } - } -} diff --git a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_sqlserver_jdbc_catalog_pool_test.groovy b/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_sqlserver_jdbc_catalog_pool_test.groovy deleted file mode 100644 index a8ebc8952fb8503..000000000000000 --- a/regression-test/suites/external_table_p0/jdbc/connection_pool_test/test_sqlserver_jdbc_catalog_pool_test.groovy +++ /dev/null @@ -1,70 +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. - -suite("test_sqlserver_jdbc_catalog_pool_test", "p0,external,sqlserver,external_docker,external_docker_sqlserver") { - String enabled = context.config.otherConfigs.get("enableJdbcTest"); - String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") - String s3_endpoint = getS3Endpoint() - String bucket = getS3BucketName() - String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mssql-jdbc-11.2.3.jre8.jar" - - if (enabled != null && enabled.equalsIgnoreCase("true")) { - String ex_db_name = "dbo"; - String sqlserver_port = context.config.otherConfigs.get("sqlserver_2022_port"); - - def poolOptions = [true, false] - - poolOptions.each { poolEnabled -> - String poolState = poolEnabled ? "pool_true" : "pool_false" - String catalog_name = "sqlserver_catalog_${poolState}"; - - sql """ drop catalog if exists ${catalog_name} """ - - sql """ create catalog if not exists ${catalog_name} properties( - "type"="jdbc", - "user"="sa", - "password"="Doris123456", - "jdbc_url" = "jdbc:sqlserver://${externalEnvIp}:${sqlserver_port};encrypt=false;databaseName=doris_test;", - "driver_url" = "${driver_url}", - "driver_class" = "com.microsoft.sqlserver.jdbc.SQLServerDriver", - "enable_connection_pool" = "${poolEnabled}" - );""" - - def tasks = (1..5).collect { - Thread.start { - sql """ switch ${catalog_name} """ - sql """ use ${ex_db_name} """ - order_qt_all_type """ select * from all_type order by id; """ - } - } - tasks*.join() - - sql """refresh catalog ${catalog_name}""" - - def refreshTasks = (1..5).collect { - Thread.start { - sql """ switch ${catalog_name} """ - sql """ use ${ex_db_name} """ - order_qt_all_type_refresh """ select * from all_type order by id; """ - } - } - refreshTasks*.join() - - sql """ drop catalog if exists ${catalog_name} """ - } - } -} diff --git a/regression-test/suites/manager/test_manager_interface_3.groovy b/regression-test/suites/manager/test_manager_interface_3.groovy index 21b4853da9aa31e..52e5f5f99db55be 100644 --- a/regression-test/suites/manager/test_manager_interface_3.groovy +++ b/regression-test/suites/manager/test_manager_interface_3.groovy @@ -424,7 +424,7 @@ suite('test_manager_interface_3',"p0") { x ++ } } - assertTrue(x == 21) + assertTrue(x == 20) connect(user=user, password="${pwd}", url=url) { result = sql """ show resources """ @@ -435,7 +435,7 @@ suite('test_manager_interface_3',"p0") { x ++ } } - assertTrue(x == 21) + assertTrue(x == 20) } @@ -469,7 +469,7 @@ suite('test_manager_interface_3',"p0") { } sql """grant USAGE_PRIV on RESOURCE ${resource_name} TO '${user}' """ - connect(user=user, password="${pwd}", url=url) { + connect(user=user, password="${pwd}", url=url) { result = sql """ show resources """ x = 0 for(int i = 0;i Date: Fri, 1 Nov 2024 09:51:00 +0800 Subject: [PATCH 25/82] [opt][file cache] Do not return error if file cache can not be created if ignore_broken_disk is set (#42145) ## Proposed changes Issue Number: close #xxx If `ignore_broken_disk` is set, do not return error if file cache can not be created Signed-off-by: ZijieLu --- be/src/runtime/exec_env_init.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 9e1536d5186cfd8..3d8affade82d273 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -442,8 +442,11 @@ void ExecEnv::init_file_cache_factory(std::vector& cache_paths } for (const auto& status : cache_status) { if (!status.ok()) { - LOG(FATAL) << "failed to init file cache, err: " << status; - exit(-1); + if (!doris::config::ignore_broken_disk) { + LOG(FATAL) << "failed to init file cache, err: " << status; + exit(-1); + } + LOG(WARNING) << "failed to init file cache, err: " << status; } } } From e9e332737aa1f8148a8f25897c1e1ab79f1f83ac Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 1 Nov 2024 09:55:12 +0800 Subject: [PATCH 26/82] [refactor](metrics) Complete metrics for operators (#43002) --- be/src/pipeline/exec/result_sink_operator.cpp | 11 ++++-- be/src/pipeline/exec/result_sink_operator.h | 5 ++- .../pipeline/exec/set_probe_sink_operator.cpp | 11 +++++- .../pipeline/exec/set_probe_sink_operator.h | 3 ++ be/src/pipeline/exec/set_sink_operator.cpp | 8 +++- be/src/pipeline/exec/set_sink_operator.h | 6 +-- be/src/pipeline/exec/set_source_operator.cpp | 37 +++++++++++-------- be/src/pipeline/exec/set_source_operator.h | 3 ++ be/src/pipeline/exec/sort_sink_operator.cpp | 8 +++- be/src/pipeline/exec/sort_sink_operator.h | 2 + .../pipeline/exec/table_function_operator.cpp | 22 +++++++++-- .../pipeline/exec/table_function_operator.h | 8 ++++ be/src/pipeline/exec/union_sink_operator.cpp | 1 + be/src/pipeline/exec/union_sink_operator.h | 2 + 14 files changed, 95 insertions(+), 32 deletions(-) diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index 99c20e3c2e64429..53a517f859c4e3c 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -41,12 +41,12 @@ Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _fetch_row_id_timer = ADD_TIMER(profile(), "FetchRowIdTime"); + _write_data_timer = ADD_TIMER(profile(), "WriteDataTime"); static const std::string timer_name = "WaitForDependencyTime"; _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL(_profile, timer_name, 1); auto fragment_instance_id = state->fragment_instance_id(); - _blocks_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "BlocksProduced", TUnit::UNIT, 1); - if (state->query_options().enable_parallel_result_sink) { _sender = _parent->cast()._sender; } else { @@ -146,11 +146,14 @@ Status ResultSinkOperatorX::sink(RuntimeState* state, vectorized::Block* block, auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)block->rows()); - COUNTER_UPDATE(local_state.blocks_sent_counter(), 1); if (_fetch_option.use_two_phase_fetch && block->rows() > 0) { + SCOPED_TIMER(local_state._fetch_row_id_timer); RETURN_IF_ERROR(_second_phase_fetch_data(state, block)); } - RETURN_IF_ERROR(local_state._writer->write(state, *block)); + { + SCOPED_TIMER(local_state._write_data_timer); + RETURN_IF_ERROR(local_state._writer->write(state, *block)); + } if (_fetch_option.use_two_phase_fetch) { // Block structure may be changed by calling _second_phase_fetch_data(). // So we should clear block in case of unmatched columns diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index ec5d480c38d1782..339c167825643bd 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -128,7 +128,6 @@ class ResultSinkLocalState final : public PipelineXSinkLocalState _sender = nullptr; std::shared_ptr _writer = nullptr; - RuntimeProfile::Counter* _blocks_sent_counter = nullptr; + + RuntimeProfile::Counter* _fetch_row_id_timer = nullptr; + RuntimeProfile::Counter* _write_data_timer = nullptr; }; class ResultSinkOperatorX final : public DataSinkOperatorX { diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index 37db9afacfcacd4..813dad3ad79de64 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -71,12 +71,16 @@ Status SetProbeSinkOperatorX::sink(RuntimeState* state, vectorized auto probe_rows = in_block->rows(); if (probe_rows > 0) { - RETURN_IF_ERROR(_extract_probe_column(local_state, *in_block, local_state._probe_columns, - _cur_child_id)); + { + SCOPED_TIMER(local_state._extract_probe_data_timer); + RETURN_IF_ERROR(_extract_probe_column(local_state, *in_block, + local_state._probe_columns, _cur_child_id)); + } RETURN_IF_ERROR(std::visit( [&](auto&& arg) -> Status { using HashTableCtxType = std::decay_t; if constexpr (!std::is_same_v) { + SCOPED_TIMER(local_state._probe_timer); vectorized::HashTableProbe process_hashtable_ctx(&local_state, probe_rows); return process_hashtable_ctx.mark_data_in_hashtable(arg); @@ -99,6 +103,9 @@ Status SetProbeSinkLocalState::init(RuntimeState* state, LocalSink RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + + _probe_timer = ADD_TIMER(Base::profile(), "ProbeTime"); + _extract_probe_data_timer = ADD_TIMER(Base::profile(), "ExtractProbeDataTime"); Parent& parent = _parent->cast(); _shared_state->probe_finished_children_dependency[parent._cur_child_id] = _dependency; _dependency->block(); diff --git a/be/src/pipeline/exec/set_probe_sink_operator.h b/be/src/pipeline/exec/set_probe_sink_operator.h index f320c8e89cdcaf4..368ea812cdfe013 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.h +++ b/be/src/pipeline/exec/set_probe_sink_operator.h @@ -60,6 +60,9 @@ class SetProbeSinkLocalState final : public PipelineXSinkLocalState diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index 9a81333efaed89d..539134e53e7fe21 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -40,8 +40,10 @@ Status SetSinkOperatorX::sink(RuntimeState* state, vectorized::Blo auto& valid_element_in_hash_tbl = local_state._shared_state->valid_element_in_hash_tbl; if (in_block->rows() != 0) { - RETURN_IF_ERROR(local_state._mutable_block.merge(*in_block)); - + { + SCOPED_TIMER(local_state._merge_block_timer); + RETURN_IF_ERROR(local_state._mutable_block.merge(*in_block)); + } if (local_state._mutable_block.rows() > std::numeric_limits::max()) { return Status::NotSupported("set operator do not support build table rows over:" + std::to_string(std::numeric_limits::max())); @@ -49,6 +51,7 @@ Status SetSinkOperatorX::sink(RuntimeState* state, vectorized::Blo } if (eos || local_state._mutable_block.allocated_bytes() >= BUILD_BLOCK_MAX_SIZE) { + SCOPED_TIMER(local_state._build_timer); build_block = local_state._mutable_block.to_block(); RETURN_IF_ERROR(_process_build_block(local_state, build_block, state)); local_state._mutable_block.clear(); @@ -152,6 +155,7 @@ Status SetSinkLocalState::init(RuntimeState* state, LocalSinkState RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _merge_block_timer = ADD_TIMER(_profile, "MergeBlocksTime"); _build_timer = ADD_TIMER(_profile, "BuildTime"); auto& parent = _parent->cast(); _shared_state->probe_finished_children_dependency[parent._cur_child_id] = _dependency; diff --git a/be/src/pipeline/exec/set_sink_operator.h b/be/src/pipeline/exec/set_sink_operator.h index 8e3c264f267f0b5..ba387d97b413600 100644 --- a/be/src/pipeline/exec/set_sink_operator.h +++ b/be/src/pipeline/exec/set_sink_operator.h @@ -49,14 +49,14 @@ class SetSinkLocalState final : public PipelineXSinkLocalState { private: friend class SetSinkOperatorX; - template - friend struct vectorized::HashTableBuild; - RuntimeProfile::Counter* _build_timer; // time to build hash table vectorized::MutableBlock _mutable_block; // every child has its result expr list vectorized::VExprContextSPtrs _child_exprs; vectorized::Arena _arena; + + RuntimeProfile::Counter* _merge_block_timer = nullptr; + RuntimeProfile::Counter* _build_timer = nullptr; }; template diff --git a/be/src/pipeline/exec/set_source_operator.cpp b/be/src/pipeline/exec/set_source_operator.cpp index 58958462c2f021a..ebcd13ddf14ce40 100644 --- a/be/src/pipeline/exec/set_source_operator.cpp +++ b/be/src/pipeline/exec/set_source_operator.cpp @@ -29,6 +29,8 @@ Status SetSourceLocalState::init(RuntimeState* state, LocalStateIn RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _get_data_timer = ADD_TIMER(_runtime_profile, "GetDataTime"); + _filter_timer = ADD_TIMER(_runtime_profile, "FilterTime"); _shared_state->probe_finished_children_dependency.resize( _parent->cast>()._child_quantity, nullptr); return Status::OK(); @@ -75,21 +77,26 @@ Status SetSourceOperatorX::get_block(RuntimeState* state, vectoriz auto& local_state = get_local_state(state); SCOPED_TIMER(local_state.exec_time_counter()); _create_mutable_cols(local_state, block); - auto st = std::visit( - [&](auto&& arg) -> Status { - using HashTableCtxType = std::decay_t; - if constexpr (!std::is_same_v) { - return _get_data_in_hashtable(local_state, arg, block, - state->batch_size(), eos); - } else { - LOG(FATAL) << "FATAL: uninited hash table"; - __builtin_unreachable(); - } - }, - local_state._shared_state->hash_table_variants->method_variant); - RETURN_IF_ERROR(st); - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, - block->columns())); + { + SCOPED_TIMER(local_state._get_data_timer); + RETURN_IF_ERROR(std::visit( + [&](auto&& arg) -> Status { + using HashTableCtxType = std::decay_t; + if constexpr (!std::is_same_v) { + return _get_data_in_hashtable(local_state, arg, block, + state->batch_size(), eos); + } else { + LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); + } + }, + local_state._shared_state->hash_table_variants->method_variant)); + } + { + SCOPED_TIMER(local_state._filter_timer); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(local_state._conjuncts, block, + block->columns())); + } local_state.reached_limit(block, eos); return Status::OK(); } diff --git a/be/src/pipeline/exec/set_source_operator.h b/be/src/pipeline/exec/set_source_operator.h index ce3d0c52edf1d52..976ffde3bf23eae 100644 --- a/be/src/pipeline/exec/set_source_operator.h +++ b/be/src/pipeline/exec/set_source_operator.h @@ -46,6 +46,9 @@ class SetSourceLocalState final : public PipelineXLocalState { std::vector _mutable_cols; //record build column type vectorized::DataTypes _left_table_data_types; + + RuntimeProfile::Counter* _get_data_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; }; template diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index 6d6684437b81249..faec4961af93b7f 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -32,6 +32,8 @@ Status SortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { SCOPED_TIMER(_init_timer); _sort_blocks_memory_usage = ADD_COUNTER_WITH_LEVEL(_profile, "MemoryUsageSortBlocks", TUnit::BYTES, 1); + _append_blocks_timer = ADD_TIMER(profile(), "AppendBlockTime"); + _update_runtime_predicate_timer = ADD_TIMER(profile(), "UpdateRuntimePredicateTime"); return Status::OK(); } @@ -119,7 +121,10 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in SCOPED_TIMER(local_state.exec_time_counter()); COUNTER_UPDATE(local_state.rows_input_counter(), (int64_t)in_block->rows()); if (in_block->rows() > 0) { - RETURN_IF_ERROR(local_state._shared_state->sorter->append_block(in_block)); + { + SCOPED_TIMER(local_state._append_blocks_timer); + RETURN_IF_ERROR(local_state._shared_state->sorter->append_block(in_block)); + } int64_t data_size = local_state._shared_state->sorter->data_size(); COUNTER_SET(local_state._sort_blocks_memory_usage, data_size); COUNTER_SET(local_state._memory_used_counter, data_size); @@ -128,6 +133,7 @@ Status SortSinkOperatorX::sink(doris::RuntimeState* state, vectorized::Block* in RETURN_IF_CANCELLED(state); if (state->get_query_ctx()->has_runtime_predicate(_node_id)) { + SCOPED_TIMER(local_state._update_runtime_predicate_timer); auto& predicate = state->get_query_ctx()->get_runtime_predicate(_node_id); if (predicate.enable()) { vectorized::Field new_top = local_state._shared_state->sorter->get_top_value(); diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index a5a24e371635fe7..6bf87164e71026e 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -46,6 +46,8 @@ class SortSinkLocalState : public PipelineXSinkLocalState { // topn top value vectorized::Field old_top {vectorized::Field::Types::Null}; + RuntimeProfile::Counter* _append_blocks_timer = nullptr; + RuntimeProfile::Counter* _update_runtime_predicate_timer = nullptr; }; class SortSinkOperatorX final : public DataSinkOperatorX { diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index 38e69f7cb0e897e..c1621470f435b4f 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -32,6 +32,18 @@ namespace doris::pipeline { TableFunctionLocalState::TableFunctionLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalState<>(state, parent), _child_block(vectorized::Block::create_unique()) {} +Status TableFunctionLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_init_timer); + _init_function_timer = ADD_TIMER(_runtime_profile, "InitTableFunctionTime"); + _process_rows_timer = ADD_TIMER(_runtime_profile, "ProcessRowsTime"); + _copy_data_timer = ADD_TIMER(_runtime_profile, "CopyDataTime"); + _filter_timer = ADD_TIMER(_runtime_profile, "FilterTime"); + _repeat_data_timer = ADD_TIMER(_runtime_profile, "RepeatDataTime"); + return Status::OK(); +} + Status TableFunctionLocalState::open(RuntimeState* state) { SCOPED_TIMER(PipelineXLocalState<>::exec_time_counter()); SCOPED_TIMER(PipelineXLocalState<>::_open_timer); @@ -59,6 +71,7 @@ void TableFunctionLocalState::_copy_output_slots( if (!_current_row_insert_times) { return; } + SCOPED_TIMER(_copy_data_timer); auto& p = _parent->cast(); for (auto index : p._output_slot_indexs) { auto src_column = _child_block->get_by_position(index).column; @@ -197,15 +210,18 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, columns[index]->insert_many_defaults(row_size - columns[index]->size()); } - // 3. eval conjuncts - RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, output_block, - output_block->columns())); + { + SCOPED_TIMER(_filter_timer); // 3. eval conjuncts + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, output_block, + output_block->columns())); + } *eos = _child_eos && _cur_child_offset == -1; return Status::OK(); } void TableFunctionLocalState::process_next_child_row() { + SCOPED_TIMER(_process_rows_timer); _cur_child_offset++; if (_cur_child_offset >= _child_block->rows()) { diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index 75b1608fad7112a..81160acb7f7611c 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -37,6 +37,7 @@ class TableFunctionLocalState final : public PipelineXLocalState<> { TableFunctionLocalState(RuntimeState* state, OperatorXBase* parent); ~TableFunctionLocalState() override = default; + Status init(RuntimeState* state, LocalStateInfo& infos) override; Status open(RuntimeState* state) override; Status close(RuntimeState* state) override { for (auto* fn : _fns) { @@ -67,6 +68,12 @@ class TableFunctionLocalState final : public PipelineXLocalState<> { std::unique_ptr _child_block; int _current_row_insert_times = 0; bool _child_eos = false; + + RuntimeProfile::Counter* _init_function_timer = nullptr; + RuntimeProfile::Counter* _process_rows_timer = nullptr; + RuntimeProfile::Counter* _copy_data_timer = nullptr; + RuntimeProfile::Counter* _filter_timer = nullptr; + RuntimeProfile::Counter* _repeat_data_timer = nullptr; }; class TableFunctionOperatorX final : public StatefulOperatorX { @@ -93,6 +100,7 @@ class TableFunctionOperatorX final : public StatefulOperatorXprocess_init(input_block, state)); } local_state.process_next_child_row(); diff --git a/be/src/pipeline/exec/union_sink_operator.cpp b/be/src/pipeline/exec/union_sink_operator.cpp index 288fc131037fabb..8467eeb1d5467a6 100644 --- a/be/src/pipeline/exec/union_sink_operator.cpp +++ b/be/src/pipeline/exec/union_sink_operator.cpp @@ -32,6 +32,7 @@ Status UnionSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_init_timer); + _expr_timer = ADD_TIMER(_profile, "ExprTime"); auto& p = _parent->cast(); _shared_state->data_queue.set_sink_dependency(_dependency, p._cur_child_id); return Status::OK(); diff --git a/be/src/pipeline/exec/union_sink_operator.h b/be/src/pipeline/exec/union_sink_operator.h index f939950143ae920..aa94ed9a73038fb 100644 --- a/be/src/pipeline/exec/union_sink_operator.h +++ b/be/src/pipeline/exec/union_sink_operator.h @@ -55,6 +55,7 @@ class UnionSinkLocalState final : public PipelineXSinkLocalState { @@ -136,6 +137,7 @@ class UnionSinkOperatorX final : public DataSinkOperatorX { Status materialize_block(RuntimeState* state, vectorized::Block* src_block, int child_idx, vectorized::Block* res_block) { auto& local_state = get_local_state(state); + SCOPED_TIMER(local_state._expr_timer); const auto& child_exprs = local_state._child_expr; vectorized::ColumnsWithTypeAndName colunms; for (size_t i = 0; i < child_exprs.size(); ++i) { From 659cd821037579fb6ebade9a4df429ab49308429 Mon Sep 17 00:00:00 2001 From: Mryange <59914473+Mryange@users.noreply.github.com> Date: Fri, 1 Nov 2024 10:37:30 +0800 Subject: [PATCH 27/82] [env](compile) open compile check in sink file (#42574) open compile check in sink file --- be/src/util/mysql_row_buffer.cpp | 4 ++-- be/src/util/mysql_row_buffer.h | 2 +- be/src/vec/sink/autoinc_buffer.cpp | 1 + be/src/vec/sink/autoinc_buffer.h | 9 ++++++--- be/src/vec/sink/delta_writer_v2_pool.cpp | 1 + be/src/vec/sink/delta_writer_v2_pool.h | 3 +++ be/src/vec/sink/load_stream_map_pool.cpp | 1 + be/src/vec/sink/load_stream_map_pool.h | 3 +++ be/src/vec/sink/load_stream_stub.cpp | 10 ++++++---- be/src/vec/sink/load_stream_stub.h | 10 ++++++++-- be/src/vec/sink/varrow_flight_result_writer.cpp | 1 + be/src/vec/sink/varrow_flight_result_writer.h | 3 +++ be/src/vec/sink/vdata_stream_sender.cpp | 1 + be/src/vec/sink/vdata_stream_sender.h | 3 +++ be/src/vec/sink/vmysql_result_writer.cpp | 6 ++++-- be/src/vec/sink/vmysql_result_writer.h | 3 +++ be/src/vec/sink/vrow_distribution.cpp | 11 ++++++----- be/src/vec/sink/vrow_distribution.h | 3 +++ be/src/vec/sink/vtablet_block_convertor.cpp | 9 +++++---- be/src/vec/sink/vtablet_block_convertor.h | 10 ++++++---- be/src/vec/sink/vtablet_finder.cpp | 1 + be/src/vec/sink/vtablet_finder.h | 4 +++- be/test/io/fs/stream_sink_file_writer_test.cpp | 2 +- 23 files changed, 72 insertions(+), 29 deletions(-) diff --git a/be/src/util/mysql_row_buffer.cpp b/be/src/util/mysql_row_buffer.cpp index 4823920508a9404..3e20a2d9de72fec 100644 --- a/be/src/util/mysql_row_buffer.cpp +++ b/be/src/util/mysql_row_buffer.cpp @@ -87,9 +87,9 @@ MysqlRowBuffer::MysqlRowBuffer() _len_pos(0) {} template -void MysqlRowBuffer::start_binary_row(uint32_t num_cols) { +void MysqlRowBuffer::start_binary_row(uint64_t num_cols) { assert(is_binary_format); - int bit_fields = (num_cols + 9) / 8; + auto bit_fields = (num_cols + 9) / 8; reserve(bit_fields + 1); memset(_pos, 0, 1 + bit_fields); _pos += bit_fields + 1; diff --git a/be/src/util/mysql_row_buffer.h b/be/src/util/mysql_row_buffer.h index b740efa7764ed1f..50b17c91c170ca8 100644 --- a/be/src/util/mysql_row_buffer.h +++ b/be/src/util/mysql_row_buffer.h @@ -62,7 +62,7 @@ class MysqlRowBuffer { // Prepare for binary row buffer // init bitmap - void start_binary_row(uint32_t num_cols); + void start_binary_row(uint64_t num_cols); // TODO(zhaochun): add signed/unsigned support int push_tinyint(int8_t data); diff --git a/be/src/vec/sink/autoinc_buffer.cpp b/be/src/vec/sink/autoinc_buffer.cpp index 4bc87dff48958c4..4c45b7bc6313d7b 100644 --- a/be/src/vec/sink/autoinc_buffer.cpp +++ b/be/src/vec/sink/autoinc_buffer.cpp @@ -30,6 +30,7 @@ #include "util/thrift_rpc_helper.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" AutoIncIDBuffer::AutoIncIDBuffer(int64_t db_id, int64_t table_id, int64_t column_id) : _db_id(db_id), diff --git a/be/src/vec/sink/autoinc_buffer.h b/be/src/vec/sink/autoinc_buffer.h index 032ac18981f4dae..82be3d9faad00f7 100644 --- a/be/src/vec/sink/autoinc_buffer.h +++ b/be/src/vec/sink/autoinc_buffer.h @@ -18,12 +18,14 @@ #pragma once #include +#include "common/cast_set.h" #include "common/config.h" #include "common/factory_creator.h" #include "common/status.h" #include "util/threadpool.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class VOlapTableSink; class OlapTableBlockConvertor; @@ -118,8 +120,8 @@ class GlobalAutoIncBuffers { GlobalAutoIncBuffers() { static_cast(ThreadPoolBuilder("AsyncFetchAutoIncIDExecutor") - .set_min_threads(config::auto_inc_fetch_thread_num) - .set_max_threads(config::auto_inc_fetch_thread_num) + .set_min_threads(cast_set(config::auto_inc_fetch_thread_num)) + .set_max_threads(cast_set(config::auto_inc_fetch_thread_num)) .set_max_queue_size(std::numeric_limits::max()) .build(&_fetch_autoinc_id_executor)); } @@ -146,4 +148,5 @@ class GlobalAutoIncBuffers { std::mutex _mutex; }; -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/delta_writer_v2_pool.cpp b/be/src/vec/sink/delta_writer_v2_pool.cpp index bc5233ac30796ea..c9ae424d4d4fe42 100644 --- a/be/src/vec/sink/delta_writer_v2_pool.cpp +++ b/be/src/vec/sink/delta_writer_v2_pool.cpp @@ -21,6 +21,7 @@ #include "util/runtime_profile.h" namespace doris { +#include "common/compile_check_begin.h" class TExpr; namespace vectorized { diff --git a/be/src/vec/sink/delta_writer_v2_pool.h b/be/src/vec/sink/delta_writer_v2_pool.h index 7e58eea31498f64..677d062f6b33630 100644 --- a/be/src/vec/sink/delta_writer_v2_pool.h +++ b/be/src/vec/sink/delta_writer_v2_pool.h @@ -51,6 +51,7 @@ #include "util/uid_util.h" namespace doris { +#include "common/compile_check_begin.h" class DeltaWriterV2; class RuntimeProfile; @@ -108,3 +109,5 @@ class DeltaWriterV2Pool { } // namespace vectorized } // namespace doris + +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/load_stream_map_pool.cpp b/be/src/vec/sink/load_stream_map_pool.cpp index dc78d306e70666d..24a1cb77a489cc3 100644 --- a/be/src/vec/sink/load_stream_map_pool.cpp +++ b/be/src/vec/sink/load_stream_map_pool.cpp @@ -20,6 +20,7 @@ #include "util/debug_points.h" namespace doris { +#include "common/compile_check_begin.h" class TExpr; LoadStreamMap::LoadStreamMap(UniqueId load_id, int64_t src_id, int num_streams, int num_use, diff --git a/be/src/vec/sink/load_stream_map_pool.h b/be/src/vec/sink/load_stream_map_pool.h index 4ecae2f16bea7f5..f1ed7b0da16ba3f 100644 --- a/be/src/vec/sink/load_stream_map_pool.h +++ b/be/src/vec/sink/load_stream_map_pool.h @@ -65,6 +65,7 @@ #include "vec/sink/load_stream_stub.h" namespace doris { +#include "common/compile_check_begin.h" class LoadStreamStub; @@ -136,3 +137,5 @@ class LoadStreamMapPool { }; } // namespace doris + +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/load_stream_stub.cpp b/be/src/vec/sink/load_stream_stub.cpp index 2a38b179b95531f..979daf6a85e6821 100644 --- a/be/src/vec/sink/load_stream_stub.cpp +++ b/be/src/vec/sink/load_stream_stub.cpp @@ -19,6 +19,7 @@ #include +#include "common/cast_set.h" #include "olap/rowset/rowset_writer.h" #include "runtime/query_context.h" #include "util/brpc_client_cache.h" @@ -28,6 +29,7 @@ #include "util/uid_util.h" namespace doris { +#include "common/compile_check_begin.h" int LoadStreamReplyHandler::on_received_messages(brpc::StreamId id, butil::IOBuf* const messages[], size_t size) { @@ -92,7 +94,7 @@ int LoadStreamReplyHandler::on_received_messages(brpc::StreamId id, butil::IOBuf TRuntimeProfileTree tprofile; const uint8_t* buf = reinterpret_cast(response.load_stream_profile().data()); - uint32_t len = response.load_stream_profile().size(); + uint32_t len = cast_set(response.load_stream_profile().size()); auto status = deserialize_thrift_msg(buf, &len, false, &tprofile); if (status.ok()) { // TODO @@ -154,7 +156,7 @@ Status LoadStreamStub::open(BrpcClientCache* client_cache, _is_init.store(true); _dst_id = node_info.id; brpc::StreamOptions opt; - opt.max_buf_size = config::load_stream_max_buf_size; + opt.max_buf_size = cast_set(config::load_stream_max_buf_size); opt.idle_timeout_ms = idle_timeout_ms; opt.messages_in_batch = config::load_stream_messages_in_batch; opt.handler = new LoadStreamReplyHandler(_load_id, _dst_id, shared_from_this()); @@ -213,7 +215,7 @@ Status LoadStreamStub::open(BrpcClientCache* client_cache, // APPEND_DATA Status LoadStreamStub::append_data(int64_t partition_id, int64_t index_id, int64_t tablet_id, - int64_t segment_id, uint64_t offset, std::span data, + int32_t segment_id, uint64_t offset, std::span data, bool segment_eos, FileType file_type) { if (!_is_open.load()) { add_failed_tablet(tablet_id, _status); @@ -240,7 +242,7 @@ Status LoadStreamStub::append_data(int64_t partition_id, int64_t index_id, int64 // ADD_SEGMENT Status LoadStreamStub::add_segment(int64_t partition_id, int64_t index_id, int64_t tablet_id, - int64_t segment_id, const SegmentStatistics& segment_stat, + int32_t segment_id, const SegmentStatistics& segment_stat, TabletSchemaSPtr flush_schema) { if (!_is_open.load()) { add_failed_tablet(tablet_id, _status); diff --git a/be/src/vec/sink/load_stream_stub.h b/be/src/vec/sink/load_stream_stub.h index 9816770c82e6725..cad7a90492ad324 100644 --- a/be/src/vec/sink/load_stream_stub.h +++ b/be/src/vec/sink/load_stream_stub.h @@ -69,6 +69,7 @@ #include "vec/exprs/vexpr_fwd.h" namespace doris { +#include "common/compile_check_begin.h" class TabletSchema; class LoadStreamStub; @@ -133,15 +134,18 @@ class LoadStreamStub : public std::enable_shared_from_this { #ifdef BE_TEST virtual #endif + // segment_id is limited by max_segment_num_per_rowset (default value of 1000), + // so in practice it will not exceed the range of i16. + // APPEND_DATA Status append_data(int64_t partition_id, int64_t index_id, int64_t tablet_id, - int64_t segment_id, uint64_t offset, std::span data, + int32_t segment_id, uint64_t offset, std::span data, bool segment_eos = false, FileType file_type = FileType::SEGMENT_FILE); // ADD_SEGMENT Status add_segment(int64_t partition_id, int64_t index_id, int64_t tablet_id, - int64_t segment_id, const SegmentStatistics& segment_stat, + int32_t segment_id, const SegmentStatistics& segment_stat, TabletSchemaSPtr flush_schema); // CLOSE_LOAD @@ -335,3 +339,5 @@ class LoadStreamStubs { }; } // namespace doris + +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/varrow_flight_result_writer.cpp b/be/src/vec/sink/varrow_flight_result_writer.cpp index b23d1668465bbd0..77788c52ef39fcd 100644 --- a/be/src/vec/sink/varrow_flight_result_writer.cpp +++ b/be/src/vec/sink/varrow_flight_result_writer.cpp @@ -25,6 +25,7 @@ #include "vec/exprs/vexpr_context.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { VArrowFlightResultWriter::VArrowFlightResultWriter( diff --git a/be/src/vec/sink/varrow_flight_result_writer.h b/be/src/vec/sink/varrow_flight_result_writer.h index ab2578421c80bcf..fcae0350b822c9b 100644 --- a/be/src/vec/sink/varrow_flight_result_writer.h +++ b/be/src/vec/sink/varrow_flight_result_writer.h @@ -30,6 +30,7 @@ #include "vec/exprs/vexpr_fwd.h" namespace doris { +#include "common/compile_check_begin.h" class BufferControlBlock; class RuntimeState; @@ -77,3 +78,5 @@ class VArrowFlightResultWriter final : public ResultWriter { }; } // namespace vectorized } // namespace doris + +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index d21b87561b5da7d..2cace094e4d1c85 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -53,6 +53,7 @@ #include "vec/sink/writer/vtablet_writer_v2.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" Status Channel::init(RuntimeState* state) { if (_brpc_dest_addr.hostname.empty()) { diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index 024d87ab32f49c7..5fe35e4da119d03 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -53,6 +53,7 @@ #include "vec/sink/vtablet_finder.h" namespace doris { +#include "common/compile_check_begin.h" class ObjectPool; class RuntimeState; class MemTracker; @@ -233,3 +234,5 @@ class Channel { } // namespace vectorized } // namespace doris + +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/vmysql_result_writer.cpp b/be/src/vec/sink/vmysql_result_writer.cpp index 932ee9555907655..d1e5baa860a054c 100644 --- a/be/src/vec/sink/vmysql_result_writer.cpp +++ b/be/src/vec/sink/vmysql_result_writer.cpp @@ -29,6 +29,7 @@ #include #include +#include "common/cast_set.h" #include "common/compiler_util.h" // IWYU pragma: keep #include "common/config.h" #include "gutil/integral_types.h" @@ -70,6 +71,7 @@ #include "vec/runtime/vdatetime_value.h" namespace doris { +#include "common/compile_check_begin.h" namespace vectorized { template @@ -143,7 +145,7 @@ Status VMysqlResultWriter::_set_options( template Status VMysqlResultWriter::_write_one_block(RuntimeState* state, Block& block) { Status status = Status::OK(); - auto num_rows = block.rows(); + int num_rows = cast_set(block.rows()); // convert one batch auto result = std::make_unique(); result->result_batch.rows.resize(num_rows); @@ -200,7 +202,7 @@ Status VMysqlResultWriter::_write_one_block(RuntimeState* stat } } - for (size_t row_idx = 0; row_idx < num_rows; ++row_idx) { + for (int row_idx = 0; row_idx < num_rows; ++row_idx) { for (size_t col_idx = 0; col_idx < num_cols; ++col_idx) { RETURN_IF_ERROR(arguments[col_idx].serde->write_column_to_mysql( *(arguments[col_idx].column), row_buffer, row_idx, diff --git a/be/src/vec/sink/vmysql_result_writer.h b/be/src/vec/sink/vmysql_result_writer.h index b89b8cf1b9086af..1c7da4f4b377800 100644 --- a/be/src/vec/sink/vmysql_result_writer.h +++ b/be/src/vec/sink/vmysql_result_writer.h @@ -31,6 +31,7 @@ #include "vec/exprs/vexpr_fwd.h" namespace doris { +#include "common/compile_check_begin.h" class BufferControlBlock; class RuntimeState; @@ -96,3 +97,5 @@ class VMysqlResultWriter final : public ResultWriter { }; } // namespace vectorized } // namespace doris + +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/vrow_distribution.cpp b/be/src/vec/sink/vrow_distribution.cpp index b79df49f0626d6e..3c15dbc8f0f81ed 100644 --- a/be/src/vec/sink/vrow_distribution.cpp +++ b/be/src/vec/sink/vrow_distribution.cpp @@ -42,6 +42,7 @@ #include "vec/sink/writer/vtablet_writer.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" std::pair VRowDistribution::_get_partition_function() { @@ -299,7 +300,7 @@ Status VRowDistribution::_filter_block(vectorized::Block* block, Status VRowDistribution::_generate_rows_distribution_for_non_auto_partition( vectorized::Block* block, bool has_filtered_rows, std::vector& row_part_tablet_ids) { - auto num_rows = block->rows(); + int num_rows = cast_set(block->rows()); bool stop_processing = false; RETURN_IF_ERROR(_tablet_finder->find_tablets(_state, block, num_rows, _partitions, @@ -318,7 +319,7 @@ Status VRowDistribution::_deal_missing_map(vectorized::Block* block, int64_t& rows_stat_val) { // for missing partition keys, calc the missing partition and save in _partitions_need_create auto [part_ctxs, part_exprs] = _get_partition_function(); - auto part_col_num = part_exprs.size(); + int part_col_num = cast_set(part_exprs.size()); // the two vectors are in column-first-order std::vector> col_strs; std::vector col_null_maps; @@ -363,7 +364,7 @@ Status VRowDistribution::_generate_rows_distribution_for_auto_partition( vectorized::Block* block, const std::vector& partition_cols_idx, bool has_filtered_rows, std::vector& row_part_tablet_ids, int64_t& rows_stat_val) { - auto num_rows = block->rows(); + int num_rows = cast_set(block->rows()); std::vector partition_keys = _vpartition->get_partition_keys(); auto& partition_col = block->get_by_position(partition_keys[0]); @@ -393,7 +394,7 @@ Status VRowDistribution::_generate_rows_distribution_for_auto_overwrite( vectorized::Block* block, const std::vector& partition_cols_idx, bool has_filtered_rows, std::vector& row_part_tablet_ids, int64_t& rows_stat_val) { - auto num_rows = block->rows(); + int num_rows = cast_set(block->rows()); // for non-auto-partition situation, goes into two 'else' branch. just find the origin partitions, replace them by rpc, // and find the new partitions to use. @@ -504,7 +505,7 @@ Status VRowDistribution::generate_rows_distribution( VLOG_DEBUG << "Partition-calculated block:" << block->dump_data(0, 1); DCHECK(result_idx != -1); - partition_cols_idx.push_back(result_idx); + partition_cols_idx.push_back(cast_set(result_idx)); } // change the column to compare to transformed. diff --git a/be/src/vec/sink/vrow_distribution.h b/be/src/vec/sink/vrow_distribution.h index 88002c3c21139d8..87fd801984ad735 100644 --- a/be/src/vec/sink/vrow_distribution.h +++ b/be/src/vec/sink/vrow_distribution.h @@ -40,6 +40,7 @@ #include "vec/sink/vtablet_finder.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class IndexChannel; class VNodeChannel; @@ -230,3 +231,5 @@ class VRowDistribution { }; } // namespace doris::vectorized + +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/vtablet_block_convertor.cpp b/be/src/vec/sink/vtablet_block_convertor.cpp index 820759af2e41b0f..26de6ea6c7e3d18 100644 --- a/be/src/vec/sink/vtablet_block_convertor.cpp +++ b/be/src/vec/sink/vtablet_block_convertor.cpp @@ -55,6 +55,7 @@ #include "vec/exprs/vexpr_context.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" Status OlapTableBlockConvertor::validate_and_convert_block( RuntimeState* state, vectorized::Block* input_block, @@ -186,11 +187,11 @@ DecimalType OlapTableBlockConvertor::_get_decimalv3_min_or_max(const TypeDescrip Status OlapTableBlockConvertor::_internal_validate_column( RuntimeState* state, const TypeDescriptor& type, bool is_nullable, vectorized::ColumnPtr column, size_t slot_index, bool* stop_processing, - fmt::memory_buffer& error_prefix, const uint32_t row_count, + fmt::memory_buffer& error_prefix, const size_t row_count, vectorized::IColumn::Permutation* rows) { DCHECK((rows == nullptr) || (rows->size() == row_count)); fmt::memory_buffer error_msg; - auto set_invalid_and_append_error_msg = [&](int row) { + auto set_invalid_and_append_error_msg = [&](size_t row) { _filter_map[row] = true; auto ret = state->append_error_msg_to_file([]() -> std::string { return ""; }, [&error_prefix, &error_msg]() -> std::string { @@ -218,7 +219,7 @@ Status OlapTableBlockConvertor::_internal_validate_column( auto* __restrict offsets = column_string->get_offsets().data(); int invalid_count = 0; - for (int j = 0; j < row_count; ++j) { + for (int64_t j = 0; j < row_count; ++j) { invalid_count += (offsets[j] - offsets[j - 1]) > limit; } @@ -452,7 +453,7 @@ Status OlapTableBlockConvertor::_internal_validate_column( } Status OlapTableBlockConvertor::_validate_data(RuntimeState* state, vectorized::Block* block, - const uint32_t rows, int& filtered_rows, + const size_t rows, int& filtered_rows, bool* stop_processing) { for (int i = 0; i < _output_tuple_desc->slots().size(); ++i) { SlotDescriptor* desc = _output_tuple_desc->slots()[i]; diff --git a/be/src/vec/sink/vtablet_block_convertor.h b/be/src/vec/sink/vtablet_block_convertor.h index 7f866c380327752..16921e082dcd62e 100644 --- a/be/src/vec/sink/vtablet_block_convertor.h +++ b/be/src/vec/sink/vtablet_block_convertor.h @@ -36,6 +36,7 @@ #include "vec/sink/autoinc_buffer.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class OlapTableBlockConvertor { public: @@ -68,7 +69,7 @@ class OlapTableBlockConvertor { Status _validate_column(RuntimeState* state, const TypeDescriptor& type, bool is_nullable, vectorized::ColumnPtr column, size_t slot_index, bool* stop_processing, - fmt::memory_buffer& error_prefix, const uint32_t row_count, + fmt::memory_buffer& error_prefix, const size_t row_count, vectorized::IColumn::Permutation* rows = nullptr) { RETURN_IF_CATCH_EXCEPTION({ return _internal_validate_column(state, type, is_nullable, column, slot_index, @@ -79,14 +80,14 @@ class OlapTableBlockConvertor { Status _internal_validate_column(RuntimeState* state, const TypeDescriptor& type, bool is_nullable, vectorized::ColumnPtr column, size_t slot_index, bool* stop_processing, - fmt::memory_buffer& error_prefix, const uint32_t row_count, + fmt::memory_buffer& error_prefix, const size_t row_count, vectorized::IColumn::Permutation* rows = nullptr); // make input data valid for OLAP table // return number of invalid/filtered rows. // invalid row number is set in Bitmap // set stop_processing if we want to stop the whole process now. - Status _validate_data(RuntimeState* state, vectorized::Block* block, const uint32_t rows, + Status _validate_data(RuntimeState* state, vectorized::Block* block, const size_t rows, int& filtered_rows, bool* stop_processing); // some output column of output expr may have different nullable property with dest slot desc @@ -123,4 +124,5 @@ class OlapTableBlockConvertor { bool _is_partial_update_and_auto_inc = false; }; -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/src/vec/sink/vtablet_finder.cpp b/be/src/vec/sink/vtablet_finder.cpp index 3bfd5bb4d22e96c..c72da75d02a29ba 100644 --- a/be/src/vec/sink/vtablet_finder.cpp +++ b/be/src/vec/sink/vtablet_finder.cpp @@ -32,6 +32,7 @@ #include "vec/core/block.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" Status OlapTabletFinder::find_tablets(RuntimeState* state, Block* block, int rows, std::vector& partitions, std::vector& tablet_index, bool& stop_processing, diff --git a/be/src/vec/sink/vtablet_finder.h b/be/src/vec/sink/vtablet_finder.h index 24f8e357e28976c..67eb7f08e3e93de 100644 --- a/be/src/vec/sink/vtablet_finder.h +++ b/be/src/vec/sink/vtablet_finder.h @@ -27,6 +27,7 @@ #include "vec/core/block.h" namespace doris::vectorized { +#include "common/compile_check_begin.h" class OlapTabletFinder { public: @@ -75,4 +76,5 @@ class OlapTabletFinder { Bitmap _filter_bitmap; }; -} // namespace doris::vectorized \ No newline at end of file +} // namespace doris::vectorized +#include "common/compile_check_end.h" diff --git a/be/test/io/fs/stream_sink_file_writer_test.cpp b/be/test/io/fs/stream_sink_file_writer_test.cpp index 69f286b205bd6fe..ef0a75288eb6032 100644 --- a/be/test/io/fs/stream_sink_file_writer_test.cpp +++ b/be/test/io/fs/stream_sink_file_writer_test.cpp @@ -59,7 +59,7 @@ class StreamSinkFileWriterTest : public testing::Test { // APPEND_DATA virtual Status append_data(int64_t partition_id, int64_t index_id, int64_t tablet_id, - int64_t segment_id, uint64_t offset, std::span data, + int32_t segment_id, uint64_t offset, std::span data, bool segment_eos = false, FileType file_type = FileType::SEGMENT_FILE) override { EXPECT_EQ(PARTITION_ID, partition_id); From faff1981db064c0d3ec8d86db48623a873a004cd Mon Sep 17 00:00:00 2001 From: qiye Date: Fri, 1 Nov 2024 11:15:29 +0800 Subject: [PATCH 28/82] [fix](build index)Fix build index failed on renamed column (#42882) Add `column_unique_ids` in `TOlapTableIndex` thrift struct, to get col_unique_id when building index. --- be/src/olap/tablet_schema.cpp | 10 +++++++++- be/src/olap/task/index_builder.cpp | 12 +++++++---- .../doris/alter/SchemaChangeHandler.java | 3 +++ .../doris/analysis/BuildIndexClause.java | 2 +- .../doris/analysis/CreateIndexClause.java | 2 +- .../doris/analysis/CreateTableStmt.java | 3 ++- .../org/apache/doris/analysis/IndexDef.java | 5 +++++ .../java/org/apache/doris/catalog/Index.java | 20 ++++++++++++++++--- .../doris/catalog/MaterializedIndexMeta.java | 7 +++++++ .../plans/commands/info/IndexDefinition.java | 2 +- .../apache/doris/catalog/OlapTableTest.java | 2 +- .../common/proc/IndexesProcNodeTest.java | 8 ++++---- .../TableAddOrDropColumnsInfoTest.java | 2 +- gensrc/thrift/Descriptors.thrift | 1 + ...test_index_change_on_renamed_column.groovy | 13 +++++++++++- 15 files changed, 73 insertions(+), 19 deletions(-) diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp index c88a23a0c360cf9..4041afac78ee136 100644 --- a/be/src/olap/tablet_schema.cpp +++ b/be/src/olap/tablet_schema.cpp @@ -747,7 +747,15 @@ void TabletIndex::init_from_thrift(const TOlapTableIndex& index, if (column_idx >= 0) { col_unique_ids[i] = tablet_schema.column(column_idx).unique_id(); } else { - col_unique_ids[i] = -1; + // if column unique id not found by column name, find by column unique id + // column unique id can not bigger than tablet schema column size, if bigger than column size means + // this column is a new column added by light schema change + if (index.__isset.column_unique_ids && + index.column_unique_ids[i] < tablet_schema.num_columns()) { + col_unique_ids[i] = index.column_unique_ids[i]; + } else { + col_unique_ids[i] = -1; + } } } _col_unique_ids = std::move(col_unique_ids); diff --git a/be/src/olap/task/index_builder.cpp b/be/src/olap/task/index_builder.cpp index 09cbdeadb3f3c97..8f8c3f7ad8004e5 100644 --- a/be/src/olap/task/index_builder.cpp +++ b/be/src/olap/task/index_builder.cpp @@ -381,10 +381,14 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta auto column_name = inverted_index.columns[0]; auto column_idx = output_rowset_schema->field_index(column_name); if (column_idx < 0) { - LOG(WARNING) << "referenced column was missing. " - << "[column=" << column_name << " referenced_column=" << column_idx - << "]"; - continue; + column_idx = + output_rowset_schema->field_index(inverted_index.column_unique_ids[0]); + if (column_idx < 0) { + LOG(WARNING) << "referenced column was missing. " + << "[column=" << column_name + << " referenced_column=" << column_idx << "]"; + continue; + } } auto column = output_rowset_schema->column(column_idx); if (!InvertedIndexColumnWriter::check_support_inverted_index(column)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 916cfd05bc9c079..0981e3e6538a6d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -2135,6 +2135,7 @@ public int getAsInt() { index.setIndexId(existedIdx.getIndexId()); index.setColumns(existedIdx.getColumns()); index.setProperties(existedIdx.getProperties()); + index.setColumnUniqueIds(existedIdx.getColumnUniqueIds()); if (indexDef.getPartitionNames().isEmpty()) { invertedIndexOnPartitions.put(index.getIndexId(), olapTable.getPartitionNames()); } else { @@ -2735,6 +2736,7 @@ private boolean processAddIndex(CreateIndexClause alterClause, OlapTable olapTab if (column != null) { indexDef.checkColumn(column, olapTable.getKeysType(), olapTable.getTableProperty().getEnableUniqueKeyMergeOnWrite()); + indexDef.getColumnUniqueIds().add(column.getUniqueId()); } else { throw new DdlException("index column does not exist in table. invalid column: " + col); } @@ -2745,6 +2747,7 @@ private boolean processAddIndex(CreateIndexClause alterClause, OlapTable olapTab // so here update column name in CreateIndexClause after checkColumn for indexDef, // there will use the column name in olapTable insead of the column name in CreateIndexClause. alterIndex.setColumns(indexDef.getColumns()); + alterIndex.setColumnUniqueIds(indexDef.getColumnUniqueIds()); newIndexes.add(alterIndex); return false; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BuildIndexClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BuildIndexClause.java index cb7ec08de78f9c8..c65766a1ae87211 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BuildIndexClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BuildIndexClause.java @@ -73,7 +73,7 @@ public void analyze(Analyzer analyzer) throws AnalysisException { indexDef.analyze(); this.index = new Index(Env.getCurrentEnv().getNextId(), indexDef.getIndexName(), indexDef.getColumns(), indexDef.getIndexType(), - indexDef.getProperties(), indexDef.getComment()); + indexDef.getProperties(), indexDef.getComment(), indexDef.getColumnUniqueIds()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java index b39c0df4a85db5a..86df87453ad5751 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateIndexClause.java @@ -73,7 +73,7 @@ public void analyze(Analyzer analyzer) throws AnalysisException { indexDef.analyze(); this.index = new Index(Env.getCurrentEnv().getNextId(), indexDef.getIndexName(), indexDef.getColumns(), indexDef.getIndexType(), - indexDef.getProperties(), indexDef.getComment()); + indexDef.getProperties(), indexDef.getComment(), indexDef.getColumnUniqueIds()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java index f92da90e5c43bd3..b07424056906b87 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java @@ -618,7 +618,8 @@ public void analyze(Analyzer analyzer) throws UserException { } } indexes.add(new Index(Env.getCurrentEnv().getNextId(), indexDef.getIndexName(), indexDef.getColumns(), - indexDef.getIndexType(), indexDef.getProperties(), indexDef.getComment())); + indexDef.getIndexType(), indexDef.getProperties(), indexDef.getComment(), + indexDef.getColumnUniqueIds())); distinct.add(indexDef.getIndexName()); distinctCol.add(Pair.of(indexDef.getIndexType(), indexDef.getColumns().stream().map(String::toUpperCase).collect(Collectors.toList()))); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java index 87bf7c5aa189cbd..b2ee45372973bfb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/IndexDef.java @@ -42,6 +42,7 @@ public class IndexDef { private Map properties; private boolean isBuildDeferred = false; private PartitionNames partitionNames; + private List columnUniqueIds = Lists.newArrayList(); public static final String NGRAM_SIZE_KEY = "gram_size"; public static final String NGRAM_BF_SIZE_KEY = "bf_size"; @@ -196,6 +197,10 @@ public List getPartitionNames() { return partitionNames == null ? Lists.newArrayList() : partitionNames.getPartitionNames(); } + public List getColumnUniqueIds() { + return columnUniqueIds; + } + public enum IndexType { BITMAP, INVERTED, diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java index 56a878c8f93948a..40db2f1d5b01d50 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Index.java @@ -48,7 +48,7 @@ /** * Internal representation of index, including index type, name, columns and comments. - * This class will used in olaptable + * This class will be used in olap table */ public class Index implements Writable { public static final int INDEX_ID_INIT_VALUE = -1; @@ -65,15 +65,19 @@ public class Index implements Writable { private Map properties; @SerializedName(value = "ct", alternate = {"comment"}) private String comment; + @SerializedName(value = "cui", alternate = {"columnUniqueIds"}) + private List columnUniqueIds; public Index(long indexId, String indexName, List columns, - IndexDef.IndexType indexType, Map properties, String comment) { + IndexDef.IndexType indexType, Map properties, String comment, + List columnUniqueIds) { this.indexId = indexId; this.indexName = indexName; this.columns = columns == null ? Lists.newArrayList() : Lists.newArrayList(columns); this.indexType = indexType; this.properties = properties == null ? Maps.newHashMap() : Maps.newHashMap(properties); this.comment = comment; + this.columnUniqueIds = columnUniqueIds == null ? Lists.newArrayList() : Lists.newArrayList(columnUniqueIds); if (indexType == IndexDef.IndexType.INVERTED) { if (this.properties != null && !this.properties.isEmpty()) { if (this.properties.containsKey(InvertedIndexUtil.INVERTED_INDEX_PARSER_KEY)) { @@ -97,6 +101,7 @@ public Index() { this.indexType = null; this.properties = null; this.comment = null; + this.columnUniqueIds = null; } public long getIndexId() { @@ -186,6 +191,14 @@ public void setComment(String comment) { this.comment = comment; } + public List getColumnUniqueIds() { + return columnUniqueIds; + } + + public void setColumnUniqueIds(List columnUniqueIds) { + this.columnUniqueIds = columnUniqueIds; + } + @Override public void write(DataOutput out) throws IOException { Text.writeString(out, GsonUtils.GSON.toJson(this)); @@ -203,7 +216,7 @@ public int hashCode() { public Index clone() { return new Index(indexId, indexName, new ArrayList<>(columns), - indexType, new HashMap<>(properties), comment); + indexType, new HashMap<>(properties), comment, columnUniqueIds); } @Override @@ -247,6 +260,7 @@ public TOlapTableIndex toThrift() { if (properties != null) { tIndex.setProperties(properties); } + tIndex.setColumnUniqueIds(columnUniqueIds); return tIndex; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java index 6125e0334003c55..5dd5776c7619021 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java @@ -387,6 +387,13 @@ public void initSchemaColumnUniqueId() { maxColUniqueId = Column.COLUMN_UNIQUE_ID_INIT_VALUE; this.schema.forEach(column -> { column.setUniqueId(incAndGetMaxColUniqueId()); + this.indexes.forEach(index -> { + index.getColumns().forEach(col -> { + if (col.equalsIgnoreCase(column.getName())) { + index.getColumnUniqueIds().add(column.getUniqueId()); + } + }); + }); if (LOG.isDebugEnabled()) { LOG.debug("indexId: {}, column:{}, uniqueId:{}", indexId, column, column.getUniqueId()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java index d134687106e2568..340ea58150419f9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/IndexDefinition.java @@ -207,6 +207,6 @@ public IndexType getIndexType() { public Index translateToCatalogStyle() { return new Index(Env.getCurrentEnv().getNextId(), name, cols, indexType, properties, - comment); + comment, null); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java index 950371de3034bd7..84b8c6062d2351b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableTest.java @@ -59,7 +59,7 @@ int getCurrentEnvJournalVersion() { } OlapTable tbl = (OlapTable) table; tbl.setIndexes(Lists.newArrayList(new Index(0, "index", Lists.newArrayList("col"), - IndexDef.IndexType.BITMAP, null, "xxxxxx"))); + IndexDef.IndexType.BITMAP, null, "xxxxxx", Lists.newArrayList(1)))); System.out.println("orig table id: " + tbl.getId()); FastByteArrayOutputStream byteArrayOutputStream = new FastByteArrayOutputStream(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/proc/IndexesProcNodeTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/proc/IndexesProcNodeTest.java index aeb5bc471fee421..966f6c38b5b7833 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/common/proc/IndexesProcNodeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/common/proc/IndexesProcNodeTest.java @@ -42,18 +42,18 @@ public class IndexesProcNodeTest { public void testFetchResult() throws AnalysisException { List indexes = new ArrayList<>(); Index indexBitmap = new Index(1, "bitmap_index", Lists.newArrayList("col_1"), - IndexType.BITMAP, null, "bitmap index on col_1"); + IndexType.BITMAP, null, "bitmap index on col_1", Lists.newArrayList(1)); Map invertedProperties = new HashMap<>(); invertedProperties.put("parser", "unicode"); Index indexInverted = new Index(2, "inverted_index", Lists.newArrayList("col_2"), - IndexType.INVERTED, invertedProperties, "inverted index on col_2"); + IndexType.INVERTED, invertedProperties, "inverted index on col_2", Lists.newArrayList(2)); Index indexBf = new Index(3, "bloomfilter_index", Lists.newArrayList("col_3"), - IndexType.BLOOMFILTER, null, "bloomfilter index on col_3"); + IndexType.BLOOMFILTER, null, "bloomfilter index on col_3", Lists.newArrayList(3)); Map ngramProperties = new HashMap<>(); ngramProperties.put("gram_size", "3"); ngramProperties.put("bf_size", "256"); Index indexNgramBf = new Index(4, "ngram_bf_index", Lists.newArrayList("col_4"), - IndexType.NGRAM_BF, ngramProperties, "ngram_bf index on col_4"); + IndexType.NGRAM_BF, ngramProperties, "ngram_bf index on col_4", Lists.newArrayList(4)); indexes.add(indexBitmap); indexes.add(indexInverted); indexes.add(indexBf); diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java index 0e04bddd681fddf..be71998eac3f778 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java @@ -65,7 +65,7 @@ public void testSerialization() throws IOException { indexSchemaMap.put(tableId, fullSchema); List indexes = Lists.newArrayList( - new Index(0, "index", Lists.newArrayList("testCol1"), IndexDef.IndexType.INVERTED, null, "xxxxxx")); + new Index(0, "index", Lists.newArrayList("testCol1"), IndexDef.IndexType.INVERTED, null, "xxxxxx", Lists.newArrayList(1))); TableAddOrDropColumnsInfo tableAddOrDropColumnsInfo1 = new TableAddOrDropColumnsInfo("", dbId, tableId, indexSchemaMap, indexes, jobId); diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index dd6ebf2248e083f..8319b8466952708 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -228,6 +228,7 @@ struct TOlapTableIndex { 4: optional string comment 5: optional i64 index_id 6: optional map properties + 7: optional list column_unique_ids } struct TOlapTableIndexSchema { diff --git a/regression-test/suites/inverted_index_p0/index_change/test_index_change_on_renamed_column.groovy b/regression-test/suites/inverted_index_p0/index_change/test_index_change_on_renamed_column.groovy index 3913b60786ed48a..6c8b8c4e0d8fbc7 100644 --- a/regression-test/suites/inverted_index_p0/index_change/test_index_change_on_renamed_column.groovy +++ b/regression-test/suites/inverted_index_p0/index_change/test_index_change_on_renamed_column.groovy @@ -18,6 +18,10 @@ import org.codehaus.groovy.runtime.IOGroovyMethods suite("test_index_change_on_renamed_column") { + def backendId_to_backendIP = [:] + def backendId_to_backendHttpPort = [:] + getBackendIpHttpPort(backendId_to_backendIP, backendId_to_backendHttpPort); + def timeout = 60000 def delta_time = 1000 def alter_res = "null" @@ -67,7 +71,7 @@ suite("test_index_change_on_renamed_column") { `id` INT COMMENT "", `s` STRING COMMENT "" ) - DUPLICATE KEY(`id`) DISTRIBUTED BY HASH(`id`) + DUPLICATE KEY(`id`) DISTRIBUTED BY HASH(`id`) BUCKETS 1 PROPERTIES ( "replication_num" = "1" ); """ @@ -101,6 +105,13 @@ suite("test_index_change_on_renamed_column") { qt_select2 """ SELECT * FROM ${tableName} order by id; """ qt_select3 """ SELECT * FROM ${tableName} where s1 match 'welcome'; """ + def tablets = sql_return_maparray """ show tablets from ${tableName}; """ + String tablet_id = tablets[0].TabletId + String backend_id = tablets[0].BackendId + String ip = backendId_to_backendIP.get(backend_id) + String port = backendId_to_backendHttpPort.get(backend_id) + check_nested_index_file(ip, port, tablet_id, 3, 1, "V2") + // drop inverted index on renamed column sql """ alter table ${tableName} drop index idx_s; """ wait_for_latest_op_on_table_finish(tableName, timeout) From 01e2b997e29a4d0a35810f07f0fe76da725b28e6 Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Fri, 1 Nov 2024 11:33:59 +0800 Subject: [PATCH 29/82] [deps](hadoop) update hadoop to 3.3.6.4 (#43019) ### What problem does this PR solve? Issue Number: close #xxx Related PR: #xxx Problem Summary: ### Check List (For Committer) - Test - [ ] Regression test - [ ] Unit Test - [ ] Manual test (add detailed scripts or steps below) - [ ] No need to test or manual test. Explain why: - [ ] This is a refactor/code format and no logic has been changed. - [ ] Previous test can cover this change. - [ ] No colde files have been changed. - [ ] Other reason - Behavior changed: - [x] No. - [ ] Yes. - Does this need documentation? - [x] No. - [ ] Yes. - Release note None ### Check List (For Reviewer who merge this PR) - [x] Confirm the release note - [x] Confirm test cases - [x] Confirm document - [ ] Add branch pick label --- thirdparty/CHANGELOG.md | 4 ++++ thirdparty/vars.sh | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/thirdparty/CHANGELOG.md b/thirdparty/CHANGELOG.md index 03c6b1ce8abdd8b..7c74c7a09974304 100644 --- a/thirdparty/CHANGELOG.md +++ b/thirdparty/CHANGELOG.md @@ -2,6 +2,10 @@ This file contains version of the third-party dependency libraries in the build-env image. The docker build-env image is apache/doris, and the tag is `build-env-${version}` +## 20241031 + +- Modified: hadoop-libs 3.3.6.3 -> 3.3.6.4 + ## 20240815 - Modified: arrow 15.0.2 -> 17.0.0 diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh index 1471aff0b591e49..5e442a2b80805a1 100644 --- a/thirdparty/vars.sh +++ b/thirdparty/vars.sh @@ -461,10 +461,10 @@ FAST_FLOAT_SOURCE=fast_float-3.9.0 FAST_FLOAT_MD5SUM="5656b0d8b150a3b157cfb092d214f6ea" # libhdfs -HADOOP_LIBS_DOWNLOAD="https://github.com/apache/doris-thirdparty/archive/refs/tags/hadoop-3.3.6.3-for-doris.tar.gz" -HADOOP_LIBS_NAME="hadoop-3.3.6.3-for-doris.tar.gz" -HADOOP_LIBS_SOURCE="doris-thirdparty-hadoop-3.3.6.3-for-doris" -HADOOP_LIBS_MD5SUM="db91164b8b1d6b347fb9199e6be77fb7" +HADOOP_LIBS_DOWNLOAD="https://github.com/apache/doris-thirdparty/archive/refs/tags/hadoop-3.3.6.4-for-doris.tar.gz" +HADOOP_LIBS_NAME="hadoop-3.3.6.4-for-doris.tar.gz" +HADOOP_LIBS_SOURCE="doris-thirdparty-hadoop-3.3.6.4-for-doris" +HADOOP_LIBS_MD5SUM="6993de49713eb0892e42c8ec57fbdc6c" # AvxToNeon AVX2NEON_DOWNLOAD="https://github.com/kunpengcompute/AvxToNeon/archive/refs/tags/v1.0.0.tar.gz" From 309037aa81df76470cfaa67f5cb89f29f446713f Mon Sep 17 00:00:00 2001 From: zfr95 <87513668+zfr9527@users.noreply.github.com> Date: Fri, 1 Nov 2024 11:39:28 +0800 Subject: [PATCH 30/82] [test](stats)Add upgrade and downgrade test for analyze (#42855) Mainly test the basic functions of collecting statistical information, including operations such as analyze and show. Mainly worry about metadata information such as changes in statistical information tables leading to problems in upgrading and downgrading versions. Here, judge whether there will be problems with test cases on the new and old versions to check. --- .../suites/stats_up_down/load.groovy | 66 ++++++++++++++++ .../test_upgrade_downgrade_stats.groovy | 75 +++++++++++++++++++ 2 files changed, 141 insertions(+) create mode 100644 regression-test/suites/stats_up_down/load.groovy create mode 100644 regression-test/suites/stats_up_down/test_upgrade_downgrade_stats.groovy diff --git a/regression-test/suites/stats_up_down/load.groovy b/regression-test/suites/stats_up_down/load.groovy new file mode 100644 index 000000000000000..57fe3d0dd1259a5 --- /dev/null +++ b/regression-test/suites/stats_up_down/load.groovy @@ -0,0 +1,66 @@ +// 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. + +suite("test_upgrade_downgrade_prepare_stats","p0,stats,restart_fe") { + + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + + sql """ + drop table if exists orders_stats + """ + + sql """CREATE TABLE `orders_stats` ( + `o_orderkey` BIGINT NULL, + `o_custkey` INT NULL, + `o_orderstatus` VARCHAR(1) NULL, + `o_totalprice` DECIMAL(15, 2) NULL, + `o_orderpriority` VARCHAR(15) NULL, + `o_clerk` VARCHAR(15) NULL, + `o_shippriority` INT NULL, + `o_comment` VARCHAR(79) NULL, + `o_orderdate` DATE not NULL + ) ENGINE=OLAP + DUPLICATE KEY(`o_orderkey`, `o_custkey`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into orders_stats values + (null, 1, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (1, null, 'o', 109.2, 'c','d',2, 'mm', '2023-10-17'), + (3, 3, null, 99.5, 'a', 'b', 1, 'yy', '2023-10-19'), + (1, 2, 'o', null, 'a', 'b', 1, 'yy', '2023-10-20'), + (2, 3, 'k', 109.2, null,'d',2, 'mm', '2023-10-21'), + (3, 1, 'k', 99.5, 'a', null, 1, 'yy', '2023-10-22'), + (1, 3, 'o', 99.5, 'a', 'b', null, 'yy', '2023-10-19'), + (2, 1, 'o', 109.2, 'c','d',2, null, '2023-10-18'), + (3, 2, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-17'), + (4, 5, 'k', 99.5, 'a', 'b', 1, 'yy', '2023-10-19'); + """ + + sql """analyze table orders_stats with sync""" + + def tb_stats = sql """show table stats orders_stats;""" + assertTrue(tb_stats.size() == 1) + + def col_stats = sql """show column stats orders_stats;""" + assertTrue(col_stats.size() == 9) +} diff --git a/regression-test/suites/stats_up_down/test_upgrade_downgrade_stats.groovy b/regression-test/suites/stats_up_down/test_upgrade_downgrade_stats.groovy new file mode 100644 index 000000000000000..978928390c41346 --- /dev/null +++ b/regression-test/suites/stats_up_down/test_upgrade_downgrade_stats.groovy @@ -0,0 +1,75 @@ +// 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. + +suite("test_upgrade_downgrade_compatibility_stats","p0,stats,restart_fe") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + + def tb_stats = sql """show table stats orders_stats;""" + assertTrue(tb_stats.size() == 1) + + def col_stats = sql """show column stats orders_stats;""" + assertTrue(col_stats.size() == 9) + + sql """ + drop table if exists lineitem_stats + """ + + sql """CREATE TABLE `lineitem_stats` ( + `l_orderkey` BIGINT NULL, + `l_linenumber` INT NULL, + `l_partkey` INT NULL, + `l_suppkey` INT NULL, + `l_quantity` DECIMAL(15, 2) NULL, + `l_extendedprice` DECIMAL(15, 2) NULL, + `l_discount` DECIMAL(15, 2) NULL, + `l_tax` DECIMAL(15, 2) NULL, + `l_returnflag` VARCHAR(1) NULL, + `l_linestatus` VARCHAR(1) NULL, + `l_commitdate` DATE NULL, + `l_receiptdate` DATE NULL, + `l_shipinstruct` VARCHAR(25) NULL, + `l_shipmode` VARCHAR(10) NULL, + `l_comment` VARCHAR(44) NULL, + `l_shipdate` DATE not NULL + ) ENGINE=OLAP + DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey ) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 96 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + + sql """ + insert into lineitem_stats values + (null, 1, 2, 3, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (1, null, 3, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (3, 3, null, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 2, 3, null, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'), + (2, 3, 2, 1, 5.5, 6.5, 7.5, 8.5, 'o', 'k', null, '2023-10-18', 'a', 'b', 'yyyyyyyyy', '2023-10-18'), + (3, 1, 1, 2, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', null, 'c', 'd', 'xxxxxxxxx', '2023-10-19'), + (1, 3, 2, 2, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy', '2023-10-17'); + """ + + sql """analyze table lineitem_stats with sync;""" + + def tb_stats2 = sql """show table stats lineitem_stats;""" + assertTrue(tb_stats2.size() == 1) + + def col_stats2 = sql """show column stats lineitem_stats;""" + assertTrue(col_stats2.size() == 16) +} From 55fde4575d69a46015053116c11a3b47586158fe Mon Sep 17 00:00:00 2001 From: seawinde <149132972+seawinde@users.noreply.github.com> Date: Fri, 1 Nov 2024 14:09:18 +0800 Subject: [PATCH 31/82] [test](mv) Insert into more data when first insert into to make sure using sync mv (#43010) Root Cause Analysis: Currently, the statistics reported by BE (Backend) nodes have higher priority than those from ANALYZE statements. During the first INSERT INTO operation, the system waits for row count reports from all tablets before updating the table statistics. Subsequent INSERT INTO operations cannot obtain the status of all tablets, so the system continues to use the statistical information from the first INSERT INTO operation. This leads to a lower estimated cost for the original table's query plan, resulting in the selection of the original table's query plan instead of the materialized view. Conclusion: The test case should be modified to include a larger dataset in the first INSERT INTO operation, which will increase the likelihood of utilizing the materialized view. This is because the cost estimation will better reflect the actual data distribution and size, leading to more accurate plan selection. --- .../data/mv_p0/ssb/q_1_1/q_1_1.out | 387 +----------------- .../suites/mv_p0/ssb/q_1_1/q_1_1.groovy | 7 +- 2 files changed, 10 insertions(+), 384 deletions(-) diff --git a/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out b/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out index d0962795f0b490e..1da845620e8dd74 100644 --- a/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out +++ b/regression-test/data/mv_p0/ssb/q_1_1/q_1_1.out @@ -1,394 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select_star -- 19930101 1 1 1 1 1 1 1 1 1 1 100 1 1 1 2023-06-09 shipmode name address city nation AMERICA phone mktsegment name address city nation AMERICA phone name MFGR#1 category brand color type 4 container +19930101 1 1 1 1 1 1 1 1 1 1 100 1 1 1 2023-06-09 shipmode name address city nation AMERICA phone mktsegment name address city nation AMERICA phone name MFGR#1 category brand color type 4 container +19930101 1 1 1 1 1 1 1 1 1 1 100 1 1 1 2023-06-09 shipmode name address city nation AMERICA phone mktsegment name address city nation AMERICA phone name MFGR#1 category brand color type 4 container +19930101 1 1 1 1 1 1 1 1 1 1 100 1 1 1 2023-06-09 shipmode name address city nation AMERICA phone mktsegment name address city nation AMERICA phone name MFGR#1 category brand color type 4 container +19930101 1 1 1 1 1 1 1 1 1 1 100 1 1 1 2023-06-09 shipmode name address city nation AMERICA phone mktsegment name address city nation AMERICA phone name MFGR#1 category brand color type 4 container 19930101 2 2 2 2 2 2 2 2 2 2 2 2 2 2 2023-06-09 shipmode name address city nation region phone mktsegment name address city nation region phone name mfgr category brand color type 4 container -- !select_mv -- 4 --- !abc -- -root:Group[@4] - logical expressions: - id:349#4 cost=null estRows=1 children=[@3 ] (plan=LogicalResultSink[350] ( outputExprs=[revenue#38] )) - physical expressions: - id:354#4 cost=0 [0/0/0/] estRows=-1 children=[@3 ] (plan=PhysicalResultSink[355] ( outputExprs=[revenue#38] )) - enforcers: - chosen expression id: 354 - chosen properties: GATHER - stats - rows=1.0 - tupleSize=4.0 - width=1 - revenue#38 -> ndv=0.0204, min=2.000000(null), max=6.000000(null), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 4.046506802721089 ANY - id:354#4 cost=0 [0/0/0/] estRows=-1 children=[@3 ] (plan=PhysicalResultSink[355] ( outputExprs=[revenue#38] )) - [GATHER] - - 4.046506802721089 GATHER - id:354#4 cost=0 [0/0/0/] estRows=-1 children=[@3 ] (plan=PhysicalResultSink[355] ( outputExprs=[revenue#38] )) - [GATHER] - struct info map -StructInfoMap{ groupExpressionMap = {}, infoMap = {}} - - -Group[@0] - logical expressions: - id:337#0 cost=null estRows=2 children=[ ] (plan=LogicalOlapScan ( qualified=internal.regression_test_mv_p0_ssb_q_1_1.lineorder_flat, indexName=, selectedIndexId=346536, preAgg=ON )) - physical expressions: - id:383#0 cost=2 [2/0/0/] estRows=-1 children=[ ] (plan=PhysicalOlapScan[lineorder_flat]@0 ( stats=null )) - enforcers: - stats - rows=2.0 - tupleSize=170.0 - width=1 - LO_ORDERDATE#0 -> ndv=1.0000, min=19930101.000000(19930101), max=19930101.000000(19930101), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_ORDERKEY#1 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_LINENUMBER#2 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=1.000000 - LO_CUSTKEY#3 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_PARTKEY#4 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_SUPPKEY#5 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_ORDERPRIORITY#6 -> ndv=2.0000, min=13792273858822144.000000(1), max=14073748835532800.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=1.000000 - LO_SHIPPRIORITY#7 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=1.000000 - LO_QUANTITY#8 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=1.000000 - LO_EXTENDEDPRICE#9 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_ORDTOTALPRICE#10 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_DISCOUNT#11 -> ndv=2.0000, min=2.000000(2), max=100.000000(100), count=2.0000, numNulls=0.0000, avgSizeByte=1.000000 - LO_REVENUE#12 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_SUPPLYCOST#13 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_TAX#14 -> ndv=2.0000, min=1.000000(1), max=2.000000(2), count=2.0000, numNulls=0.0000, avgSizeByte=1.000000 - LO_COMMITDATE#15 -> ndv=1.0000, min=20230609000000.000000(2023-06-09), max=20230609000000.000000(2023-06-09), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - LO_SHIPMODE#16 -> ndv=1.0000, min=32484424388800356.000000(shipmode), max=32484424388800356.000000(shipmode), count=2.0000, numNulls=0.0000, avgSizeByte=8.000000 - C_NAME#17 -> ndv=1.0000, min=31069369912000512.000000(name), max=31069369912000512.000000(name), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - C_ADDRESS#18 -> ndv=1.0000, min=27413455319692148.000000(address), max=27413455319692148.000000(address), count=2.0000, numNulls=0.0000, avgSizeByte=7.000000 - C_CITY#19 -> ndv=1.0000, min=27981971661520896.000000(city), max=27981971661520896.000000(city), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - C_NATION#20 -> ndv=1.0000, min=31069400051183104.000000(nation), max=31069400051183104.000000(nation), count=2.0000, numNulls=0.0000, avgSizeByte=6.000000 - C_REGION#21 -> ndv=2.0000, min=18380833614807872.000000(AMERICA), max=32199642169961984.000000(region), count=2.0000, numNulls=0.0000, avgSizeByte=6.500000 - C_PHONE#22 -> ndv=1.0000, min=31640025194364928.000000(phone), max=31640025194364928.000000(phone), count=2.0000, numNulls=0.0000, avgSizeByte=5.000000 - C_MKTSEGMENT#23 -> ndv=1.0000, min=30798920357865324.000000(mktsegment), max=30798920357865324.000000(mktsegment), count=2.0000, numNulls=0.0000, avgSizeByte=10.000000 - S_NAME#24 -> ndv=1.0000, min=31069369912000512.000000(name), max=31069369912000512.000000(name), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - S_ADDRESS#25 -> ndv=1.0000, min=27413455319692148.000000(address), max=27413455319692148.000000(address), count=2.0000, numNulls=0.0000, avgSizeByte=7.000000 - S_CITY#26 -> ndv=1.0000, min=27981971661520896.000000(city), max=27981971661520896.000000(city), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - S_NATION#27 -> ndv=1.0000, min=31069400051183104.000000(nation), max=31069400051183104.000000(nation), count=2.0000, numNulls=0.0000, avgSizeByte=6.000000 - S_REGION#28 -> ndv=2.0000, min=18380833614807872.000000(AMERICA), max=32199642169961984.000000(region), count=2.0000, numNulls=0.0000, avgSizeByte=6.500000 - S_PHONE#29 -> ndv=1.0000, min=31640025194364928.000000(phone), max=31640025194364928.000000(phone), count=2.0000, numNulls=0.0000, avgSizeByte=5.000000 - P_NAME#30 -> ndv=1.0000, min=31069369912000512.000000(name), max=31069369912000512.000000(name), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - P_MFGR#31 -> ndv=2.0000, min=21750845341380864.000000(MFGR#1), max=30793366941728768.000000(mfgr), count=2.0000, numNulls=0.0000, avgSizeByte=5.000000 - P_CATEGORY#32 -> ndv=1.0000, min=27973175239733104.000000(category), max=27973175239733104.000000(category), count=2.0000, numNulls=0.0000, avgSizeByte=8.000000 - P_BRAND#33 -> ndv=1.0000, min=27710310507085824.000000(brand), max=27710310507085824.000000(brand), count=2.0000, numNulls=0.0000, avgSizeByte=5.000000 - P_COLOR#34 -> ndv=1.0000, min=27988534211248128.000000(color), max=27988534211248128.000000(color), count=2.0000, numNulls=0.0000, avgSizeByte=5.000000 - P_TYPE#35 -> ndv=1.0000, min=32784620936232960.000000(type), max=32784620936232960.000000(type), count=2.0000, numNulls=0.0000, avgSizeByte=4.000000 - P_SIZE#36 -> ndv=1.0000, min=4.000000(4), max=4.000000(4), count=2.0000, numNulls=0.0000, avgSizeByte=1.000000 - P_CONTAINER#37 -> ndv=1.0000, min=27988542883981680.000000(container), max=27988542883981680.000000(container), count=2.0000, numNulls=0.0000, avgSizeByte=9.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 2.0 ANY - id:383#0 cost=2 [2/0/0/] estRows=-1 children=[ ] (plan=PhysicalOlapScan[lineorder_flat]@0 ( stats=null )) - [] - - 2.0 DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([]) - id:383#0 cost=2 [2/0/0/] estRows=-1 children=[ ] (plan=PhysicalOlapScan[lineorder_flat]@0 ( stats=null )) - [] - struct info map -StructInfoMap{ groupExpressionMap = {{0}=id:337#0 cost=null estRows=2 children=[ ] (plan=LogicalOlapScan ( qualified=internal.regression_test_mv_p0_ssb_q_1_1.lineorder_flat, indexName=, selectedIndexId=346536, preAgg=ON )):[]}, infoMap = {}} - - -Group[@1] - logical expressions: - id:340#1 cost=null estRows=0.02 children=[@0 ] (plan=LogicalFilter[341] ( predicates=(((((LO_QUANTITY#8 < 25) AND (LO_ORDERDATE#0 >= 19930101)) AND (LO_DISCOUNT#11 >= 1)) AND (LO_ORDERDATE#0 <= 19931231)) AND (LO_DISCOUNT#11 <= 3)) )) - physical expressions: - id:380#1 cost=0 [0/0/0/] estRows=-1 children=[@0 ] (plan=PhysicalFilter[381]@1 ( stats=null, predicates=(((((LO_QUANTITY#8 < 25) AND (LO_ORDERDATE#0 >= 19930101)) AND (LO_DISCOUNT#11 >= 1)) AND (LO_ORDERDATE#0 <= 19931231)) AND (LO_DISCOUNT#11 <= 3)) )) - enforcers: - stats - rows=0.02040816326530612 - tupleSize=170.0 - width=1 - LO_ORDERDATE#0 -> ndv=0.0204, min=19930101.000000(19930101), max=19930101.000000(19930101), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_ORDERKEY#1 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_LINENUMBER#2 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=1.000000 - LO_CUSTKEY#3 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_PARTKEY#4 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_SUPPKEY#5 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_ORDERPRIORITY#6 -> ndv=0.0204, min=13792273858822144.000000(1), max=14073748835532800.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=1.000000 - LO_SHIPPRIORITY#7 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=1.000000 - LO_QUANTITY#8 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=1.000000 - LO_EXTENDEDPRICE#9 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_ORDTOTALPRICE#10 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_DISCOUNT#11 -> ndv=0.0204, min=2.000000(2), max=3.000000(3), count=0.0204, numNulls=0.0000, avgSizeByte=1.000000 - LO_REVENUE#12 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_SUPPLYCOST#13 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_TAX#14 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=1.000000 - LO_COMMITDATE#15 -> ndv=0.0204, min=20230609000000.000000(2023-06-09), max=20230609000000.000000(2023-06-09), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_SHIPMODE#16 -> ndv=0.0204, min=32484424388800356.000000(shipmode), max=32484424388800356.000000(shipmode), count=0.0204, numNulls=0.0000, avgSizeByte=8.000000 - C_NAME#17 -> ndv=0.0204, min=31069369912000512.000000(name), max=31069369912000512.000000(name), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - C_ADDRESS#18 -> ndv=0.0204, min=27413455319692148.000000(address), max=27413455319692148.000000(address), count=0.0204, numNulls=0.0000, avgSizeByte=7.000000 - C_CITY#19 -> ndv=0.0204, min=27981971661520896.000000(city), max=27981971661520896.000000(city), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - C_NATION#20 -> ndv=0.0204, min=31069400051183104.000000(nation), max=31069400051183104.000000(nation), count=0.0204, numNulls=0.0000, avgSizeByte=6.000000 - C_REGION#21 -> ndv=0.0204, min=18380833614807872.000000(AMERICA), max=32199642169961984.000000(region), count=0.0204, numNulls=0.0000, avgSizeByte=6.500000 - C_PHONE#22 -> ndv=0.0204, min=31640025194364928.000000(phone), max=31640025194364928.000000(phone), count=0.0204, numNulls=0.0000, avgSizeByte=5.000000 - C_MKTSEGMENT#23 -> ndv=0.0204, min=30798920357865324.000000(mktsegment), max=30798920357865324.000000(mktsegment), count=0.0204, numNulls=0.0000, avgSizeByte=10.000000 - S_NAME#24 -> ndv=0.0204, min=31069369912000512.000000(name), max=31069369912000512.000000(name), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - S_ADDRESS#25 -> ndv=0.0204, min=27413455319692148.000000(address), max=27413455319692148.000000(address), count=0.0204, numNulls=0.0000, avgSizeByte=7.000000 - S_CITY#26 -> ndv=0.0204, min=27981971661520896.000000(city), max=27981971661520896.000000(city), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - S_NATION#27 -> ndv=0.0204, min=31069400051183104.000000(nation), max=31069400051183104.000000(nation), count=0.0204, numNulls=0.0000, avgSizeByte=6.000000 - S_REGION#28 -> ndv=0.0204, min=18380833614807872.000000(AMERICA), max=32199642169961984.000000(region), count=0.0204, numNulls=0.0000, avgSizeByte=6.500000 - S_PHONE#29 -> ndv=0.0204, min=31640025194364928.000000(phone), max=31640025194364928.000000(phone), count=0.0204, numNulls=0.0000, avgSizeByte=5.000000 - P_NAME#30 -> ndv=0.0204, min=31069369912000512.000000(name), max=31069369912000512.000000(name), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - P_MFGR#31 -> ndv=0.0204, min=21750845341380864.000000(MFGR#1), max=30793366941728768.000000(mfgr), count=0.0204, numNulls=0.0000, avgSizeByte=5.000000 - P_CATEGORY#32 -> ndv=0.0204, min=27973175239733104.000000(category), max=27973175239733104.000000(category), count=0.0204, numNulls=0.0000, avgSizeByte=8.000000 - P_BRAND#33 -> ndv=0.0204, min=27710310507085824.000000(brand), max=27710310507085824.000000(brand), count=0.0204, numNulls=0.0000, avgSizeByte=5.000000 - P_COLOR#34 -> ndv=0.0204, min=27988534211248128.000000(color), max=27988534211248128.000000(color), count=0.0204, numNulls=0.0000, avgSizeByte=5.000000 - P_TYPE#35 -> ndv=0.0204, min=32784620936232960.000000(type), max=32784620936232960.000000(type), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - P_SIZE#36 -> ndv=0.0204, min=4.000000(4), max=4.000000(4), count=0.0204, numNulls=0.0000, avgSizeByte=1.000000 - P_CONTAINER#37 -> ndv=0.0204, min=27988542883981680.000000(container), max=27988542883981680.000000(container), count=0.0204, numNulls=0.0000, avgSizeByte=9.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 2.00058 ANY - id:380#1 cost=0 [0/0/0/] estRows=-1 children=[@0 ] (plan=PhysicalFilter[381]@1 ( stats=null, predicates=(((((LO_QUANTITY#8 < 25) AND (LO_ORDERDATE#0 >= 19930101)) AND (LO_DISCOUNT#11 >= 1)) AND (LO_ORDERDATE#0 <= 19931231)) AND (LO_DISCOUNT#11 <= 3)) )) - [DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([])] - - 2.00058 DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([]) - id:380#1 cost=0 [0/0/0/] estRows=-1 children=[@0 ] (plan=PhysicalFilter[381]@1 ( stats=null, predicates=(((((LO_QUANTITY#8 < 25) AND (LO_ORDERDATE#0 >= 19930101)) AND (LO_DISCOUNT#11 >= 1)) AND (LO_ORDERDATE#0 <= 19931231)) AND (LO_DISCOUNT#11 <= 3)) )) - [DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([])] - struct info map -StructInfoMap{ groupExpressionMap = {{0}=id:340#1 cost=null estRows=0.02 children=[@0 ] (plan=LogicalFilter[341] ( predicates=(((((LO_QUANTITY#8 < 25) AND (LO_ORDERDATE#0 >= 19930101)) AND (LO_DISCOUNT#11 >= 1)) AND (LO_ORDERDATE#0 <= 19931231)) AND (LO_DISCOUNT#11 <= 3)) )):[{0}]}, infoMap = {}} - - -Group[@2] - logical expressions: - id:343#2 cost=null estRows=0.02 children=[@1 ] (plan=LogicalProject[344] ( distinct=false, projects=[LO_EXTENDEDPRICE#9, LO_DISCOUNT#11], excepts=[] )) - physical expressions: - id:375#2 cost=1 [1/0/0/] estRows=-1 children=[@1 ] (plan=PhysicalProject[376]@2 ( stats=null, projects=[LO_EXTENDEDPRICE#9, LO_DISCOUNT#11] )) - enforcers: - stats - rows=0.02040816326530612 - tupleSize=5.0 - width=1 - LO_EXTENDEDPRICE#9 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - LO_DISCOUNT#11 -> ndv=0.0204, min=2.000000(2), max=3.000000(3), count=0.0204, numNulls=0.0000, avgSizeByte=1.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 3.00058 ANY - id:375#2 cost=1 [1/0/0/] estRows=-1 children=[@1 ] (plan=PhysicalProject[376]@2 ( stats=null, projects=[LO_EXTENDEDPRICE#9, LO_DISCOUNT#11] )) - [DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([])] - - 3.00058 DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([]) - id:375#2 cost=1 [1/0/0/] estRows=-1 children=[@1 ] (plan=PhysicalProject[376]@2 ( stats=null, projects=[LO_EXTENDEDPRICE#9, LO_DISCOUNT#11] )) - [DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([])] - struct info map -StructInfoMap{ groupExpressionMap = {{0}=id:343#2 cost=null estRows=0.02 children=[@1 ] (plan=LogicalProject[344] ( distinct=false, projects=[LO_EXTENDEDPRICE#9, LO_DISCOUNT#11], excepts=[] )):[{0}]}, infoMap = {}} - - -Group[@3] - logical expressions: - id:346#3 cost=null estRows=1 children=[@2 ] (plan=LogicalAggregate[347] ( groupByExpr=[], outputExpr=[sum((LO_EXTENDEDPRICE#9 * LO_DISCOUNT#11)) AS `revenue`#38], hasRepeat=false )) - id:541#3 cost=null estRows=1 children=[@8 ] (plan=LogicalProject[542] ( distinct=false, projects=[sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#82 AS `revenue`#38], excepts=[] )) - physical expressions: - id:370#3 cost=2 [1/1/0/] estRows=1 children=[@5 ] (plan=PhysicalHashAggregate[371]@3 ( aggPhase=GLOBAL, aggMode=BUFFER_TO_RESULT, maybeUseStreaming=false, groupByExpr=[], outputExpr=[sum(partial_sum((LO_EXTENDEDPRICE * LO_DISCOUNT))#81) AS `revenue`#38], partitionExpr=Optional[[]], requireProperties=[GATHER], topnOpt=false, stats=null )) - id:546#3 cost=1 [1/0/0/] estRows=-1 children=[@8 ] (plan=PhysicalProject[547]@3 ( stats=null, projects=[sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#82 AS `revenue`#38] )) - enforcers: - chosen expression id: 546 - chosen properties: GATHER - stats - rows=1.0 - tupleSize=4.0 - width=1 - revenue#38 -> ndv=0.0204, min=2.000000(null), max=6.000000(null), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 4.046506802721089 ANY - id:546#3 cost=1 [1/0/0/] estRows=-1 children=[@8 ] (plan=PhysicalProject[547]@3 ( stats=null, projects=[sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#82 AS `revenue`#38] )) - [GATHER] - - 4.046506802721089 GATHER - id:546#3 cost=1 [1/0/0/] estRows=-1 children=[@8 ] (plan=PhysicalProject[547]@3 ( stats=null, projects=[sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#82 AS `revenue`#38] )) - [GATHER] - struct info map -StructInfoMap{ groupExpressionMap = {{0}=id:346#3 cost=null estRows=1 children=[@2 ] (plan=LogicalAggregate[347] ( groupByExpr=[], outputExpr=[sum((LO_EXTENDEDPRICE#9 * LO_DISCOUNT#11)) AS `revenue`#38], hasRepeat=false )):[{0}]}, infoMap = {{0}=StructInfo{ originalPlanId = ObjectId#346, relations = [LogicalOlapScan ( qualified=internal.regression_test_mv_p0_ssb_q_1_1.lineorder_flat, indexName=, selectedIndexId=346536, preAgg=ON )]}}} - - -Group[@4] - logical expressions: - id:349#4 cost=null estRows=1 children=[@3 ] (plan=LogicalResultSink[350] ( outputExprs=[revenue#38] )) - physical expressions: - id:354#4 cost=0 [0/0/0/] estRows=-1 children=[@3 ] (plan=PhysicalResultSink[355] ( outputExprs=[revenue#38] )) - enforcers: - chosen expression id: 354 - chosen properties: GATHER - stats - rows=1.0 - tupleSize=4.0 - width=1 - revenue#38 -> ndv=0.0204, min=2.000000(null), max=6.000000(null), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 4.046506802721089 ANY - id:354#4 cost=0 [0/0/0/] estRows=-1 children=[@3 ] (plan=PhysicalResultSink[355] ( outputExprs=[revenue#38] )) - [GATHER] - - 4.046506802721089 GATHER - id:354#4 cost=0 [0/0/0/] estRows=-1 children=[@3 ] (plan=PhysicalResultSink[355] ( outputExprs=[revenue#38] )) - [GATHER] - struct info map -StructInfoMap{ groupExpressionMap = {}, infoMap = {}} - - -Group[@5] - logical expressions: - physical expressions: - id:367#5 cost=0 [0/0/0/] estRows=1 children=[@2 ] (plan=PhysicalHashAggregate[368]@5 ( aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=false, groupByExpr=[], outputExpr=[partial_sum((LO_EXTENDEDPRICE#9 * LO_DISCOUNT#11)) AS `partial_sum((LO_EXTENDEDPRICE * LO_DISCOUNT))`#81], partitionExpr=Optional[[]], requireProperties=[ANY], topnOpt=false, stats=null )) - enforcers: - id:469#5 cost=0 [0/0/0/] estRows=1 children=[@5 ] (plan=PhysicalDistribute[470]@5 ( stats=null, distributionSpec=DistributionSpecGather )) - stats - rows=1.0 - tupleSize=1.0 - width=1 - partial_sum((LO_EXTENDEDPRICE * LO_DISCOUNT))#81 -> ndv=0.0204, min=2.000000(null), max=6.000000(null), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 3.0113827210884354 ANY - id:367#5 cost=0 [0/0/0/] estRows=1 children=[@2 ] (plan=PhysicalHashAggregate[368]@5 ( aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=false, groupByExpr=[], outputExpr=[partial_sum((LO_EXTENDEDPRICE#9 * LO_DISCOUNT#11)) AS `partial_sum((LO_EXTENDEDPRICE * LO_DISCOUNT))`#81], partitionExpr=Optional[[]], requireProperties=[ANY], topnOpt=false, stats=null )) - [DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([])] - - 3.0113827210884354 DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([]) - id:367#5 cost=0 [0/0/0/] estRows=1 children=[@2 ] (plan=PhysicalHashAggregate[368]@5 ( aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=false, groupByExpr=[], outputExpr=[partial_sum((LO_EXTENDEDPRICE#9 * LO_DISCOUNT#11)) AS `partial_sum((LO_EXTENDEDPRICE * LO_DISCOUNT))`#81], partitionExpr=Optional[[]], requireProperties=[ANY], topnOpt=false, stats=null )) - [DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([])] - - 3.0188827210884353 GATHER - id:469#5 cost=0 [0/0/0/] estRows=1 children=[@5 ] (plan=PhysicalDistribute[470]@5 ( stats=null, distributionSpec=DistributionSpecGather )) - [DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=NATURAL, tableId=346535, selectedIndexId=346536, partitionIds=[346534], equivalenceExprIds=[[1]], exprIdToEquivalenceSet={1=0} ) Order: ([])] - struct info map -StructInfoMap{ groupExpressionMap = {}, infoMap = {}} - - -Group[@6] - logical expressions: - id:532#6 cost=null estRows=0.02 children=[ ] (plan=LogicalOlapScan ( qualified=regression_test_mv_p0_ssb_q_1_1.lineorder_flat, indexName=lineorder_q_1_1, selectedIndexId=346636, preAgg=ON )) - physical expressions: - id:571#6 cost=0 [0/0/0/] estRows=-1 children=[ ] (plan=PhysicalOlapScan[lineorder_flat]@6 ( stats=null )) - enforcers: - chosen expression id: 571 - chosen properties: DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([]) - stats - rows=0.02040816326530612 - tupleSize=8.0 - width=1 - mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80 -> ndv=0.0204, min=2.000000(null), max=6.000000(null), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - mv_LO_ORDERKEY#79 -> ndv=0.0204, min=1.000000(1), max=2.000000(2), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 0.01020408163265306 ANY - id:571#6 cost=0 [0/0/0/] estRows=-1 children=[ ] (plan=PhysicalOlapScan[lineorder_flat]@6 ( stats=null )) - [] - - 0.01020408163265306 DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([]) - id:571#6 cost=0 [0/0/0/] estRows=-1 children=[ ] (plan=PhysicalOlapScan[lineorder_flat]@6 ( stats=null )) - [] - struct info map -StructInfoMap{ groupExpressionMap = {{0}=id:532#6 cost=null estRows=0.02 children=[ ] (plan=LogicalOlapScan ( qualified=regression_test_mv_p0_ssb_q_1_1.lineorder_flat, indexName=lineorder_q_1_1, selectedIndexId=346636, preAgg=ON )):[]}, infoMap = {}} - - -Group[@7] - logical expressions: - id:535#7 cost=null estRows=0.02 children=[@6 ] (plan=LogicalProject[536] ( distinct=false, projects=[mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80], excepts=[] )) - physical expressions: - id:568#7 cost=1 [1/0/0/] estRows=-1 children=[@6 ] (plan=PhysicalProject[569]@7 ( stats=null, projects=[mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80] )) - enforcers: - chosen expression id: 568 - chosen properties: DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([]) - stats - rows=0.02040816326530612 - tupleSize=4.0 - width=1 - mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80 -> ndv=0.0204, min=2.000000(null), max=6.000000(null), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 1.010204081632653 ANY - id:568#7 cost=1 [1/0/0/] estRows=-1 children=[@6 ] (plan=PhysicalProject[569]@7 ( stats=null, projects=[mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80] )) - [DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([])] - - 1.010204081632653 DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([]) - id:568#7 cost=1 [1/0/0/] estRows=-1 children=[@6 ] (plan=PhysicalProject[569]@7 ( stats=null, projects=[mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80] )) - [DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([])] - struct info map -StructInfoMap{ groupExpressionMap = {{0}=id:535#7 cost=null estRows=0.02 children=[@6 ] (plan=LogicalProject[536] ( distinct=false, projects=[mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80], excepts=[] )):[{0}]}, infoMap = {}} - - -Group[@8] - logical expressions: - id:538#8 cost=null estRows=1 children=[@7 ] (plan=LogicalAggregate[539] ( groupByExpr=[], outputExpr=[sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80) AS `sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#82], hasRepeat=false )) - physical expressions: - id:562#8 cost=2 [1/1/0/] estRows=1 children=[@9 ] (plan=PhysicalHashAggregate[563]@8 ( aggPhase=GLOBAL, aggMode=BUFFER_TO_RESULT, maybeUseStreaming=false, groupByExpr=[], outputExpr=[sum(partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#85) AS `sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#82], partitionExpr=Optional[[]], requireProperties=[GATHER], topnOpt=false, stats=null )) - enforcers: - chosen expression id: 562 - chosen properties: GATHER - stats - rows=1.0 - tupleSize=4.0 - width=1 - sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#82 -> ndv=0.0204, min=2.000000(null), max=6.000000(null), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 3.046506802721088 ANY - id:562#8 cost=2 [1/1/0/] estRows=1 children=[@9 ] (plan=PhysicalHashAggregate[563]@8 ( aggPhase=GLOBAL, aggMode=BUFFER_TO_RESULT, maybeUseStreaming=false, groupByExpr=[], outputExpr=[sum(partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#85) AS `sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#82], partitionExpr=Optional[[]], requireProperties=[GATHER], topnOpt=false, stats=null )) - [GATHER] - - 3.046506802721088 GATHER - id:562#8 cost=2 [1/1/0/] estRows=1 children=[@9 ] (plan=PhysicalHashAggregate[563]@8 ( aggPhase=GLOBAL, aggMode=BUFFER_TO_RESULT, maybeUseStreaming=false, groupByExpr=[], outputExpr=[sum(partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#85) AS `sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#82], partitionExpr=Optional[[]], requireProperties=[GATHER], topnOpt=false, stats=null )) - [GATHER] - struct info map -StructInfoMap{ groupExpressionMap = {{0}=id:538#8 cost=null estRows=1 children=[@7 ] (plan=LogicalAggregate[539] ( groupByExpr=[], outputExpr=[sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80) AS `sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#82], hasRepeat=false )):[{0}]}, infoMap = {{0}=StructInfo{ originalPlanId = ObjectId#538, relations = [LogicalOlapScan ( qualified=regression_test_mv_p0_ssb_q_1_1.lineorder_flat, indexName=lineorder_q_1_1, selectedIndexId=346636, preAgg=ON )]}}} - - -Group[@9] - logical expressions: - physical expressions: - id:559#9 cost=0 [0/0/0/] estRows=1 children=[@7 ] (plan=PhysicalHashAggregate[560]@9 ( aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=false, groupByExpr=[], outputExpr=[partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80) AS `partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#85], partitionExpr=Optional[[]], requireProperties=[ANY], topnOpt=false, stats=null )) - enforcers: - id:595#9 cost=0 [0/0/0/] estRows=1 children=[@9 ] (plan=PhysicalDistribute[596]@9 ( stats=null, distributionSpec=DistributionSpecGather )) - chosen enforcer(id, requiredProperties): - (0)595, GATHER - chosen expression id: 559 - chosen properties: DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([]) - stats - rows=1.0 - tupleSize=1.0 - width=1 - partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#85 -> ndv=0.0204, min=2.000000(null), max=6.000000(null), count=0.0204, numNulls=0.0000, avgSizeByte=4.000000 - lowest Plan(cost, properties, plan, childrenRequires) - - 1.0190068027210881 ANY - id:559#9 cost=0 [0/0/0/] estRows=1 children=[@7 ] (plan=PhysicalHashAggregate[560]@9 ( aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=false, groupByExpr=[], outputExpr=[partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80) AS `partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#85], partitionExpr=Optional[[]], requireProperties=[ANY], topnOpt=false, stats=null )) - [DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([])] - - 1.0190068027210881 DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([]) - id:559#9 cost=0 [0/0/0/] estRows=1 children=[@7 ] (plan=PhysicalHashAggregate[560]@9 ( aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=false, groupByExpr=[], outputExpr=[partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80) AS `partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#85], partitionExpr=Optional[[]], requireProperties=[ANY], topnOpt=false, stats=null )) - [DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([])] - - 1.0265068027210882 GATHER - id:595#9 cost=0 [0/0/0/] estRows=1 children=[@9 ] (plan=PhysicalDistribute[596]@9 ( stats=null, distributionSpec=DistributionSpecGather )) - [DistributionSpecHash ( orderedShuffledColumns=[79], shuffleType=NATURAL, tableId=346535, selectedIndexId=346636, partitionIds=[346534], equivalenceExprIds=[[79]], exprIdToEquivalenceSet={79=0} ) Order: ([])] - struct info map -StructInfoMap{ groupExpressionMap = {}, infoMap = {}} - - -========== OPTIMIZED PLAN ========== -PhysicalResultSink[657] ( outputExprs=[revenue#38] ) -+--PhysicalProject[654]@3 ( stats=1, projects=[sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#82 AS `revenue`#38] ) - +--PhysicalHashAggregate[651]@8 ( aggPhase=GLOBAL, aggMode=BUFFER_TO_RESULT, maybeUseStreaming=false, groupByExpr=[], outputExpr=[sum(partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))#85) AS `sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#82], partitionExpr=Optional[[]], requireProperties=[GATHER], topnOpt=false, stats=1 ) - +--PhysicalDistribute[648]@9 ( stats=1, distributionSpec=DistributionSpecGather ) - +--PhysicalHashAggregate[645]@9 ( aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=false, groupByExpr=[], outputExpr=[partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80) AS `partial_sum(mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`))`#85], partitionExpr=Optional[[]], requireProperties=[ANY], topnOpt=false, stats=1 ) - +--PhysicalProject[642]@7 ( stats=0.02, projects=[mva_SUM__(`LO_EXTENDEDPRICE` * `LO_DISCOUNT`)#80] ) - +--PhysicalOlapScan[lineorder_flat]@6 ( stats=0.02 ) - -========== MATERIALIZATIONS ========== -materializationContexts: - -MaterializationContext[[internal, regression_test_mv_p0_ssb_q_1_1, lineorder_flat, lineorder_q_1_1]] ( rewriteSuccess=true, failReason=[ - -] ) - -- !select -- 4 diff --git a/regression-test/suites/mv_p0/ssb/q_1_1/q_1_1.groovy b/regression-test/suites/mv_p0/ssb/q_1_1/q_1_1.groovy index 34fb07d4f14213e..30959801e72af19 100644 --- a/regression-test/suites/mv_p0/ssb/q_1_1/q_1_1.groovy +++ b/regression-test/suites/mv_p0/ssb/q_1_1/q_1_1.groovy @@ -72,8 +72,13 @@ suite ("mv_ssb_q_1_1") { ); """ + // Add more data when insert into firstly to make sure use mv easier sql """INSERT INTO lineorder_flat (LO_ORDERDATE, LO_ORDERKEY, LO_LINENUMBER, LO_CUSTKEY, LO_PARTKEY, LO_SUPPKEY, LO_ORDERPRIORITY, LO_SHIPPRIORITY, LO_QUANTITY, LO_EXTENDEDPRICE, LO_ORDTOTALPRICE, LO_DISCOUNT, LO_REVENUE, LO_SUPPLYCOST, LO_TAX, LO_COMMITDATE, LO_SHIPMODE, C_NAME, C_ADDRESS, C_CITY, C_NATION, C_REGION, C_PHONE, C_MKTSEGMENT, S_NAME, S_ADDRESS, S_CITY, S_NATION, S_REGION, S_PHONE, P_NAME, P_MFGR, P_CATEGORY, P_BRAND, P_COLOR,P_TYPE,P_SIZE,P_CONTAINER) - VALUES (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container');""" + VALUES (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), + (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), + (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), + (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container'), + (19930101 , 1 , 1 , 1 , 1 , 1 , '1' , 1 , 1 , 1 , 1 , 100 , 1 , 1 , 1 , '2023-06-09' , 'shipmode' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' , 'phone' , 'mktsegment' , 'name' , 'address' , 'city' , 'nation' , 'AMERICA' ,'phone', 'name', 'MFGR#1', 'category', 'brand', 'color', 'type', 4 ,'container');""" createMV ("""create materialized view lineorder_q_1_1 as SELECT LO_ORDERKEY, SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue From 601d8016822602f5c39d75e8e2fec4e7b41b609e Mon Sep 17 00:00:00 2001 From: Jibing-Li <64681310+Jibing-Li@users.noreply.github.com> Date: Fri, 1 Nov 2024 14:26:42 +0800 Subject: [PATCH 32/82] [improvement](statistics)External table getRowCount return -1 when row count is not available or row count is 0. (#43009) External table getRowCount return -1 when row count is not available or row count is 0. So the behavior of external table could match with internal olap table. --- .../org/apache/doris/catalog/OlapTable.java | 8 ++--- .../java/org/apache/doris/catalog/Table.java | 2 +- .../org/apache/doris/catalog/TableIf.java | 2 ++ .../datasource/ExternalRowCountCache.java | 6 ++-- .../doris/datasource/ExternalTable.java | 4 +-- .../datasource/hive/HMSExternalTable.java | 18 +++++------ .../iceberg/IcebergExternalTable.java | 3 +- .../datasource/iceberg/IcebergUtils.java | 5 +-- .../paimon/PaimonExternalTable.java | 8 +++-- .../statistics/StatisticsAutoCollector.java | 2 +- .../doris/statistics/util/StatisticsUtil.java | 8 ++--- .../datasource/ExternalRowCountCacheTest.java | 32 +++++++++++++++++-- 12 files changed, 66 insertions(+), 32 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 999e0c43995f00a..b0d27ac7b5c7d99 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -139,8 +139,6 @@ public enum OlapTableState { WAITING_STABLE } - public static long ROW_COUNT_BEFORE_REPORT = -1; - @SerializedName(value = "tst", alternate = {"state"}) private volatile OlapTableState state; @@ -1618,12 +1616,12 @@ public long getRowCountForIndex(long indexId, boolean strict) { if (index == null) { LOG.warn("Index {} not exist in partition {}, table {}, {}", indexId, entry.getValue().getName(), id, name); - return ROW_COUNT_BEFORE_REPORT; + return UNKNOWN_ROW_COUNT; } if (strict && !index.getRowCountReported()) { - return ROW_COUNT_BEFORE_REPORT; + return UNKNOWN_ROW_COUNT; } - rowCount += index.getRowCount() == -1 ? 0 : index.getRowCount(); + rowCount += index.getRowCount() == UNKNOWN_ROW_COUNT ? 0 : index.getRowCount(); } return rowCount; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java index d98bba5edaca841..d85d98a8ea550f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java @@ -623,7 +623,7 @@ public List getChunkSizes() { @Override public long fetchRowCount() { - return 0; + return UNKNOWN_ROW_COUNT; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index 3a688a7b59d17ad..8f6e924f44a54d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -55,6 +55,8 @@ public interface TableIf { Logger LOG = LogManager.getLogger(TableIf.class); + long UNKNOWN_ROW_COUNT = -1; + default void readLock() { } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalRowCountCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalRowCountCache.java index 075091e682d7224..fc955c4964a30c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalRowCountCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalRowCountCache.java @@ -102,7 +102,7 @@ protected Optional doLoad(RowCountKey rowCountKey) { } /** - * Get cached row count for the given table. Return 0 if cached not loaded or table not exists. + * Get cached row count for the given table. Return -1 if cached not loaded or table not exists. * Cached will be loaded async. * @return Cached row count or -1 if not exist */ @@ -111,13 +111,13 @@ public long getCachedRowCount(long catalogId, long dbId, long tableId) { try { CompletableFuture> f = rowCountCache.get(key); if (f.isDone()) { - return f.get().orElse(-1L); + return f.get().orElse(TableIf.UNKNOWN_ROW_COUNT); } LOG.info("Row count for table {}.{}.{} is still processing.", catalogId, dbId, tableId); } catch (Exception e) { LOG.warn("Unexpected exception while returning row count", e); } - return -1; + return TableIf.UNKNOWN_ROW_COUNT; } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java index 5c57c13b4b85ad9..1eadb46fe82eedd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalTable.java @@ -195,7 +195,7 @@ public long getRowCount() { makeSureInitialized(); } catch (Exception e) { LOG.warn("Failed to initialize table {}.{}.{}", catalog.getName(), dbName, name, e); - return -1; + return TableIf.UNKNOWN_ROW_COUNT; } // All external table should get external row count from cache. return Env.getCurrentEnv().getExtMetaCacheMgr().getRowCountCache().getCachedRowCount(catalog.getId(), dbId, id); @@ -221,7 +221,7 @@ public long getCachedRowCount() { * This is called by ExternalRowCountCache to load row count cache. */ public long fetchRowCount() { - return -1; + return UNKNOWN_ROW_COUNT; } @Override 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 f72421da8a11343..b48b47acf1378e5 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 @@ -337,7 +337,7 @@ public long getCreateTime() { } private long getRowCountFromExternalSource() { - long rowCount = -1; + long rowCount = UNKNOWN_ROW_COUNT; switch (dlaType) { case HIVE: rowCount = StatisticsUtil.getHiveRowCount(this); @@ -350,7 +350,7 @@ private long getRowCountFromExternalSource() { LOG.debug("getRowCount for dlaType {} is not supported.", dlaType); } } - return rowCount; + return rowCount > 0 ? rowCount : UNKNOWN_ROW_COUNT; } @Override @@ -524,7 +524,7 @@ public long fetchRowCount() { // Get row count from hive metastore property. long rowCount = getRowCountFromExternalSource(); // Only hive table supports estimate row count by listing file. - if (rowCount == -1 && dlaType.equals(DLAType.HIVE)) { + if (rowCount == UNKNOWN_ROW_COUNT && dlaType.equals(DLAType.HIVE)) { LOG.info("Will estimate row count for table {} from file list.", name); rowCount = getRowCountFromFileList(); } @@ -834,11 +834,11 @@ public boolean isPartitionColumnAllowNull() { */ private long getRowCountFromFileList() { if (!GlobalVariable.enable_get_row_count_from_file_list) { - return -1; + return UNKNOWN_ROW_COUNT; } if (isView()) { - LOG.info("Table {} is view, return 0.", name); - return 0; + LOG.info("Table {} is view, return -1.", name); + return UNKNOWN_ROW_COUNT; } HiveMetaStoreCache.HivePartitionValues partitionValues = getAllPartitionValues(); @@ -865,8 +865,8 @@ private long getRowCountFromFileList() { estimatedRowSize += column.getDataType().getSlotSize(); } if (estimatedRowSize == 0) { - LOG.warn("Table {} estimated size is 0, return 0.", name); - return 0; + LOG.warn("Table {} estimated size is 0, return -1.", name); + return UNKNOWN_ROW_COUNT; } int totalPartitionSize = partitionValues == null ? 1 : partitionValues.getIdToPartitionItem().size(); @@ -878,7 +878,7 @@ private long getRowCountFromFileList() { long rows = totalSize / estimatedRowSize; LOG.info("Table {} rows {}, total size is {}, estimatedRowSize is {}", name, rows, totalSize, estimatedRowSize); - return rows; + return rows > 0 ? rows : UNKNOWN_ROW_COUNT; } // Get all partition values from cache. diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java index d4361a47797a2e1..feded88ea326f03 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java @@ -83,7 +83,8 @@ public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { @Override public long fetchRowCount() { makeSureInitialized(); - return IcebergUtils.getIcebergRowCount(getCatalog(), getDbName(), getName()); + long rowCount = IcebergUtils.getIcebergRowCount(getCatalog(), getDbName(), getName()); + return rowCount > 0 ? rowCount : UNKNOWN_ROW_COUNT; } public Table getIcebergTable() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java index 7ae600756f17a5a..ba6d628e492c209 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergUtils.java @@ -41,6 +41,7 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.StructField; import org.apache.doris.catalog.StructType; +import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.Type; import org.apache.doris.common.UserException; import org.apache.doris.common.info.SimpleTableInfo; @@ -604,9 +605,9 @@ public static long getIcebergRowCount(ExternalCatalog catalog, String dbName, St .getIcebergTable(catalog, dbName, tbName); Snapshot snapshot = icebergTable.currentSnapshot(); if (snapshot == null) { - LOG.info("Iceberg table {}.{}.{} is empty, return row count 0.", catalog.getName(), dbName, tbName); + LOG.info("Iceberg table {}.{}.{} is empty, return -1.", catalog.getName(), dbName, tbName); // empty table - return 0; + return TableIf.UNKNOWN_ROW_COUNT; } Map summary = snapshot.summary(); long rows = Long.parseLong(summary.get(TOTAL_RECORDS)) - Long.parseLong(summary.get(TOTAL_POSITION_DELETES)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java index 4b364ef45ca321d..c9eaf1b7df32ef1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/paimon/PaimonExternalTable.java @@ -193,12 +193,16 @@ public long fetchRowCount() { Table paimonTable = schemaCacheValue.map(value -> ((PaimonSchemaCacheValue) value).getPaimonTable()) .orElse(null); if (paimonTable == null) { - return -1; + LOG.info("Paimon table {} is null.", name); + return UNKNOWN_ROW_COUNT; } List splits = paimonTable.newReadBuilder().newScan().plan().splits(); for (Split split : splits) { rowCount += split.rowCount(); } - return rowCount; + if (rowCount == 0) { + LOG.info("Paimon table {} row count is 0, return -1", name); + } + return rowCount > 0 ? rowCount : UNKNOWN_ROW_COUNT; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java index 9ba52169605b435..28c2bd95c968803 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsAutoCollector.java @@ -208,7 +208,7 @@ protected AnalysisInfo createAnalyzeJobForTbl( } if (table instanceof OlapTable && analysisMethod.equals(AnalysisMethod.SAMPLE)) { OlapTable ot = (OlapTable) table; - if (ot.getRowCountForIndex(ot.getBaseIndexId(), true) == OlapTable.ROW_COUNT_BEFORE_REPORT) { + if (ot.getRowCountForIndex(ot.getBaseIndexId(), true) == TableIf.UNKNOWN_ROW_COUNT) { LOG.info("Table {} row count is not fully reported, skip auto analyzing this time.", ot.getName()); return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java index b0fc3b9c1cfab14..a9c1612eb48b9c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/util/StatisticsUtil.java @@ -619,19 +619,19 @@ public static int getTableHealth(long totalRows, long updatedRows) { public static long getHiveRowCount(HMSExternalTable table) { Map parameters = table.getRemoteTable().getParameters(); if (parameters == null) { - return -1; + return TableIf.UNKNOWN_ROW_COUNT; } // Table parameters contains row count, simply get and return it. if (parameters.containsKey(NUM_ROWS)) { long rows = Long.parseLong(parameters.get(NUM_ROWS)); // Sometimes, the NUM_ROWS in hms is 0 but actually is not. Need to check TOTAL_SIZE if NUM_ROWS is 0. - if (rows != 0) { + if (rows > 0) { LOG.info("Get row count {} for hive table {} in table parameters.", rows, table.getName()); return rows; } } if (!parameters.containsKey(TOTAL_SIZE)) { - return -1; + return TableIf.UNKNOWN_ROW_COUNT; } // Table parameters doesn't contain row count but contain total size. Estimate row count : totalSize/rowSize long totalSize = Long.parseLong(parameters.get(TOTAL_SIZE)); @@ -641,7 +641,7 @@ public static long getHiveRowCount(HMSExternalTable table) { } if (estimatedRowSize == 0) { LOG.warn("Hive table {} estimated row size is invalid {}", table.getName(), estimatedRowSize); - return -1; + return TableIf.UNKNOWN_ROW_COUNT; } long rows = totalSize / estimatedRowSize; LOG.info("Get row count {} for hive table {} by total size {} and row size {}", diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalRowCountCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalRowCountCacheTest.java index 10b6b01527cb98b..81605f93dcd1c18 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalRowCountCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/ExternalRowCountCacheTest.java @@ -17,6 +17,7 @@ package org.apache.doris.datasource; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.ThreadPoolManager; import mockit.Mock; @@ -44,7 +45,7 @@ protected Optional doLoad(ExternalRowCountCache.RowCountKey rowCountKey) { }; ExternalRowCountCache cache = new ExternalRowCountCache(executor); long cachedRowCount = cache.getCachedRowCount(1, 1, 1); - Assertions.assertEquals(-1, cachedRowCount); + Assertions.assertEquals(TableIf.UNKNOWN_ROW_COUNT, cachedRowCount); for (int i = 0; i < 60; i++) { if (counter.get() == 1) { break; @@ -63,12 +64,39 @@ protected Optional doLoad(ExternalRowCountCache.RowCountKey rowCountKey) { cache.getCachedRowCount(1, 1, 1); for (int i = 0; i < 60; i++) { cachedRowCount = cache.getCachedRowCount(1, 1, 1); - if (cachedRowCount != -1) { + if (cachedRowCount != TableIf.UNKNOWN_ROW_COUNT) { Assertions.assertEquals(100, cachedRowCount); break; } Thread.sleep(1000); } + cachedRowCount = cache.getCachedRowCount(1, 1, 1); + Assertions.assertEquals(100, cachedRowCount); Assertions.assertEquals(2, counter.get()); + + new MockUp() { + @Mock + protected Optional doLoad(ExternalRowCountCache.RowCountKey rowCountKey) { + counter.incrementAndGet(); + try { + Thread.sleep(1000000); + } catch (InterruptedException e) { + e.printStackTrace(); + } + return Optional.of(100L); + } + }; + cachedRowCount = cache.getCachedRowCount(2, 2, 2); + Assertions.assertEquals(TableIf.UNKNOWN_ROW_COUNT, cachedRowCount); + Thread.sleep(1000); + cachedRowCount = cache.getCachedRowCount(2, 2, 2); + Assertions.assertEquals(TableIf.UNKNOWN_ROW_COUNT, cachedRowCount); + for (int i = 0; i < 60; i++) { + if (counter.get() == 3) { + break; + } + Thread.sleep(1000); + } + Assertions.assertEquals(3, counter.get()); } } From 866c4fdd9bb038d93dacac9f3ecca096c1d637ae Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Fri, 1 Nov 2024 14:52:02 +0800 Subject: [PATCH 33/82] [Opt](Variant) merge schema in sync_rowsets to prevents from CPU overhead each time describe table (#42856) Should prevent from merge schema each time calling `merged_tablet_schema`. So this pr put the merge logic in `sync_rowsets` stage. --- be/src/cloud/cloud_tablet.cpp | 45 ++++++++++++++++++++++------- be/src/cloud/cloud_tablet.h | 6 ++++ be/src/service/internal_service.cpp | 5 +++- 3 files changed, 45 insertions(+), 11 deletions(-) diff --git a/be/src/cloud/cloud_tablet.cpp b/be/src/cloud/cloud_tablet.cpp index d3b131d055d35cb..54ea450f2049040 100644 --- a/be/src/cloud/cloud_tablet.cpp +++ b/be/src/cloud/cloud_tablet.cpp @@ -108,6 +108,36 @@ Status CloudTablet::capture_rs_readers(const Version& spec_version, return capture_rs_readers_unlocked(version_path, rs_splits); } +Status CloudTablet::merge_rowsets_schema() { + // Find the rowset with the max version + auto max_version_rowset = + std::max_element( + _rs_version_map.begin(), _rs_version_map.end(), + [](const auto& a, const auto& b) { + return !a.second->tablet_schema() + ? true + : (!b.second->tablet_schema() + ? false + : a.second->tablet_schema()->schema_version() < + b.second->tablet_schema() + ->schema_version()); + }) + ->second; + TabletSchemaSPtr max_version_schema = max_version_rowset->tablet_schema(); + // If the schema has variant columns, perform a merge to create a wide tablet schema + if (max_version_schema->num_variant_columns() > 0) { + std::vector schemas; + std::transform(_rs_version_map.begin(), _rs_version_map.end(), std::back_inserter(schemas), + [](const auto& rs_meta) { return rs_meta.second->tablet_schema(); }); + // Merge the collected schemas to obtain the least common schema + RETURN_IF_ERROR(vectorized::schema_util::get_least_common_schema(schemas, nullptr, + max_version_schema)); + VLOG_DEBUG << "dump schema: " << max_version_schema->dump_full_schema(); + _merged_tablet_schema = max_version_schema; + } + return Status::OK(); +} + // There are only two tablet_states RUNNING and NOT_READY in cloud mode // This function will erase the tablet from `CloudTabletMgr` when it can't find this tablet in MS. Status CloudTablet::sync_rowsets(int64_t query_version, bool warmup_delta_data) { @@ -133,6 +163,10 @@ Status CloudTablet::sync_rowsets(int64_t query_version, bool warmup_delta_data) if (st.is()) { clear_cache(); } + + // Merge all rowset schemas within a CloudTablet + RETURN_IF_ERROR(merge_rowsets_schema()); + return st; } @@ -188,16 +222,7 @@ Status CloudTablet::sync_if_not_running() { } TabletSchemaSPtr CloudTablet::merged_tablet_schema() const { - std::shared_lock rdlock(_meta_lock); - TabletSchemaSPtr target_schema; - std::vector schemas; - for (const auto& [_, rowset] : _rs_version_map) { - schemas.push_back(rowset->tablet_schema()); - } - // get the max version schema and merge all schema - static_cast( - vectorized::schema_util::get_least_common_schema(schemas, nullptr, target_schema)); - return target_schema; + return _merged_tablet_schema; } void CloudTablet::add_rowsets(std::vector to_add, bool version_overlap, diff --git a/be/src/cloud/cloud_tablet.h b/be/src/cloud/cloud_tablet.h index 53747dc19e27dea..a79d25f7540d005 100644 --- a/be/src/cloud/cloud_tablet.h +++ b/be/src/cloud/cloud_tablet.h @@ -208,6 +208,9 @@ class CloudTablet final : public BaseTablet { Status sync_if_not_running(); + // Merge all rowset schemas within a CloudTablet + Status merge_rowsets_schema(); + CloudStorageEngine& _engine; // this mutex MUST ONLY be used when sync meta @@ -246,6 +249,9 @@ class CloudTablet final : public BaseTablet { std::mutex _base_compaction_lock; std::mutex _cumulative_compaction_lock; mutable std::mutex _rowset_update_lock; + + // Schema will be merged from all rowsets when sync_rowsets + TabletSchemaSPtr _merged_tablet_schema; }; using CloudTabletSPtr = std::shared_ptr; diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 8217bd11bb91563..c23cc05758451dc 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -1159,7 +1159,10 @@ void PInternalService::fetch_remote_tablet_schema(google::protobuf::RpcControlle LOG(WARNING) << "tablet does not exist, tablet id is " << tablet_id; continue; } - tablet_schemas.push_back(res.value()->merged_tablet_schema()); + auto schema = res.value()->merged_tablet_schema(); + if (schema != nullptr) { + tablet_schemas.push_back(schema); + } } if (!tablet_schemas.empty()) { // merge all From 06164d9d1caaa2c4b08bd4c8a778e4337713f31e Mon Sep 17 00:00:00 2001 From: LiBinfeng <46676950+LiBinfeng-01@users.noreply.github.com> Date: Fri, 1 Nov 2024 15:49:24 +0800 Subject: [PATCH 34/82] [update](collaborators) add LiBinfeng-01 to collaborator list (#43045) --- .asf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index a2a3064783f0e98..3892aca2eddb77c 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -152,13 +152,13 @@ github: - LemonLiTree - Yukang-Lian - TangSiyang2001 - - Lchangliang - freemandealer - shuke987 - wm1581066 - KassieZ - yujun777 - doris-robot + - LiBinfeng-01 notifications: pullrequests_status: commits@doris.apache.org From 05c2fcb1611812b4a57e360eeae591c20d041128 Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Fri, 1 Nov 2024 16:02:57 +0800 Subject: [PATCH 35/82] [fix](auditlog) set isQuery to true when query is short circuited (#42647) --- fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 1726e85fb73d997..494232ef38cb344 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -1912,6 +1912,7 @@ public void executeAndSendResult(boolean isOutfileQuery, boolean isSendFields, : new ShortCircuitQueryContext(planner, (Queriable) parsedStmt); coordBase = new PointQueryExecutor(shortCircuitQueryContext, context.getSessionVariable().getMaxMsgSizeOfResultReceiver()); + context.getState().setIsQuery(true); } else if (planner instanceof NereidsPlanner && ((NereidsPlanner) planner).getDistributedPlans() != null) { coord = new NereidsCoordinator(context, analyzer, planner, context.getStatsErrorEstimator(), From 59fd2eabf83deaea4fd5b85a660082b77c5c1018 Mon Sep 17 00:00:00 2001 From: zclllhhjj Date: Fri, 1 Nov 2024 16:16:48 +0800 Subject: [PATCH 36/82] [Enhancement](log) Make log of vlog adjustment clear (#42997) before: ``` adjust log level success, origin level -1 ``` no info log now: ``` adjust vlog of vrow_distribution from 10 to -1 succeed ``` and have info log --- be/src/http/action/adjust_log_level.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/be/src/http/action/adjust_log_level.cpp b/be/src/http/action/adjust_log_level.cpp index 687639a9b58deaa..a8644a0fb5f52a3 100644 --- a/be/src/http/action/adjust_log_level.cpp +++ b/be/src/http/action/adjust_log_level.cpp @@ -17,8 +17,9 @@ #include +#include + #include "common/logging.h" -#include "common/status.h" #include "http/http_channel.h" #include "http/http_request.h" @@ -26,7 +27,7 @@ namespace doris { // **Note**: If the module_name does not exist in the vlog modules, vlog // would create corresponding module for it. -int handle_request(HttpRequest* req) { +std::tuple handle_request(HttpRequest* req) { auto parse_param = [&req](std::string param) { const auto& value = req->param(param); if (value.empty()) { @@ -38,13 +39,16 @@ int handle_request(HttpRequest* req) { const auto& module = parse_param("module"); const auto& level = parse_param("level"); int new_level = std::stoi(level); - return google::SetVLOGLevel(module.c_str(), new_level); + return std::make_tuple(module, google::SetVLOGLevel(module.c_str(), new_level), new_level); } void AdjustLogLevelAction::handle(HttpRequest* req) { try { - auto old_level = handle_request(req); - auto msg = fmt::format("adjust log level success, origin level is {}", old_level); + auto handle_result = handle_request(req); + auto msg = + fmt::format("adjust vlog of {} from {} to {} succeed", std::get<0>(handle_result), + std::get<1>(handle_result), std::get<2>(handle_result)); + LOG(INFO) << msg; HttpChannel::send_reply(req, msg); } catch (const std::exception& e) { HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, e.what()); From 03c625f69375bd74e419fed1f8ee6fd6a13bc3e4 Mon Sep 17 00:00:00 2001 From: daidai <2017501503@qq.com> Date: Fri, 1 Nov 2024 16:29:53 +0800 Subject: [PATCH 37/82] [fix](hive)fix hive catalog miss partition that have special characters. (#42906) ## Proposed changes Previously, when processing partition values, Hive catalog parsed them in the URL format. However, this is different from the encoding method of Hive, which results in missing some partitions with special characters when reading the partition table. This PR is to fix this problem. Ref: common/src/java/org/apache/hadoop/hive/common/FileUtils.java:`makePartName(List partCols, List vals,String defaultStr)` --- .../datasource/TablePartitionValues.java | 30 -- .../datasource/hive/HiveMetaStoreCache.java | 13 +- .../doris/datasource/hive/HiveUtil.java | 29 +- .../org/apache/doris/qe/ShowExecutor.java | 5 + .../doris/statistics/HMSAnalysisTask.java | 9 +- .../hive/test_hive_special_char_partition.out | 396 ++++++++++++++++++ .../test_hive_special_char_partition.groovy | 199 ++++++++- 7 files changed, 626 insertions(+), 55 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java index d5e8a39e605a8b0..c7f2ce6f712d6b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/TablePartitionValues.java @@ -34,11 +34,7 @@ import com.google.common.collect.RangeMap; import lombok.Data; -import java.io.UnsupportedEncodingException; -import java.net.URLDecoder; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -80,11 +76,6 @@ public TablePartitionValues(List partitionNames, List> part addPartitions(partitionNames, partitionValues, types); } - public TablePartitionValues(List partitionNames, List types) { - this(); - addPartitions(partitionNames, types); - } - public void addPartitions(List partitionNames, List> partitionValues, List types) { Preconditions.checkState(partitionNames.size() == partitionValues.size()); List addPartitionNames = new ArrayList<>(); @@ -105,10 +96,6 @@ public void addPartitions(List partitionNames, List> partit addPartitionItems(addPartitionNames, addPartitionItems, types); } - public void addPartitions(List partitionNames, List types) { - addPartitions(partitionNames, - partitionNames.stream().map(this::getHivePartitionValues).collect(Collectors.toList()), types); - } private void addPartitionItems(List partitionNames, List partitionItems, List types) { Preconditions.checkState(partitionNames.size() == partitionItems.size()); @@ -196,23 +183,6 @@ private ListPartitionItem toListPartitionItem(List partitionValues, List } } - private List getHivePartitionValues(String partitionName) { - // Partition name will be in format: nation=cn/city=beijing - // parse it to get values "cn" and "beijing" - return Arrays.stream(partitionName.split("/")).map(part -> { - String[] kv = part.split("="); - Preconditions.checkState(kv.length == 2, partitionName); - String partitionValue; - try { - // hive partition value maybe contains special characters like '=' and '/' - partitionValue = URLDecoder.decode(kv[1], StandardCharsets.UTF_8.name()); - } catch (UnsupportedEncodingException e) { - // It should not be here - throw new RuntimeException(e); - } - return partitionValue; - }).collect(Collectors.toList()); - } @Data public static class TablePartitionKey { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java index fbfd7dd27986689..ea42dfa2f52a01d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreCache.java @@ -244,7 +244,7 @@ public Long getValue() { } private HivePartitionValues loadPartitionValues(PartitionValueCacheKey key) { - // partition name format: nation=cn/city=beijing + // partition name format: nation=cn/city=beijing,`listPartitionNames` returned string is the encoded string. List partitionNames = catalog.getClient().listPartitionNames(key.dbName, key.tblName); if (LOG.isDebugEnabled()) { LOG.debug("load #{} partitions for {} in catalog {}", partitionNames.size(), key, catalog.getName()); @@ -281,11 +281,10 @@ private HivePartitionValues loadPartitionValues(PartitionValueCacheKey key) { public ListPartitionItem toListPartitionItem(String partitionName, List types) { // Partition name will be in format: nation=cn/city=beijing // parse it to get values "cn" and "beijing" - String[] parts = partitionName.split("/"); - Preconditions.checkState(parts.length == types.size(), partitionName + " vs. " + types); + List partitionValues = HiveUtil.toPartitionValues(partitionName); + Preconditions.checkState(partitionValues.size() == types.size(), partitionName + " vs. " + types); List values = Lists.newArrayListWithExpectedSize(types.size()); - for (String part : parts) { - String partitionValue = HiveUtil.getHivePartitionValue(part); + for (String partitionValue : partitionValues) { values.add(new PartitionValue(partitionValue, HIVE_DEFAULT_PARTITION.equals(partitionValue))); } try { @@ -325,9 +324,9 @@ private Map loadPartitions(Iterable List(["c1","a"], ["c2","b"], ["c3","c"]) + // Similar to the `toPartitionValues` method, except that it adds the partition column name. + public static List toPartitionColNameAndValues(String partitionName) { + + String[] parts = partitionName.split("/"); + List result = new ArrayList<>(parts.length); + for (String part : parts) { + String[] kv = part.split("="); + Preconditions.checkState(kv.length == 2, String.format("Malformed partition name %s", part)); + + result.add(new String[] { + FileUtils.unescapePathName(kv[0]), + FileUtils.unescapePathName(kv[1]) + }); } + return result; } // "c1=a/c2=b/c3=c" ---> List("a","b","c") @@ -151,6 +154,8 @@ public static List toPartitionValues(String partitionName) { if (start > partitionName.length()) { break; } + //Ref: common/src/java/org/apache/hadoop/hive/common/FileUtils.java + //makePartName(List partCols, List vals,String defaultStr) resultBuilder.add(FileUtils.unescapePathName(partitionName.substring(start, end))); start = end + 1; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index 6c584f7255a5caa..9d814fd081ad311 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -1933,6 +1933,11 @@ private void handleShowHMSTablePartitions(ShowPartitionsStmt showStmt) throws An Map filterMap = showStmt.getFilterMap(); List orderByPairs = showStmt.getOrderByPairs(); + // catalog.getClient().listPartitionNames() returned string is the encoded string. + // example: insert into tmp partition(pt="1=3/3") values( xxx ); + // show partitions from tmp: pt=1%3D3%2F3 + // Need to consider whether to call `HiveUtil.toPartitionColNameAndValues` method + if (limit != null && limit.hasLimit() && limit.getOffset() == 0 && (orderByPairs == null || !orderByPairs.get(0).isDesc())) { // hmsClient returns unordered partition list, hence if offset > 0 cannot pass limit diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java index 7b807b4661cdc24..06009b381778c21 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/HMSAnalysisTask.java @@ -104,10 +104,11 @@ private void getPartitionColumnStats() { for (String names : partitionNames) { // names is like "date=20230101" for one level partition // and like "date=20230101/hour=12" for two level partition - String[] parts = names.split("/"); - for (String part : parts) { - if (part.startsWith(col.getName())) { - String value = HiveUtil.getHivePartitionValue(part); + List parts = HiveUtil.toPartitionColNameAndValues(names); + for (String[] part : parts) { + String colName = part[0]; + String value = part[1]; + if (colName != null && colName.equals(col.getName())) { // HIVE_DEFAULT_PARTITION hive partition value when the partition name is not specified. if (value == null || value.isEmpty() || value.equals(HiveMetaStoreCache.HIVE_DEFAULT_PARTITION)) { numNulls += 1; diff --git a/regression-test/data/external_table_p0/hive/test_hive_special_char_partition.out b/regression-test/data/external_table_p0/hive/test_hive_special_char_partition.out index f81719d2d0e1c73..13c1d2c155542b7 100644 --- a/regression-test/data/external_table_p0/hive/test_hive_special_char_partition.out +++ b/regression-test/data/external_table_p0/hive/test_hive_special_char_partition.out @@ -49,6 +49,204 @@ name6 2023%01%01 -- !13 -- name# 2023#01#01 +-- !sql1 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +1 %100, @@@@@@ , %100 , !!asd!!, A%%Z +2 abc:xyz, 1123,1222, ::::::: +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z +200 abc:xyz, 1123,1222, ::::::: + +-- !sql2 -- +pt= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A + +-- !sql_where1 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 + +-- !sql_where1 -- +1 %100, @@@@@@ , %100 , !!asd!!, A%%Z +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z + +-- !sql_where1 -- +2 abc:xyz, 1123,1222, ::::::: +200 abc:xyz, 1123,1222, ::::::: + +-- !sql3 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 %100, @@@@@@ , %100 , !!asd!!, A%%Z +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 abc:xyz, 1123,1222, ::::::: +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z %100, @@@@@@ , %100 , !!asd!!, A%%Z +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z abc:xyz, 1123,1222, ::::::: +200 abc:xyz, 1123,1222, ::::::: 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +201 abc:xyz, 1123,1222, ::::::: %100, @@@@@@ , %100 , !!asd!!, A%%Z +202 abc:xyz, 1123,1222, ::::::: abc:xyz, 1123,1222, ::::::: + +-- !sql4 -- +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A + +-- !sql_where2 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +200 abc:xyz, 1123,1222, ::::::: 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 + +-- !sql_where2 -- +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 %100, @@@@@@ , %100 , !!asd!!, A%%Z +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z %100, @@@@@@ , %100 , !!asd!!, A%%Z +201 abc:xyz, 1123,1222, ::::::: %100, @@@@@@ , %100 , !!asd!!, A%%Z + +-- !sql_where2 -- +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 abc:xyz, 1123,1222, ::::::: +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z abc:xyz, 1123,1222, ::::::: +202 abc:xyz, 1123,1222, ::::::: abc:xyz, 1123,1222, ::::::: + +-- !sql5 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 abc:xyz, 1123,1222, ::::::: 1 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 abc:xyz, 1123,1222, ::::::: 1 +200 abc:xyz, 1123,1222, ::::::: 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +201 abc:xyz, 1123,1222, ::::::: 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +202 abc:xyz, 1123,1222, ::::::: 1 1 abc:xyz, 1123,1222, ::::::: 1 + +-- !sql6 -- +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 + +-- !sql_where3 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +200 abc:xyz, 1123,1222, ::::::: 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 + +-- !sql_where3 -- +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +201 abc:xyz, 1123,1222, ::::::: 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 + +-- !sql_where3 -- +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 abc:xyz, 1123,1222, ::::::: 1 +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 abc:xyz, 1123,1222, ::::::: 1 +202 abc:xyz, 1123,1222, ::::::: 1 1 abc:xyz, 1123,1222, ::::::: 1 + +-- !sql1 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +1 %100, @@@@@@ , %100 , !!asd!!, A%%Z +2 abc:xyz, 1123,1222, ::::::: +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z +200 abc:xyz, 1123,1222, ::::::: + +-- !sql2 -- +pt= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A + +-- !sql_where1 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 + +-- !sql_where1 -- +1 %100, @@@@@@ , %100 , !!asd!!, A%%Z +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z + +-- !sql_where1 -- +2 abc:xyz, 1123,1222, ::::::: +200 abc:xyz, 1123,1222, ::::::: + +-- !sql3 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 %100, @@@@@@ , %100 , !!asd!!, A%%Z +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 abc:xyz, 1123,1222, ::::::: +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z %100, @@@@@@ , %100 , !!asd!!, A%%Z +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z abc:xyz, 1123,1222, ::::::: +200 abc:xyz, 1123,1222, ::::::: 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +201 abc:xyz, 1123,1222, ::::::: %100, @@@@@@ , %100 , !!asd!!, A%%Z +202 abc:xyz, 1123,1222, ::::::: abc:xyz, 1123,1222, ::::::: + +-- !sql4 -- +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A + +-- !sql_where2 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +200 abc:xyz, 1123,1222, ::::::: 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 + +-- !sql_where2 -- +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 %100, @@@@@@ , %100 , !!asd!!, A%%Z +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z %100, @@@@@@ , %100 , !!asd!!, A%%Z +201 abc:xyz, 1123,1222, ::::::: %100, @@@@@@ , %100 , !!asd!!, A%%Z + +-- !sql_where2 -- +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 abc:xyz, 1123,1222, ::::::: +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z abc:xyz, 1123,1222, ::::::: +202 abc:xyz, 1123,1222, ::::::: abc:xyz, 1123,1222, ::::::: + +-- !sql5 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 abc:xyz, 1123,1222, ::::::: 1 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 abc:xyz, 1123,1222, ::::::: 1 +200 abc:xyz, 1123,1222, ::::::: 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +201 abc:xyz, 1123,1222, ::::::: 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +202 abc:xyz, 1123,1222, ::::::: 1 1 abc:xyz, 1123,1222, ::::::: 1 + +-- !sql6 -- +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 + +-- !sql_where3 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +200 abc:xyz, 1123,1222, ::::::: 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 + +-- !sql_where3 -- +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +201 abc:xyz, 1123,1222, ::::::: 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 + +-- !sql_where3 -- +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 abc:xyz, 1123,1222, ::::::: 1 +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 abc:xyz, 1123,1222, ::::::: 1 +202 abc:xyz, 1123,1222, ::::::: 1 1 abc:xyz, 1123,1222, ::::::: 1 + -- !1 -- name# 2023#01#01 name1 2023/01/01 @@ -99,3 +297,201 @@ name6 2023%01%01 -- !13 -- name# 2023#01#01 +-- !sql1 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +1 %100, @@@@@@ , %100 , !!asd!!, A%%Z +2 abc:xyz, 1123,1222, ::::::: +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z +200 abc:xyz, 1123,1222, ::::::: + +-- !sql2 -- +pt= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A + +-- !sql_where1 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 + +-- !sql_where1 -- +1 %100, @@@@@@ , %100 , !!asd!!, A%%Z +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z + +-- !sql_where1 -- +2 abc:xyz, 1123,1222, ::::::: +200 abc:xyz, 1123,1222, ::::::: + +-- !sql3 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 %100, @@@@@@ , %100 , !!asd!!, A%%Z +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 abc:xyz, 1123,1222, ::::::: +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z %100, @@@@@@ , %100 , !!asd!!, A%%Z +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z abc:xyz, 1123,1222, ::::::: +200 abc:xyz, 1123,1222, ::::::: 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +201 abc:xyz, 1123,1222, ::::::: %100, @@@@@@ , %100 , !!asd!!, A%%Z +202 abc:xyz, 1123,1222, ::::::: abc:xyz, 1123,1222, ::::::: + +-- !sql4 -- +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A + +-- !sql_where2 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +200 abc:xyz, 1123,1222, ::::::: 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 + +-- !sql_where2 -- +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 %100, @@@@@@ , %100 , !!asd!!, A%%Z +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z %100, @@@@@@ , %100 , !!asd!!, A%%Z +201 abc:xyz, 1123,1222, ::::::: %100, @@@@@@ , %100 , !!asd!!, A%%Z + +-- !sql_where2 -- +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 abc:xyz, 1123,1222, ::::::: +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z abc:xyz, 1123,1222, ::::::: +202 abc:xyz, 1123,1222, ::::::: abc:xyz, 1123,1222, ::::::: + +-- !sql5 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 abc:xyz, 1123,1222, ::::::: 1 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 abc:xyz, 1123,1222, ::::::: 1 +200 abc:xyz, 1123,1222, ::::::: 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +201 abc:xyz, 1123,1222, ::::::: 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +202 abc:xyz, 1123,1222, ::::::: 1 1 abc:xyz, 1123,1222, ::::::: 1 + +-- !sql6 -- +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 + +-- !sql_where3 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +200 abc:xyz, 1123,1222, ::::::: 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 + +-- !sql_where3 -- +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +201 abc:xyz, 1123,1222, ::::::: 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 + +-- !sql_where3 -- +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 abc:xyz, 1123,1222, ::::::: 1 +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 abc:xyz, 1123,1222, ::::::: 1 +202 abc:xyz, 1123,1222, ::::::: 1 1 abc:xyz, 1123,1222, ::::::: 1 + +-- !sql1 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +1 %100, @@@@@@ , %100 , !!asd!!, A%%Z +2 abc:xyz, 1123,1222, ::::::: +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z +200 abc:xyz, 1123,1222, ::::::: + +-- !sql2 -- +pt= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A + +-- !sql_where1 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 + +-- !sql_where1 -- +1 %100, @@@@@@ , %100 , !!asd!!, A%%Z +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z + +-- !sql_where1 -- +2 abc:xyz, 1123,1222, ::::::: +200 abc:xyz, 1123,1222, ::::::: + +-- !sql3 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 %100, @@@@@@ , %100 , !!asd!!, A%%Z +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 abc:xyz, 1123,1222, ::::::: +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z %100, @@@@@@ , %100 , !!asd!!, A%%Z +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z abc:xyz, 1123,1222, ::::::: +200 abc:xyz, 1123,1222, ::::::: 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +201 abc:xyz, 1123,1222, ::::::: %100, @@@@@@ , %100 , !!asd!!, A%%Z +202 abc:xyz, 1123,1222, ::::::: abc:xyz, 1123,1222, ::::::: + +-- !sql4 -- +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A + +-- !sql_where2 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 +200 abc:xyz, 1123,1222, ::::::: 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 + +-- !sql_where2 -- +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 %100, @@@@@@ , %100 , !!asd!!, A%%Z +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z %100, @@@@@@ , %100 , !!asd!!, A%%Z +201 abc:xyz, 1123,1222, ::::::: %100, @@@@@@ , %100 , !!asd!!, A%%Z + +-- !sql_where2 -- +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 abc:xyz, 1123,1222, ::::::: +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z abc:xyz, 1123,1222, ::::::: +202 abc:xyz, 1123,1222, ::::::: abc:xyz, 1123,1222, ::::::: + +-- !sql5 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 abc:xyz, 1123,1222, ::::::: 1 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 abc:xyz, 1123,1222, ::::::: 1 +200 abc:xyz, 1123,1222, ::::::: 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +201 abc:xyz, 1123,1222, ::::::: 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +202 abc:xyz, 1123,1222, ::::::: 1 1 abc:xyz, 1123,1222, ::::::: 1 + +-- !sql6 -- +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4= %25100, @@@@@@ , %25100 , !!asd!!, A%25%25Z/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=1,1%3D1, 3%3D2+1, 1%3D3-2, 3%2F3%3D1, 2%2F2%3D1, 2%2F1%3D2, 2%2F1%3D2 +1 -1,2%2F1%3D2 %2A3 %2F3/pt5=1 +pt1=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt2%3Dx!!!! %2A%2A1+1%2F&%5E%253=1/pt3=1/pt4=abc%3Axyz, 1123,1222, %3A%3A%3A%3A%3A%3A%3A/pt5=1 + +-- !sql_where3 -- +0 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +100 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 +200 abc:xyz, 1123,1222, ::::::: 1 1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 + +-- !sql_where3 -- +1 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +101 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 +201 abc:xyz, 1123,1222, ::::::: 1 1 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 + +-- !sql_where3 -- +2 1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3 1 1 abc:xyz, 1123,1222, ::::::: 1 +102 %100, @@@@@@ , %100 , !!asd!!, A%%Z 1 1 abc:xyz, 1123,1222, ::::::: 1 +202 abc:xyz, 1123,1222, ::::::: 1 1 abc:xyz, 1123,1222, ::::::: 1 + diff --git a/regression-test/suites/external_table_p0/hive/test_hive_special_char_partition.groovy b/regression-test/suites/external_table_p0/hive/test_hive_special_char_partition.groovy index 8b78ab2e3ce9b45..d22041459d5ae53 100644 --- a/regression-test/suites/external_table_p0/hive/test_hive_special_char_partition.groovy +++ b/regression-test/suites/external_table_p0/hive/test_hive_special_char_partition.groovy @@ -23,14 +23,18 @@ suite("test_hive_special_char_partition", "p0,external,hive,external_docker,exte } for (String hivePrefix : ["hive2", "hive3"]) { + + setHivePrefix(hivePrefix) String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") String catalog_name = "${hivePrefix}_test_hive_special_char_partition" String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") sql """drop catalog if exists ${catalog_name}""" sql """create catalog if not exists ${catalog_name} properties ( - "type"="hms", - 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}' + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}' );""" logger.info("catalog " + catalog_name + " created") sql """switch ${catalog_name};""" @@ -49,6 +53,197 @@ suite("test_hive_special_char_partition", "p0,external,hive,external_docker,exte qt_11 "select * from special_character_1_partition where part='2023\\\\01\\\\01'" qt_12 "select * from special_character_1_partition where part='2023%01%01'" qt_13 "select * from special_character_1_partition where part='2023#01#01'" + + + + // append some case. + String table_name_1 = "partition_special_characters_1" + String table_name_2 = "partition_special_characters_2" + String table_name_3 = "partition_special_characters_3" + + hive_docker """ set hive.stats.column.autogather=false """ + hive_docker """ use `default`""" + def special_characters = [ + "1,1=1, 3=2+1, 1=3-2, 3/3=1, 2/2=1, 2/1=2, 2/1=2 +1 -1,2/1=2 *3 /3", + " %100, @@@@@@ , %100 , !!asd!!, A%%Z", + "abc:xyz, 1123,1222, :::::::" + ] + + + logger.info(""" docker insert 1""") + + hive_docker """ drop table if exists ${table_name_1} """ + hive_docker """ create table ${table_name_1} (id int) partitioned by (pt string) """ + special_characters.eachWithIndex { item, index -> + hive_docker """ insert into ${table_name_1} partition(pt="${item}") values ("${index}"); """ + hive_docker """ insert into ${table_name_1} partition(pt="${item}") values ("${index*100}"); """ + println("Index: ${index}, Item: ${item}") + } + + + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + + qt_sql1 """ select * from ${table_name_1} order by id """ + qt_sql2 """ show partitions from ${table_name_1} """ + + special_characters.eachWithIndex { item, index -> + qt_sql_where1 """ select * from ${table_name_1} where pt = "${item}" order by id""" + } + + + + logger.info(""" docker insert 2""") + hive_docker """ drop table if exists ${table_name_2} """ + hive_docker """ create table ${table_name_2} (id int) partitioned by (pt1 string, `pt2=x!!!! **1+1/&^%3` string) """ + + special_characters.eachWithIndex { outerItem, outerIndex -> + special_characters.eachWithIndex { innerItem, innerIndex -> + + def insert_value = outerIndex * 100 + innerIndex; + + hive_docker """ insert into ${table_name_2} partition(pt1="${outerItem}",`pt2=x!!!! **1+1/&^%3`="${innerItem}") values ("${insert_value}"); """ + println(" Outer Item: ${outerItem}, Inner Item: ${innerItem}, value = ${insert_value}") + } + } + + + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + + qt_sql3 """ select * from ${table_name_2} order by id """ + qt_sql4 """ show partitions from ${table_name_2} """ + + special_characters.eachWithIndex { item, index -> + qt_sql_where2 """ select * from ${table_name_2} where `pt2=x!!!! **1+1/&^%3` = "${item}" order by id""" + } + + + logger.info(""" docker insert 3""") + hive_docker """ drop table if exists ${table_name_3} """ + hive_docker """ create table ${table_name_3} (id int) partitioned by (pt1 string, `pt2=x!!!! **1+1/&^%3` string,pt3 string,pt4 string,pt5 string) """ + + + special_characters.eachWithIndex { outerItem, outerIndex -> + special_characters.eachWithIndex { innerItem, innerIndex -> + + def insert_value = outerIndex * 100 + innerIndex; + hive_docker """ insert into ${table_name_3} partition(pt1="${outerItem}", `pt2=x!!!! **1+1/&^%3`="1", pt3="1",pt4="${innerItem}",pt5="1") values ("${insert_value}"); """ + println(" Outer Item: ${outerItem}, Inner Item: ${innerItem}, value = ${insert_value}") + } + } + + + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + + qt_sql5 """ select * from ${table_name_3} order by id """ + qt_sql6 """ show partitions from ${table_name_3} """ + + special_characters.eachWithIndex { item, index -> + qt_sql_where3 """ select * from ${table_name_3} where pt4 = "${item}" order by id""" + } + + + + + + + logger.info(""" ---------- doris insert -------------""") + + logger.info(""" doris insert 1""") + hive_docker """ drop table if exists ${table_name_1} """ + hive_docker """ create table ${table_name_1} (id int) partitioned by (pt string) """ + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + special_characters.eachWithIndex { item, index -> + sql """ insert into ${table_name_1} (pt,id) values ("${item}","${index}"); """ + sql """ insert into ${table_name_1} (pt,id) values ("${item}","${index*100}"); """ + + println("Index: ${index}, Item: ${item}") + } + + + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + + qt_sql1 """ select * from ${table_name_1} order by id """ + qt_sql2 """ show partitions from ${table_name_1} """ + + special_characters.eachWithIndex { item, index -> + qt_sql_where1 """ select * from ${table_name_1} where pt = "${item}" order by id """ + } + + + + logger.info(""" doris insert 2""") + hive_docker """ drop table if exists ${table_name_2} """ + hive_docker """ create table ${table_name_2} (id int) partitioned by (pt1 string, `pt2=x!!!! **1+1/&^%3` string) """ + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + + special_characters.eachWithIndex { outerItem, outerIndex -> + special_characters.eachWithIndex { innerItem, innerIndex -> + + def insert_value = outerIndex * 100 + innerIndex; + + sql """ insert into ${table_name_2} (pt1,`pt2=x!!!! **1+1/&^%3`,id) values ("${outerItem}","${innerItem}","${insert_value}"); """ + println(" Outer Item: ${outerItem}, Inner Item: ${innerItem}, value = ${insert_value}") + } + } + + + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + + qt_sql3 """ select * from ${table_name_2} order by id """ + qt_sql4 """ show partitions from ${table_name_2} """ + + special_characters.eachWithIndex { item, index -> + qt_sql_where2 """ select * from ${table_name_2} where `pt2=x!!!! **1+1/&^%3` = "${item}" order by id""" + } + + + + + logger.info(""" docker insert 3""") + hive_docker """ drop table if exists ${table_name_3} """ + hive_docker """ create table ${table_name_3} (id int) partitioned by (pt1 string, `pt2=x!!!! **1+1/&^%3` string,pt3 string,pt4 string,pt5 string) """ + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + + special_characters.eachWithIndex { outerItem, outerIndex -> + special_characters.eachWithIndex { innerItem, innerIndex -> + + def insert_value = outerIndex * 100 + innerIndex; + sql """ insert into ${table_name_3} (pt1, `pt2=x!!!! **1+1/&^%3`, pt3,pt4,pt5,id) values ("${outerItem}","1","1","${innerItem}","1","${insert_value}"); """ + println(" Outer Item: ${outerItem}, Inner Item: ${innerItem}, value = ${insert_value}") + } + } + + + sql """refresh catalog ${catalog_name} """ + sql """switch ${catalog_name} """ + sql """ use `default` """ + + qt_sql5 """ select * from ${table_name_3} order by id """ + qt_sql6 """ show partitions from ${table_name_3} """ + + special_characters.eachWithIndex { item, index -> + qt_sql_where3 """ select * from ${table_name_3} where pt4 = "${item}" order by id""" + } + + + + } } From 64870b3bbbb245a5bf8c02d58cf8e9a8dd2532d8 Mon Sep 17 00:00:00 2001 From: Pxl Date: Fri, 1 Nov 2024 18:04:20 +0800 Subject: [PATCH 38/82] [Chore](hash-table) remove has_nullable_keys template argument of MethodKeysFixed (#43005) ## Proposed changes remove has_nullable_keys template argument of MethodKeysFixed --- be/src/pipeline/common/agg_utils.h | 54 +++++----- be/src/pipeline/common/distinct_agg_utils.h | 45 ++++----- be/src/pipeline/common/join_utils.h | 66 +++++-------- be/src/pipeline/common/partition_sort_utils.h | 98 ++++++++----------- be/src/pipeline/common/set_utils.h | 26 ++--- .../exec/join/process_hash_table_probe_impl.h | 34 +++---- be/src/vec/common/columns_hashing.h | 18 +--- be/src/vec/common/columns_hashing_impl.h | 58 ----------- .../vec/common/hash_table/hash_map_context.h | 47 ++++----- be/src/vec/common/hash_table/hash_map_util.h | 12 +-- 10 files changed, 164 insertions(+), 294 deletions(-) diff --git a/be/src/pipeline/common/agg_utils.h b/be/src/pipeline/common/agg_utils.h index a3cc175b1ed0a25..135bc67712345f9 100644 --- a/be/src/pipeline/common/agg_utils.h +++ b/be/src/pipeline/common/agg_utils.h @@ -80,23 +80,19 @@ using AggregatedMethodVariants = std::variant< vectorized::UInt256, AggDataNullable>>, vectorized::MethodSingleNullableColumn< vectorized::MethodStringNoCache>, - vectorized::MethodKeysFixed, false>, - vectorized::MethodKeysFixed, true>, - vectorized::MethodKeysFixed, false>, - vectorized::MethodKeysFixed, true>, - vectorized::MethodKeysFixed, false>, - vectorized::MethodKeysFixed, true>, - vectorized::MethodKeysFixed, false>, - vectorized::MethodKeysFixed, true>>; + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>>; struct AggregatedDataVariants : public DataVariants { + vectorized::MethodOneNumber, vectorized::DataWithNullKey> { AggregatedDataWithoutKey without_key = nullptr; - template void init(const std::vector& data_types, HashKeyType type) { + bool nullable = data_types.size() == 1 && data_types[0]->is_nullable(); + switch (type) { case HashKeyType::without_key: break; @@ -104,28 +100,28 @@ struct AggregatedDataVariants method_variant.emplace>(); break; case HashKeyType::int8_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int16_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int32_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int32_key_phase2: - emplace_single(); + emplace_single(nullable); break; case HashKeyType::int64_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int64_key_phase2: - emplace_single(); + emplace_single(nullable); break; case HashKeyType::int128_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int256_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::string_key: if (nullable) { @@ -138,24 +134,20 @@ struct AggregatedDataVariants } break; case HashKeyType::fixed64: - method_variant - .emplace, nullable>>( - get_key_sizes(data_types)); + method_variant.emplace>>( + get_key_sizes(data_types)); break; case HashKeyType::fixed128: - method_variant - .emplace, nullable>>( - get_key_sizes(data_types)); + method_variant.emplace>>( + get_key_sizes(data_types)); break; case HashKeyType::fixed136: - method_variant - .emplace, nullable>>( - get_key_sizes(data_types)); + method_variant.emplace>>( + get_key_sizes(data_types)); break; case HashKeyType::fixed256: - method_variant - .emplace, nullable>>( - get_key_sizes(data_types)); + method_variant.emplace>>( + get_key_sizes(data_types)); break; default: throw Exception(ErrorCode::INTERNAL_ERROR, diff --git a/be/src/pipeline/common/distinct_agg_utils.h b/be/src/pipeline/common/distinct_agg_utils.h index c7ecbd2142c7f0f..806039d5a36a4b4 100644 --- a/be/src/pipeline/common/distinct_agg_utils.h +++ b/be/src/pipeline/common/distinct_agg_utils.h @@ -72,48 +72,43 @@ using DistinctMethodVariants = std::variant< vectorized::DataWithNullKey>>>, vectorized::MethodSingleNullableColumn>>, - vectorized::MethodKeysFixed, false>, - vectorized::MethodKeysFixed, true>, - vectorized::MethodKeysFixed, false>, - vectorized::MethodKeysFixed, true>, - vectorized::MethodKeysFixed, false>, - vectorized::MethodKeysFixed, true>, - vectorized::MethodKeysFixed, false>, - vectorized::MethodKeysFixed, true>>; + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>>; struct DistinctDataVariants : public DataVariants { - template + vectorized::MethodOneNumber, vectorized::DataWithNullKey> { void init(const std::vector& data_types, HashKeyType type) { + bool nullable = data_types.size() == 1 && data_types[0]->is_nullable(); switch (type) { case HashKeyType::serialized: method_variant.emplace>(); break; case HashKeyType::int8_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int16_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int32_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int32_key_phase2: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int64_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int64_key_phase2: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int128_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::int256_key: - emplace_single, nullable>(); + emplace_single>(nullable); break; case HashKeyType::string_key: if (nullable) { @@ -126,23 +121,19 @@ struct DistinctDataVariants } break; case HashKeyType::fixed64: - method_variant.emplace< - vectorized::MethodKeysFixed, nullable>>( + method_variant.emplace>>( get_key_sizes(data_types)); break; case HashKeyType::fixed128: - method_variant.emplace< - vectorized::MethodKeysFixed, nullable>>( + method_variant.emplace>>( get_key_sizes(data_types)); break; case HashKeyType::fixed136: - method_variant.emplace< - vectorized::MethodKeysFixed, nullable>>( + method_variant.emplace>>( get_key_sizes(data_types)); break; case HashKeyType::fixed256: - method_variant.emplace< - vectorized::MethodKeysFixed, nullable>>( + method_variant.emplace>>( get_key_sizes(data_types)); break; default: diff --git a/be/src/pipeline/common/join_utils.h b/be/src/pipeline/common/join_utils.h index 52c56abde1afef2..e214d1a52931a90 100644 --- a/be/src/pipeline/common/join_utils.h +++ b/be/src/pipeline/common/join_utils.h @@ -36,43 +36,29 @@ using JoinOpVariants = std::integral_constant, std::integral_constant>; -using SerializedHashTableContext = vectorized::MethodSerialized>; -using I8HashTableContext = vectorized::PrimaryTypeHashTableContext; -using I16HashTableContext = vectorized::PrimaryTypeHashTableContext; -using I32HashTableContext = vectorized::PrimaryTypeHashTableContext; -using I64HashTableContext = vectorized::PrimaryTypeHashTableContext; -using I128HashTableContext = vectorized::PrimaryTypeHashTableContext; -using I256HashTableContext = vectorized::PrimaryTypeHashTableContext; -using MethodOneString = vectorized::MethodStringNoCache>; -template -using I64FixedKeyHashTableContext = - vectorized::FixedKeyHashTableContext; - -template -using I128FixedKeyHashTableContext = - vectorized::FixedKeyHashTableContext; +template +using PrimaryTypeHashTableContext = vectorized::MethodOneNumber>>; -template -using I256FixedKeyHashTableContext = - vectorized::FixedKeyHashTableContext; +template +using FixedKeyHashTableContext = vectorized::MethodKeysFixed>>; -template -using I136FixedKeyHashTableContext = - vectorized::FixedKeyHashTableContext; +using SerializedHashTableContext = vectorized::MethodSerialized>; +using MethodOneString = vectorized::MethodStringNoCache>; -using HashTableVariants = - std::variant, - I64FixedKeyHashTableContext, I128FixedKeyHashTableContext, - I128FixedKeyHashTableContext, I256FixedKeyHashTableContext, - I256FixedKeyHashTableContext, I136FixedKeyHashTableContext, - I136FixedKeyHashTableContext, MethodOneString>; +using HashTableVariants = std::variant< + std::monostate, SerializedHashTableContext, PrimaryTypeHashTableContext, + PrimaryTypeHashTableContext, + PrimaryTypeHashTableContext, + PrimaryTypeHashTableContext, + PrimaryTypeHashTableContext, + PrimaryTypeHashTableContext, + FixedKeyHashTableContext, FixedKeyHashTableContext, + FixedKeyHashTableContext, + FixedKeyHashTableContext, MethodOneString>; struct JoinDataVariants { HashTableVariants method_variant; - template void init(const std::vector& data_types, HashKeyType type) { // todo: support single column nullable context switch (type) { @@ -80,40 +66,40 @@ struct JoinDataVariants { method_variant.emplace(); break; case HashKeyType::int8_key: - method_variant.emplace(); + method_variant.emplace>(); break; case HashKeyType::int16_key: - method_variant.emplace(); + method_variant.emplace>(); break; case HashKeyType::int32_key: - method_variant.emplace(); + method_variant.emplace>(); break; case HashKeyType::int64_key: - method_variant.emplace(); + method_variant.emplace>(); break; case HashKeyType::int128_key: - method_variant.emplace(); + method_variant.emplace>(); break; case HashKeyType::int256_key: - method_variant.emplace(); + method_variant.emplace>(); break; case HashKeyType::string_key: method_variant.emplace(); break; case HashKeyType::fixed64: - method_variant.emplace>( + method_variant.emplace>( get_key_sizes(data_types)); break; case HashKeyType::fixed128: - method_variant.emplace>( + method_variant.emplace>( get_key_sizes(data_types)); break; case HashKeyType::fixed136: - method_variant.emplace>( + method_variant.emplace>( get_key_sizes(data_types)); break; case HashKeyType::fixed256: - method_variant.emplace>( + method_variant.emplace>( get_key_sizes(data_types)); break; default: diff --git a/be/src/pipeline/common/partition_sort_utils.h b/be/src/pipeline/common/partition_sort_utils.h index 38bc8744dc14623..9317a783ba68bfe 100644 --- a/be/src/pipeline/common/partition_sort_utils.h +++ b/be/src/pipeline/common/partition_sort_utils.h @@ -123,57 +123,41 @@ struct PartitionBlocks { using PartitionDataPtr = PartitionBlocks*; using PartitionDataWithStringKey = PHHashMap; using PartitionDataWithShortStringKey = StringHashMap; -using PartitionDataWithUInt8Key = PHHashMap; -using PartitionDataWithUInt16Key = PHHashMap; -using PartitionDataWithUInt32Key = - PHHashMap>; -using PartitionDataWithUInt64Key = - PHHashMap>; -using PartitionDataWithUInt128Key = - PHHashMap>; -using PartitionDataWithUInt256Key = - PHHashMap>; -using PartitionDataWithUInt136Key = - PHHashMap>; + +template +using PartitionData = PHHashMap>; + +template +using PartitionDataSingle = vectorized::MethodOneNumber>; + +template +using PartitionDataSingleNullable = vectorized::MethodSingleNullableColumn< + vectorized::MethodOneNumber>>>; using PartitionedMethodVariants = std::variant< std::monostate, vectorized::MethodSerialized, - vectorized::MethodOneNumber, - vectorized::MethodOneNumber, - vectorized::MethodOneNumber, - vectorized::MethodOneNumber, - vectorized::MethodOneNumber, - vectorized::MethodOneNumber, - vectorized::MethodSingleNullableColumn>>, - vectorized::MethodSingleNullableColumn>>, - vectorized::MethodSingleNullableColumn>>, - vectorized::MethodSingleNullableColumn>>, - vectorized::MethodSingleNullableColumn>>, - vectorized::MethodSingleNullableColumn>>, - vectorized::MethodKeysFixed, - vectorized::MethodKeysFixed, - vectorized::MethodKeysFixed, - vectorized::MethodKeysFixed, - vectorized::MethodKeysFixed, - vectorized::MethodKeysFixed, - vectorized::MethodKeysFixed, - vectorized::MethodKeysFixed, + PartitionDataSingle, PartitionDataSingle, + PartitionDataSingle, PartitionDataSingle, + PartitionDataSingle, PartitionDataSingle, + PartitionDataSingleNullable, + PartitionDataSingleNullable, + PartitionDataSingleNullable, + PartitionDataSingleNullable, + PartitionDataSingleNullable, + PartitionDataSingleNullable, + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>, + vectorized::MethodKeysFixed>, vectorized::MethodStringNoCache, vectorized::MethodSingleNullableColumn>>>; struct PartitionedHashMapVariants : public DataVariants { - template + vectorized::MethodOneNumber, vectorized::DataWithNullKey> { void init(const std::vector& data_types, HashKeyType type) { + bool nullable = data_types.size() == 1 && data_types[0]->is_nullable(); switch (type) { case HashKeyType::without_key: { break; @@ -183,27 +167,27 @@ struct PartitionedHashMapVariants break; } case HashKeyType::int8_key: { - emplace_single(); + emplace_single>(nullable); break; } case HashKeyType::int16_key: { - emplace_single(); + emplace_single>(nullable); break; } case HashKeyType::int32_key: { - emplace_single(); + emplace_single>(nullable); break; } case HashKeyType::int64_key: { - emplace_single(); + emplace_single>(nullable); break; } case HashKeyType::int128_key: { - emplace_single(); + emplace_single>(nullable); break; } case HashKeyType::int256_key: { - emplace_single(); + emplace_single>(nullable); break; } case HashKeyType::string_key: { @@ -218,24 +202,20 @@ struct PartitionedHashMapVariants break; } case HashKeyType::fixed64: - method_variant - .emplace>( - get_key_sizes(data_types)); + method_variant.emplace>>( + get_key_sizes(data_types)); break; case HashKeyType::fixed128: - method_variant - .emplace>( - get_key_sizes(data_types)); + method_variant.emplace>>( + get_key_sizes(data_types)); break; case HashKeyType::fixed136: - method_variant - .emplace>( - get_key_sizes(data_types)); + method_variant.emplace>>( + get_key_sizes(data_types)); break; case HashKeyType::fixed256: - method_variant - .emplace>( - get_key_sizes(data_types)); + method_variant.emplace>>( + get_key_sizes(data_types)); break; default: throw Exception(ErrorCode::INTERNAL_ERROR, diff --git a/be/src/pipeline/common/set_utils.h b/be/src/pipeline/common/set_utils.h index 014546be124ced7..9b1a2579cf9b937 100644 --- a/be/src/pipeline/common/set_utils.h +++ b/be/src/pipeline/common/set_utils.h @@ -25,10 +25,9 @@ namespace doris { -template +template using SetFixedKeyHashTableContext = - vectorized::MethodKeysFixed>, - has_null>; + vectorized::MethodKeysFixed>>; template using SetPrimaryTypeHashTableContext = @@ -47,19 +46,14 @@ using SetHashTableVariants = SetPrimaryTypeHashTableContext, SetPrimaryTypeHashTableContext, SetPrimaryTypeHashTableContext, - SetFixedKeyHashTableContext, - SetFixedKeyHashTableContext, - SetFixedKeyHashTableContext, - SetFixedKeyHashTableContext, - SetFixedKeyHashTableContext, - SetFixedKeyHashTableContext, - SetFixedKeyHashTableContext, - SetFixedKeyHashTableContext>; + SetFixedKeyHashTableContext, + SetFixedKeyHashTableContext, + SetFixedKeyHashTableContext, + SetFixedKeyHashTableContext>; struct SetDataVariants { SetHashTableVariants method_variant; - template void init(const std::vector& data_types, HashKeyType type) { switch (type) { case HashKeyType::serialized: @@ -87,19 +81,19 @@ struct SetDataVariants { method_variant.emplace(); break; case HashKeyType::fixed64: - method_variant.emplace>( + method_variant.emplace>( get_key_sizes(data_types)); break; case HashKeyType::fixed128: - method_variant.emplace>( + method_variant.emplace>( get_key_sizes(data_types)); break; case HashKeyType::fixed136: - method_variant.emplace>( + method_variant.emplace>( get_key_sizes(data_types)); break; case HashKeyType::fixed256: - method_variant.emplace>( + method_variant.emplace>( get_key_sizes(data_types)); break; default: diff --git a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h index 231c231c81326ee..05cd3d7d9e0590f 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h @@ -714,24 +714,20 @@ struct ExtractType { ExtractType::Type & hash_table_ctx, vectorized::MutableBlock & mutable_block, \ vectorized::Block * output_block, bool* eos, bool is_mark_join); -#define INSTANTIATION_FOR(JoinOpType) \ - template struct ProcessHashTableProbe; \ - \ - INSTANTIATION(JoinOpType, (SerializedHashTableContext)); \ - INSTANTIATION(JoinOpType, (I8HashTableContext)); \ - INSTANTIATION(JoinOpType, (I16HashTableContext)); \ - INSTANTIATION(JoinOpType, (I32HashTableContext)); \ - INSTANTIATION(JoinOpType, (I64HashTableContext)); \ - INSTANTIATION(JoinOpType, (I128HashTableContext)); \ - INSTANTIATION(JoinOpType, (I256HashTableContext)); \ - INSTANTIATION(JoinOpType, (I64FixedKeyHashTableContext)); \ - INSTANTIATION(JoinOpType, (I64FixedKeyHashTableContext)); \ - INSTANTIATION(JoinOpType, (I128FixedKeyHashTableContext)); \ - INSTANTIATION(JoinOpType, (I128FixedKeyHashTableContext)); \ - INSTANTIATION(JoinOpType, (I256FixedKeyHashTableContext)); \ - INSTANTIATION(JoinOpType, (I256FixedKeyHashTableContext)); \ - INSTANTIATION(JoinOpType, (I136FixedKeyHashTableContext)); \ - INSTANTIATION(JoinOpType, (MethodOneString)); \ - INSTANTIATION(JoinOpType, (I136FixedKeyHashTableContext)); +#define INSTANTIATION_FOR(JoinOpType) \ + template struct ProcessHashTableProbe; \ + \ + INSTANTIATION(JoinOpType, (SerializedHashTableContext)); \ + INSTANTIATION(JoinOpType, (PrimaryTypeHashTableContext)); \ + INSTANTIATION(JoinOpType, (PrimaryTypeHashTableContext)); \ + INSTANTIATION(JoinOpType, (PrimaryTypeHashTableContext)); \ + INSTANTIATION(JoinOpType, (PrimaryTypeHashTableContext)); \ + INSTANTIATION(JoinOpType, (PrimaryTypeHashTableContext)); \ + INSTANTIATION(JoinOpType, (PrimaryTypeHashTableContext)); \ + INSTANTIATION(JoinOpType, (FixedKeyHashTableContext)); \ + INSTANTIATION(JoinOpType, (FixedKeyHashTableContext)); \ + INSTANTIATION(JoinOpType, (FixedKeyHashTableContext)); \ + INSTANTIATION(JoinOpType, (FixedKeyHashTableContext)); \ + INSTANTIATION(JoinOpType, (MethodOneString)); #include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/vec/common/columns_hashing.h b/be/src/vec/common/columns_hashing.h index 4bdbf51444fbbf8..6a59c5964e47a4e 100644 --- a/be/src/vec/common/columns_hashing.h +++ b/be/src/vec/common/columns_hashing.h @@ -38,11 +38,6 @@ namespace doris::vectorized { using Sizes = std::vector; -inline bool has_nullable_key(const std::vector& data_types) { - return std::ranges::any_of(data_types.begin(), data_types.end(), - [](auto t) { return t->is_nullable(); }); -} - inline Sizes get_key_sizes(const std::vector& data_types) { Sizes key_sizes; for (const auto& data_type : data_types) { @@ -101,17 +96,14 @@ struct HashMethodSerialized }; /// For the case when all keys are of fixed length, and they fit in N (for example, 128) bits. -template +template struct HashMethodKeysFixed - : private columns_hashing_impl::BaseStateKeysFixed, - public columns_hashing_impl::HashMethodBase< - HashMethodKeysFixed, Value, Mapped, - false> { - using Self = HashMethodKeysFixed; + : public columns_hashing_impl::HashMethodBase, + Value, Mapped, false> { + using Self = HashMethodKeysFixed; using BaseHashed = columns_hashing_impl::HashMethodBase; - using Base = columns_hashing_impl::BaseStateKeysFixed; - HashMethodKeysFixed(const ColumnRawPtrs& key_columns) : Base(key_columns) {} + HashMethodKeysFixed(const ColumnRawPtrs& key_columns) {} }; template diff --git a/be/src/vec/common/columns_hashing_impl.h b/be/src/vec/common/columns_hashing_impl.h index 2665d9b797903a5..a11ec17ec705276 100644 --- a/be/src/vec/common/columns_hashing_impl.h +++ b/be/src/vec/common/columns_hashing_impl.h @@ -149,64 +149,6 @@ class HashMethodBase { } }; -template -struct MappedCache : public PaddedPODArray {}; - -template <> -struct MappedCache {}; - -/// This class is designed to provide the functionality that is required for -/// supporting nullable keys in HashMethodKeysFixed. If there are -/// no nullable keys, this class is merely implemented as an empty shell. -template -class BaseStateKeysFixed; - -/// Case where nullable keys are supported. -template -class BaseStateKeysFixed { -protected: - BaseStateKeysFixed(const ColumnRawPtrs& key_columns) { - null_maps.reserve(key_columns.size()); - actual_columns.reserve(key_columns.size()); - - for (const auto& col : key_columns) { - if (auto* nullable_col = check_and_get_column(col)) { - actual_columns.push_back(&nullable_col->get_nested_column()); - null_maps.push_back(&nullable_col->get_null_map_column()); - } else { - actual_columns.push_back(col); - null_maps.push_back(nullptr); - } - } - } - - /// Return the columns which actually contain the values of the keys. - /// For a given key column, if it is nullable, we return its nested - /// column. Otherwise we return the key column itself. - const ColumnRawPtrs& get_actual_columns() const { return actual_columns; } - - const ColumnRawPtrs& get_nullmap_columns() const { return null_maps; } - -private: - ColumnRawPtrs actual_columns; - ColumnRawPtrs null_maps; -}; - -/// Case where nullable keys are not supported. -template -class BaseStateKeysFixed { -protected: - BaseStateKeysFixed(const ColumnRawPtrs& columns) : actual_columns(columns) {} - - const ColumnRawPtrs& get_actual_columns() const { return actual_columns; } - - const ColumnRawPtrs& get_nullmap_columns() const { return null_maps; } - -private: - ColumnRawPtrs actual_columns; - ColumnRawPtrs null_maps; -}; - } // namespace columns_hashing_impl } // namespace ColumnsHashing diff --git a/be/src/vec/common/hash_table/hash_map_context.h b/be/src/vec/common/hash_table/hash_map_context.h index 973f04f064fea34..5354155c5298e83 100644 --- a/be/src/vec/common/hash_table/hash_map_context.h +++ b/be/src/vec/common/hash_table/hash_map_context.h @@ -375,7 +375,7 @@ struct MethodOneNumber : public MethodBase { } }; -template +template struct MethodKeysFixed : public MethodBase { using Base = MethodBase; using typename Base::Key; @@ -384,8 +384,7 @@ struct MethodKeysFixed : public MethodBase { using Base::hash_table; using Base::iterator; - using State = ColumnsHashing::HashMethodKeysFixed; + using State = ColumnsHashing::HashMethodKeysFixed; // need keep until the hash probe end. use only in join std::vector build_stored_keys; @@ -469,20 +468,22 @@ struct MethodKeysFixed : public MethodBase { bool is_build = false, uint32_t bucket_size = 0) override { ColumnRawPtrs actual_columns; ColumnRawPtrs null_maps; - if (has_nullable_keys) { - actual_columns.reserve(key_columns.size()); - null_maps.reserve(key_columns.size()); - for (const auto& col : key_columns) { - if (const auto* nullable_col = check_and_get_column(col)) { - actual_columns.push_back(&nullable_col->get_nested_column()); - null_maps.push_back(&nullable_col->get_null_map_column()); - } else { - actual_columns.push_back(col); - null_maps.push_back(nullptr); - } + actual_columns.reserve(key_columns.size()); + null_maps.reserve(key_columns.size()); + bool has_nullable_key = false; + + for (const auto& col : key_columns) { + if (const auto* nullable_col = check_and_get_column(col)) { + actual_columns.push_back(&nullable_col->get_nested_column()); + null_maps.push_back(&nullable_col->get_null_map_column()); + has_nullable_key = true; + } else { + actual_columns.push_back(col); + null_maps.push_back(nullptr); } - } else { - actual_columns = key_columns; + } + if (!has_nullable_key) { + null_maps.clear(); } if (is_build) { @@ -503,7 +504,13 @@ struct MethodKeysFixed : public MethodBase { void insert_keys_into_columns(std::vector& input_keys, MutableColumns& key_columns, const size_t num_rows) override { // In any hash key value, column values to be read start just after the bitmap, if it exists. - size_t pos = has_nullable_keys ? get_bitmap_size(key_columns.size()) : 0; + size_t pos = 0; + for (size_t i = 0; i < key_columns.size(); ++i) { + if (key_columns[i]->is_nullable()) { + pos = get_bitmap_size(key_columns.size()); + break; + } + } for (size_t i = 0; i < key_columns.size(); ++i) { size_t size = key_sizes[i]; @@ -607,10 +614,4 @@ struct MethodSingleNullableColumn : public SingleColumnMethod { } }; -template -using PrimaryTypeHashTableContext = MethodOneNumber>>; - -template -using FixedKeyHashTableContext = MethodKeysFixed>, has_null>; - } // namespace doris::vectorized \ No newline at end of file diff --git a/be/src/vec/common/hash_table/hash_map_util.h b/be/src/vec/common/hash_table/hash_map_util.h index 292e6307851c58d..d949fafecf95025 100644 --- a/be/src/vec/common/hash_table/hash_map_util.h +++ b/be/src/vec/common/hash_table/hash_map_util.h @@ -36,11 +36,7 @@ Status init_hash_method(DataVariants* data, const std::vectortemplate init(data_types, type); - } else { - data->template init(data_types, type); - } + data->init(data_types, type); } catch (const Exception& e) { // method_variant may meet valueless_by_exception, so we set it to monostate data->method_variant.template emplace(); @@ -58,15 +54,15 @@ Status init_hash_method(DataVariants* data, const std::vector typename MethodNullable, template typename MethodOneNumber, - template typename MethodFixed, template typename DataNullable> + template typename DataNullable> struct DataVariants { DataVariants() = default; DataVariants(const DataVariants&) = delete; DataVariants& operator=(const DataVariants&) = delete; MethodVariants method_variant; - template - void emplace_single() { + template + void emplace_single(bool nullable) { if (nullable) { method_variant.template emplace>>>(); } else { From 77ac531786eceb270d394c468112d615f4a8143f Mon Sep 17 00:00:00 2001 From: "Mingyu Chen (Rayner)" Date: Fri, 1 Nov 2024 18:07:16 +0800 Subject: [PATCH 39/82] [fix](export) fix concurrent modification issue with export job (#43051) ### What problem does this PR solve? Related PR: #42950 Problem Summary: PR #42950 change some logic in ExportJob, by removing the `taskIdToExecutor`, which is a thread safe ConcurrentHashMap. But there is a problem that, when cancelling a export job, it will clear the `jobExecutorList` in ExportJob, and meanwhile, this `jobExecutorList` may being traversed when creating the export job, causing concurrent modification exception. This PR fix it by locking the writeLock of ExportMgr when cancelling the export job. --- .../java/org/apache/doris/load/ExportMgr.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java index 5636f1aaad3e719..80f738a4cdf9bd4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java @@ -67,8 +67,8 @@ public class ExportMgr { // dbid ->