From f06765fd68c9ab4b3787a4ab4495becf7afdf5b5 Mon Sep 17 00:00:00 2001 From: zzwwhh <30350667+iwanttobepowerful@users.noreply.github.com> Date: Thu, 11 Apr 2024 16:45:20 +0800 Subject: [PATCH 01/71] [fix](nereids) do not transpose semi join agg when mark join (#32475) --- .../rules/rewrite/TransposeSemiJoinAgg.java | 1 + .../rewrite/TransposeSemiJoinAggProject.java | 1 + .../transposeJoin/transposeSemiJoinAgg.out | 8 ++++++++ .../transposeJoin/transposeSemiJoinAgg.groovy | 16 ++++++++++++++++ 4 files changed, 26 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/TransposeSemiJoinAgg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/TransposeSemiJoinAgg.java index 9ae2e14be56506..4006d614ca8b37 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/TransposeSemiJoinAgg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/TransposeSemiJoinAgg.java @@ -36,6 +36,7 @@ public Rule build() { return logicalJoin(logicalAggregate(), any()) .whenNot(join -> ConnectContext.get().getSessionVariable().isDisableJoinReorder()) .when(join -> join.getJoinType().isLeftSemiOrAntiJoin()) + .whenNot(join -> join.isMarkJoin()) .then(join -> { LogicalAggregate aggregate = join.left(); if (!canTranspose(aggregate, join)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/TransposeSemiJoinAggProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/TransposeSemiJoinAggProject.java index 9c6b355e465f50..17ca8f71395a69 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/TransposeSemiJoinAggProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/TransposeSemiJoinAggProject.java @@ -33,6 +33,7 @@ public Rule build() { return logicalJoin(logicalProject(logicalAggregate()), any()) .whenNot(join -> ConnectContext.get().getSessionVariable().isDisableJoinReorder()) .when(join -> join.getJoinType().isLeftSemiOrAntiJoin()) + .whenNot(join -> join.isMarkJoin()) .when(join -> join.left().isAllSlots()) .then(join -> { LogicalProject> project = join.left(); diff --git a/regression-test/data/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.out b/regression-test/data/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.out index d71c15d216fa13..1b9ec434894e19 100644 --- a/regression-test/data/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.out +++ b/regression-test/data/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.out @@ -79,3 +79,11 @@ PhysicalResultSink ----filter((T2.__DORIS_DELETE_SIGN__ = 0)) ------PhysicalOlapScan[T2] +-- !groupby_negative_case3 -- +PhysicalResultSink +--hashJoin[LEFT_SEMI_JOIN] hashCondition=() otherCondition=() markCondition=((T3.len = T3.len)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalOlapScan[T3] +----PhysicalOlapScan[T3] + diff --git a/regression-test/suites/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.groovy b/regression-test/suites/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.groovy index a0e411073fe4cf..305a03f067cae8 100644 --- a/regression-test/suites/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.groovy +++ b/regression-test/suites/nereids_rules_p0/transposeJoin/transposeSemiJoinAgg.groovy @@ -78,6 +78,17 @@ suite("transposeSemiJoinAgg") { sql ''' alter table T2 modify column a set stats ('ndv'='100', 'num_nulls'='0', 'row_count'='100'); ''' + + sql "drop table if exists T3;" + sql """ + CREATE TABLE T3 ( + str varchar(100), + len int + ) DUPLICATE KEY(str) + DISTRIBUTED BY HASH(str) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + // RULE: TransposeSemiJoinAggProject // 1. group-by(without grouping sets) // agg-leftSemi => leftSemi-agg @@ -146,4 +157,9 @@ suite("transposeSemiJoinAgg") { from (select sum(C) as D from T1 group by grouping sets ((a, b), (a), ())) T3 left semi join T2 on T3.D=T2.a; """ + // https://github.com/apache/doris/issues/31308 + qt_groupby_negative_case3 """ + explain shape plan + select case when len in (select len from T3) then 1 else 1 end c1 from T3 group by len; + """ } \ No newline at end of file From 4e62d50c9241376edb4f4983218a28d4177b2745 Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Thu, 11 Apr 2024 16:47:07 +0800 Subject: [PATCH 02/71] (Chore)[regression-test] fix unstable output variant case (#33520) --- regression-test/data/variant_p0/compaction_sparse_column.out | 4 ++-- .../suites/variant_p0/compaction_sparse_column.groovy | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/regression-test/data/variant_p0/compaction_sparse_column.out b/regression-test/data/variant_p0/compaction_sparse_column.out index 3b2ee1dbb9705e..1cc64db5fe989e 100644 --- a/regression-test/data/variant_p0/compaction_sparse_column.out +++ b/regression-test/data/variant_p0/compaction_sparse_column.out @@ -45,7 +45,7 @@ 4096 -- !select_all_bfcompact -- -3 {"a":1234,"point":1,"xxxx":"ddddd"} +3 1234 \N ddddd 1 \N -- !select_b -- 12291 @@ -93,5 +93,5 @@ 4096 -- !select_all -- -3 {"a":1234,"xxxx":"ddddd","point":1} +3 1234 \N ddddd 1 \N diff --git a/regression-test/suites/variant_p0/compaction_sparse_column.groovy b/regression-test/suites/variant_p0/compaction_sparse_column.groovy index 366b5e9eae9527..be3c003d1c8659 100644 --- a/regression-test/suites/variant_p0/compaction_sparse_column.groovy +++ b/regression-test/suites/variant_p0/compaction_sparse_column.groovy @@ -100,7 +100,7 @@ suite("test_compaction_sparse_column", "nonConcurrent") { qt_select_4_1_bfcompact """ SELECT count(cast(v['b'] as int)) FROM ${tableName} where cast(v['b'] as int) = 42003;""" qt_select_5_1_bfcompact """ SELECT count(cast(v['b'] as int)) FROM ${tableName} where cast(v['b'] as int) = 42004;""" qt_select_6_1_bfcompact """ SELECT count(cast(v['b'] as int)) FROM ${tableName} where cast(v['b'] as int) = 42005;""" - qt_select_all_bfcompact """SELECT * from ${tableName} where (cast(v['point'] as int) = 1);""" + qt_select_all_bfcompact """SELECT k, v['a'], v['b'], v['xxxx'], v['point'], v['ddddd'] from ${tableName} where (cast(v['point'] as int) = 1);""" //TabletId,ReplicaId,BackendId,SchemaHash,Version,LstSuccessVersion,LstFailedVersion,LstFailedTime,LocalDataSize,RemoteDataSize,RowCount,State,LstConsistencyCheckTime,CheckVersion,VersionCount,PathHash,MetaUrl,CompactionStatus def tablets = sql_return_maparray """ show tablets from ${tableName}; """ @@ -166,7 +166,7 @@ suite("test_compaction_sparse_column", "nonConcurrent") { qt_select_4_1 """ SELECT count(cast(v['b'] as int)) FROM ${tableName} where cast(v['b'] as int) = 42003;""" qt_select_5_1 """ SELECT count(cast(v['b'] as int)) FROM ${tableName} where cast(v['b'] as int) = 42004;""" qt_select_6_1 """ SELECT count(cast(v['b'] as int)) FROM ${tableName} where cast(v['b'] as int) = 42005;""" - qt_select_all """SELECT * from ${tableName} where (cast(v['point'] as int) = 1);""" + qt_select_all """SELECT k, v['a'], v['b'], v['xxxx'], v['point'], v['ddddd'] from ${tableName} where (cast(v['point'] as int) = 1);""" } finally { // try_sql("DROP TABLE IF EXISTS ${tableName}") } From c2d3bc162c708fb8651194cb805d0104eeb36fac Mon Sep 17 00:00:00 2001 From: Gabriel Date: Thu, 11 Apr 2024 17:51:49 +0800 Subject: [PATCH 03/71] Revert "[refactor](pipelineX) Reduce prepare overhead (#33421)" (#33542) This reverts commit f56ce5a83ab46fc748fb90c5ba91329c3118f5a9. --- .../exec/aggregation_sink_operator.cpp | 46 ++++++------ .../exec/aggregation_source_operator.cpp | 2 +- .../pipeline/exec/analytic_sink_operator.cpp | 14 ++-- be/src/pipeline/exec/analytic_sink_operator.h | 1 - .../exec/analytic_source_operator.cpp | 14 ++-- .../pipeline/exec/analytic_source_operator.h | 1 - ...istinct_streaming_aggregation_operator.cpp | 19 ++--- .../distinct_streaming_aggregation_operator.h | 1 - .../pipeline/exec/exchange_sink_operator.cpp | 47 +++++------- be/src/pipeline/exec/exchange_sink_operator.h | 12 ++-- .../exec/exchange_source_operator.cpp | 14 ++-- be/src/pipeline/exec/hashjoin_build_sink.cpp | 71 +++++++++---------- .../pipeline/exec/hashjoin_probe_operator.cpp | 43 ++++++----- be/src/pipeline/exec/join_probe_operator.cpp | 17 ++--- be/src/pipeline/exec/join_probe_operator.h | 1 - .../exec/multi_cast_data_stream_source.cpp | 21 ++---- .../exec/multi_cast_data_stream_source.h | 6 +- .../exec/nested_loop_join_build_operator.cpp | 4 +- .../exec/nested_loop_join_build_operator.h | 2 +- .../exec/nested_loop_join_probe_operator.cpp | 10 +-- .../exec/nested_loop_join_probe_operator.h | 1 - be/src/pipeline/exec/olap_scan_operator.cpp | 2 +- .../exec/olap_table_sink_operator.cpp | 9 +++ .../pipeline/exec/olap_table_sink_operator.h | 6 ++ .../exec/olap_table_sink_v2_operator.cpp | 9 +++ .../exec/olap_table_sink_v2_operator.h | 7 ++ .../exec/partition_sort_sink_operator.cpp | 17 ++--- .../exec/partition_sort_sink_operator.h | 1 - .../exec/partition_sort_source_operator.cpp | 2 +- .../partitioned_aggregation_sink_operator.cpp | 2 +- ...artitioned_aggregation_source_operator.cpp | 2 +- .../partitioned_hash_join_probe_operator.cpp | 4 -- .../partitioned_hash_join_sink_operator.cpp | 4 -- be/src/pipeline/exec/repeat_operator.cpp | 4 +- be/src/pipeline/exec/repeat_operator.h | 2 +- be/src/pipeline/exec/result_sink_operator.cpp | 9 ++- be/src/pipeline/exec/result_sink_operator.h | 4 +- be/src/pipeline/exec/scan_operator.cpp | 23 +++--- be/src/pipeline/exec/schema_scan_operator.cpp | 5 +- .../pipeline/exec/set_probe_sink_operator.cpp | 18 ++--- .../pipeline/exec/set_probe_sink_operator.h | 1 - be/src/pipeline/exec/set_sink_operator.cpp | 28 +++----- be/src/pipeline/exec/set_sink_operator.h | 1 - be/src/pipeline/exec/set_source_operator.cpp | 4 +- be/src/pipeline/exec/sort_sink_operator.cpp | 14 ++-- be/src/pipeline/exec/sort_sink_operator.h | 5 +- .../exec/spill_sort_sink_operator.cpp | 9 +-- .../pipeline/exec/spill_sort_sink_operator.h | 1 + .../exec/spill_sort_source_operator.cpp | 4 -- .../exec/streaming_aggregation_operator.cpp | 28 +++----- .../exec/streaming_aggregation_operator.h | 1 - .../pipeline/exec/table_function_operator.cpp | 6 +- .../pipeline/exec/table_function_operator.h | 2 +- be/src/pipeline/exec/union_sink_operator.cpp | 12 +--- be/src/pipeline/exec/union_sink_operator.h | 1 - .../pipeline/exec/union_source_operator.cpp | 17 ++--- be/src/pipeline/exec/union_source_operator.h | 1 - .../local_exchange_sink_operator.cpp | 9 +-- .../local_exchange_sink_operator.h | 1 - .../local_exchange_source_operator.cpp | 10 +-- .../local_exchange_source_operator.h | 1 - be/src/pipeline/pipeline_x/operator.cpp | 42 +++++------ be/src/pipeline/pipeline_x/operator.h | 5 +- .../pipeline/pipeline_x/pipeline_x_task.cpp | 15 ++-- be/src/vec/sink/vdata_stream_sender.cpp | 70 +----------------- be/src/vec/sink/vdata_stream_sender.h | 7 +- be/src/vec/sink/volap_table_sink.cpp | 8 ++- be/src/vec/sink/volap_table_sink.h | 5 ++ be/src/vec/sink/volap_table_sink_v2.cpp | 8 ++- be/src/vec/sink/volap_table_sink_v2.h | 4 ++ be/src/vec/sink/writer/vtablet_writer.cpp | 6 +- be/src/vec/sink/writer/vtablet_writer.h | 2 + be/src/vec/sink/writer/vtablet_writer_v2.cpp | 6 +- be/src/vec/sink/writer/vtablet_writer_v2.h | 2 + 74 files changed, 325 insertions(+), 488 deletions(-) diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index e29d6de2860895..6e0042da7d2631 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -63,9 +63,22 @@ AggSinkLocalState::AggSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_init_timer); + SCOPED_TIMER(Base::_open_timer); _agg_data = Base::_shared_state->agg_data.get(); _agg_arena_pool = Base::_shared_state->agg_arena_pool.get(); + auto& p = Base::_parent->template cast(); + Base::_shared_state->align_aggregate_states = p._align_aggregate_states; + Base::_shared_state->total_size_of_aggregate_states = p._total_size_of_aggregate_states; + Base::_shared_state->offsets_of_aggregate_states = p._offsets_of_aggregate_states; + Base::_shared_state->make_nullable_keys = p._make_nullable_keys; + for (auto& evaluator : p._aggregate_evaluators) { + Base::_shared_state->aggregate_evaluators.push_back(evaluator->clone(state, p._pool)); + } + Base::_shared_state->probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); + for (size_t i = 0; i < Base::_shared_state->probe_expr_ctxs.size(); i++) { + RETURN_IF_ERROR( + p._probe_expr_ctxs[i]->clone(state, Base::_shared_state->probe_expr_ctxs[i])); + } _hash_table_memory_usage = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "HashTable", TUnit::BYTES, "MemoryUsage", 1); _serialize_key_arena_memory_usage = Base::profile()->AddHighWaterMarkCounter( @@ -82,30 +95,12 @@ Status AggSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); _max_row_size_counter = ADD_COUNTER(Base::profile(), "MaxRowSizeInBytes", TUnit::UNIT); + COUNTER_SET(_max_row_size_counter, (int64_t)0); - return Status::OK(); -} - -Status AggSinkLocalState::open(RuntimeState* state) { - SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_open_timer); - RETURN_IF_ERROR(Base::open(state)); - auto& p = Base::_parent->template cast(); - Base::_shared_state->align_aggregate_states = p._align_aggregate_states; - Base::_shared_state->total_size_of_aggregate_states = p._total_size_of_aggregate_states; - Base::_shared_state->offsets_of_aggregate_states = p._offsets_of_aggregate_states; - Base::_shared_state->make_nullable_keys = p._make_nullable_keys; - for (auto& evaluator : p._aggregate_evaluators) { - Base::_shared_state->aggregate_evaluators.push_back(evaluator->clone(state, p._pool)); - } - Base::_shared_state->probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); - for (size_t i = 0; i < Base::_shared_state->probe_expr_ctxs.size(); i++) { - RETURN_IF_ERROR( - p._probe_expr_ctxs[i]->clone(state, Base::_shared_state->probe_expr_ctxs[i])); - } for (auto& evaluator : Base::_shared_state->aggregate_evaluators) { evaluator->set_timer(_merge_timer, _expr_timer); } + Base::_shared_state->agg_profile_arena = std::make_unique(); if (Base::_shared_state->probe_expr_ctxs.empty()) { @@ -144,6 +139,15 @@ Status AggSinkLocalState::open(RuntimeState* state) { (!p._have_conjuncts) && // no having conjunct p._needs_finalize; // agg's finalize step } + + return Status::OK(); +} + +Status AggSinkLocalState::open(RuntimeState* state) { + SCOPED_TIMER(Base::exec_time_counter()); + SCOPED_TIMER(Base::_open_timer); + RETURN_IF_ERROR(Base::open(state)); + _agg_data = Base::_shared_state->agg_data.get(); // move _create_agg_status to open not in during prepare, // because during prepare and open thread is not the same one, // this could cause unable to get JVM diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index cff6f9fec42fe9..9c47f1c8cb3800 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -625,7 +625,7 @@ Status AggLocalState::close(RuntimeState* state) { } /// _hash_table_size_counter may be null if prepare failed. - if (_hash_table_size_counter && _shared_state->ready_to_execute) { + if (_hash_table_size_counter) { std::visit( [&](auto&& agg_method) { COUNTER_SET(_hash_table_size_counter, int64_t(agg_method.hash_table->size())); diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index a1d3384edc6dde..26f0b0812f731d 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -29,21 +29,15 @@ OPERATOR_CODE_GENERATOR(AnalyticSinkOperator, StreamingOperator) Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); - _blocks_memory_usage = - _profile->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage", 1); - _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); - return Status::OK(); -} - -Status AnalyticSinkLocalState::open(RuntimeState* state) { - RETURN_IF_ERROR(PipelineXSinkLocalState::open(state)); - SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); _shared_state->partition_by_column_idxs.resize(p._partition_by_eq_expr_ctxs.size()); _shared_state->ordey_by_column_idxs.resize(p._order_by_eq_expr_ctxs.size()); + _blocks_memory_usage = + _profile->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage", 1); + _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); + size_t agg_size = p._agg_expr_ctxs.size(); _agg_expr_ctxs.resize(agg_size); _shared_state->agg_input_columns.resize(agg_size); diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index 3ae4a7b5cff5ca..e04259a0fc4bc7 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -55,7 +55,6 @@ class AnalyticSinkLocalState : public PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; - Status open(RuntimeState* state) override; private: friend class AnalyticSinkOperatorX; diff --git a/be/src/pipeline/exec/analytic_source_operator.cpp b/be/src/pipeline/exec/analytic_source_operator.cpp index f6658583d4657a..d8befd152a2793 100644 --- a/be/src/pipeline/exec/analytic_source_operator.cpp +++ b/be/src/pipeline/exec/analytic_source_operator.cpp @@ -161,22 +161,16 @@ bool AnalyticLocalState::_whether_need_next_partition( Status AnalyticLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); - _blocks_memory_usage = - profile()->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage", 1); - _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); - return Status::OK(); -} - -Status AnalyticLocalState::open(RuntimeState* state) { - RETURN_IF_ERROR(PipelineXLocalState::open(state)); - SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); _agg_arena_pool = std::make_unique(); auto& p = _parent->cast(); _agg_functions_size = p._agg_functions.size(); + _blocks_memory_usage = + profile()->AddHighWaterMarkCounter("Blocks", TUnit::BYTES, "MemoryUsage", 1); + _evaluation_timer = ADD_TIMER(profile(), "EvaluationTime"); + _agg_functions.resize(p._agg_functions.size()); for (size_t i = 0; i < _agg_functions.size(); i++) { _agg_functions[i] = p._agg_functions[i]->clone(state, state->obj_pool()); diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 17a4d34ec739b3..cdfe2644f456c2 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -53,7 +53,6 @@ class AnalyticLocalState final : public PipelineXLocalState AnalyticLocalState(RuntimeState* state, OperatorXBase* parent); Status init(RuntimeState* state, LocalStateInfo& info) override; - Status open(RuntimeState* state) override; Status close(RuntimeState* state) override; void init_result_columns(); diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp index 2c6ff561bedeed..7983b269488eca 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp @@ -68,19 +68,6 @@ DistinctStreamingAggLocalState::DistinctStreamingAggLocalState(RuntimeState* sta Status DistinctStreamingAggLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_init_timer); - _build_timer = ADD_TIMER(Base::profile(), "BuildTime"); - _exec_timer = ADD_TIMER(Base::profile(), "ExecTime"); - _hash_table_compute_timer = ADD_TIMER(Base::profile(), "HashTableComputeTime"); - _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); - _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); - - return Status::OK(); -} - -Status DistinctStreamingAggLocalState::open(RuntimeState* state) { - RETURN_IF_ERROR(Base::open(state)); - SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_open_timer); auto& p = Base::_parent->template cast(); for (auto& evaluator : p._aggregate_evaluators) { @@ -91,6 +78,12 @@ Status DistinctStreamingAggLocalState::open(RuntimeState* state) { RETURN_IF_ERROR(p._probe_expr_ctxs[i]->clone(state, _probe_expr_ctxs[i])); } + _build_timer = ADD_TIMER(Base::profile(), "BuildTime"); + _exec_timer = ADD_TIMER(Base::profile(), "ExecTime"); + _hash_table_compute_timer = ADD_TIMER(Base::profile(), "HashTableComputeTime"); + _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); + _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); + if (_probe_expr_ctxs.empty()) { _agg_data->without_key = reinterpret_cast( _agg_profile_arena->alloc(p._total_size_of_aggregate_states)); diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h index 00f80693f8d004..4c15194362e193 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h @@ -43,7 +43,6 @@ class DistinctStreamingAggLocalState final : public PipelineXLocalStatecast()._transfer_large_data_by_brpc; } -static const std::string timer_name = "WaitForDependencyTime"; - Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); _sender_id = info.sender_id; _bytes_sent_counter = ADD_COUNTER(_profile, "BytesSent", TUnit::BYTES); @@ -127,6 +125,7 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf ""); _merge_block_timer = ADD_TIMER(profile(), "MergeBlockTime"); _local_bytes_send_counter = ADD_COUNTER(_profile, "LocalBytesSent", TUnit::BYTES); + static const std::string timer_name = "WaitForDependencyTime"; _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL(_profile, timer_name, 1); _wait_queue_timer = ADD_CHILD_TIMER_WITH_LEVEL(_profile, "WaitForRpcBufferQueue", timer_name, 1); @@ -151,24 +150,9 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf } SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); - // Make sure brpc stub is ready before execution. - for (int i = 0; i < channels.size(); ++i) { - RETURN_IF_ERROR(channels[i]->init_stub(state)); - } - return Status::OK(); -} - -Status ExchangeSinkLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); - auto& p = _parent->cast(); - _part_type = p._part_type; - SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get()); - int local_size = 0; for (int i = 0; i < channels.size(); ++i) { - RETURN_IF_ERROR(channels[i]->open(state)); + RETURN_IF_ERROR(channels[i]->init(state)); if (channels[i]->is_local()) { local_size++; } @@ -207,13 +191,15 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { ADD_CHILD_TIMER(_profile, "WaitForBroadcastBuffer", timer_name); } else if (local_size > 0) { size_t dep_id = 0; + _local_channels_dependency.resize(local_size); + _wait_channel_timer.resize(local_size); for (auto* channel : channels) { if (channel->is_local()) { - _local_channels_dependency.push_back(channel->get_local_channel_dependency()); + _local_channels_dependency[dep_id] = channel->get_local_channel_dependency(); DCHECK(_local_channels_dependency[dep_id] != nullptr); - _wait_channel_timer.push_back(ADD_CHILD_TIMER_WITH_LEVEL( + _wait_channel_timer[dep_id] = ADD_CHILD_TIMER_WITH_LEVEL( _profile, fmt::format("WaitForLocalExchangeBuffer{}", dep_id), timer_name, - 1)); + 1); dep_id++; } } @@ -289,6 +275,12 @@ Status ExchangeSinkLocalState::open(RuntimeState* state) { } _finish_dependency->block(); + + return Status::OK(); +} + +Status ExchangeSinkLocalState::open(RuntimeState* state) { + RETURN_IF_ERROR(Base::open(state)); if (_part_type == TPartitionType::HASH_PARTITIONED || _part_type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED || _part_type == TPartitionType::TABLE_SINK_HASH_PARTITIONED) { @@ -686,10 +678,7 @@ Status ExchangeSinkLocalState::close(RuntimeState* state, Status exec_status) { } SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_close_timer); - if (_queue_dependency) { - COUNTER_UPDATE(_wait_queue_timer, _queue_dependency->watcher_elapse_time()); - } - + COUNTER_UPDATE(_wait_queue_timer, _queue_dependency->watcher_elapse_time()); COUNTER_SET(_wait_for_finish_dependency_timer, _finish_dependency->watcher_elapse_time()); if (_broadcast_dependency) { COUNTER_UPDATE(_wait_broadcast_buffer_timer, _broadcast_dependency->watcher_elapse_time()); @@ -698,10 +687,8 @@ Status ExchangeSinkLocalState::close(RuntimeState* state, Status exec_status) { COUNTER_UPDATE(_wait_channel_timer[i], _local_channels_dependency[i]->watcher_elapse_time()); } - if (_sink_buffer) { - _sink_buffer->update_profile(profile()); - _sink_buffer->close(); - } + _sink_buffer->update_profile(profile()); + _sink_buffer->close(); return Base::close(state, exec_status); } diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h index 9c40242cd030be..17878fc6eadc7a 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.h +++ b/be/src/pipeline/exec/exchange_sink_operator.h @@ -158,7 +158,7 @@ class ExchangeSinkLocalState final : public PipelineXSinkLocalState<> { friend class vectorized::PipChannel; friend class vectorized::BlockSerializer; - std::unique_ptr> _sink_buffer = nullptr; + std::unique_ptr> _sink_buffer; RuntimeProfile::Counter* _serialize_batch_timer = nullptr; RuntimeProfile::Counter* _compress_timer = nullptr; RuntimeProfile::Counter* _brpc_send_timer = nullptr; @@ -268,7 +268,7 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX _texprs; + const std::vector& _texprs; const RowDescriptor& _row_desc; @@ -291,10 +291,10 @@ class ExchangeSinkOperatorX final : public DataSinkOperatorX _pool; diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp index 1d0c2ba3868e80..664e576e1ce341 100644 --- a/be/src/pipeline/exec/exchange_source_operator.cpp +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -69,7 +69,7 @@ std::string ExchangeSourceOperatorX::debug_string(int indentation_level) const { Status ExchangeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); stream_recvr = state->exec_env()->vstream_mgr()->create_recvr( state, p.input_row_desc(), state->fragment_instance_id(), p.node_id(), p.num_senders(), @@ -77,16 +77,19 @@ Status ExchangeLocalState::init(RuntimeState* state, LocalStateInfo& info) { const auto& queues = stream_recvr->sender_queues(); deps.resize(queues.size()); metrics.resize(queues.size()); - static const std::string timer_name = "WaitForDependencyTime"; - _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, timer_name, 1); for (size_t i = 0; i < queues.size(); i++) { deps[i] = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), "SHUFFLE_DATA_DEPENDENCY", state->get_query_ctx()); queues[i]->set_dependency(deps[i]); + } + static const std::string timer_name = "WaitForDependencyTime"; + _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, timer_name, 1); + for (size_t i = 0; i < queues.size(); i++) { metrics[i] = ADD_CHILD_TIMER_WITH_LEVEL(_runtime_profile, fmt::format("WaitForData{}", i), timer_name, 1); } - + RETURN_IF_ERROR(_parent->cast()._vsort_exec_exprs.clone( + state, vsort_exec_exprs)); return Status::OK(); } @@ -94,9 +97,6 @@ Status ExchangeLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); RETURN_IF_ERROR(Base::open(state)); - - RETURN_IF_ERROR(_parent->cast()._vsort_exec_exprs.clone( - state, vsort_exec_exprs)); return Status::OK(); } diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 53309b9b63da41..d4dc1956400f03 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -48,35 +48,23 @@ HashJoinBuildSinkLocalState::HashJoinBuildSinkLocalState(DataSinkOperatorXBase* Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(JoinBuildSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); - _build_blocks_memory_usage = - ADD_CHILD_COUNTER_WITH_LEVEL(profile(), "BuildBlocks", TUnit::BYTES, "MemoryUsage", 1); - _hash_table_memory_usage = - ADD_CHILD_COUNTER_WITH_LEVEL(profile(), "HashTable", TUnit::BYTES, "MemoryUsage", 1); - _build_arena_memory_usage = - profile()->AddHighWaterMarkCounter("BuildKeyArena", TUnit::BYTES, "MemoryUsage", 1); - - // 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_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"); + SCOPED_TIMER(_open_timer); + auto& p = _parent->cast(); + _shared_state->join_op_variants = p._join_op_variants; - _allocate_resource_timer = ADD_TIMER(profile(), "AllocateResourceTime"); + _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; + _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])); + } + _shared_state->build_exprs_size = _build_expr_ctxs.size(); _should_build_hash_table = true; - auto& p = _parent->cast(); - if (p._is_broadcast_join) { - if (state->enable_share_hash_table_for_broadcast_join()) { - _should_build_hash_table = info.task_idx == 0; - } - } - _shared_state->join_op_variants = p._join_op_variants; 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"); CHECK(p._shared_hashtable_controller->should_build_hash_table( @@ -93,23 +81,24 @@ Status HashJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo _dependency->shared_from_this(), _finish_dependency->shared_from_this()); } - return Status::OK(); -} -Status HashJoinBuildSinkLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(JoinBuildSinkLocalState::open(state)); + _build_blocks_memory_usage = + ADD_CHILD_COUNTER_WITH_LEVEL(profile(), "BuildBlocks", TUnit::BYTES, "MemoryUsage", 1); + _hash_table_memory_usage = + ADD_CHILD_COUNTER_WITH_LEVEL(profile(), "HashTable", TUnit::BYTES, "MemoryUsage", 1); + _build_arena_memory_usage = + profile()->AddHighWaterMarkCounter("BuildKeyArena", TUnit::BYTES, "MemoryUsage", 1); - auto& p = _parent->cast(); + // 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_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"); - _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; - _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])); - } - _shared_state->build_exprs_size = _build_expr_ctxs.size(); // Hash Table Init _hash_table_init(state); _runtime_filters.resize(p._runtime_filter_descs.size()); @@ -121,6 +110,14 @@ Status HashJoinBuildSinkLocalState::open(RuntimeState* state) { _runtime_filter_slots = std::make_shared(_build_expr_ctxs, runtime_filters()); + + return Status::OK(); +} + +Status HashJoinBuildSinkLocalState::open(RuntimeState* state) { + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_open_timer); + RETURN_IF_ERROR(JoinBuildSinkLocalState::open(state)); return Status::OK(); } diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index bbe15e18c3360d..878a813ce067af 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -33,7 +33,25 @@ HashJoinProbeLocalState::HashJoinProbeLocalState(RuntimeState* state, OperatorXB Status HashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(JoinProbeLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); + auto& p = _parent->cast(); + _shared_state->probe_ignore_null = p._probe_ignore_null; + _probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); + for (size_t i = 0; i < _probe_expr_ctxs.size(); i++) { + RETURN_IF_ERROR(p._probe_expr_ctxs[i]->clone(state, _probe_expr_ctxs[i])); + } + _other_join_conjuncts.resize(p._other_join_conjuncts.size()); + for (size_t i = 0; i < _other_join_conjuncts.size(); i++) { + RETURN_IF_ERROR(p._other_join_conjuncts[i]->clone(state, _other_join_conjuncts[i])); + } + + _mark_join_conjuncts.resize(p._mark_join_conjuncts.size()); + for (size_t i = 0; i < _mark_join_conjuncts.size(); i++) { + RETURN_IF_ERROR(p._mark_join_conjuncts[i]->clone(state, _mark_join_conjuncts[i])); + } + + _construct_mutable_join_block(); + _probe_column_disguise_null.reserve(_probe_expr_ctxs.size()); _probe_arena_memory_usage = profile()->AddHighWaterMarkCounter("ProbeKeyArena", TUnit::BYTES, "MemoryUsage", 1); // Probe phase @@ -45,8 +63,6 @@ Status HashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) _probe_process_hashtable_timer = ADD_TIMER(profile(), "ProbeWhenProcessHashTableTime"); _process_other_join_conjunct_timer = ADD_TIMER(profile(), "OtherJoinConjunctTime"); _init_probe_side_timer = ADD_TIMER(profile(), "InitProbeSideTime"); - - _shared_state->probe_ignore_null = _parent->cast()._probe_ignore_null; return Status::OK(); } @@ -55,24 +71,8 @@ Status HashJoinProbeLocalState::open(RuntimeState* state) { SCOPED_TIMER(_open_timer); RETURN_IF_ERROR(JoinProbeLocalState::open(state)); - auto& p = _parent->cast(); - _probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); - for (size_t i = 0; i < _probe_expr_ctxs.size(); i++) { - RETURN_IF_ERROR(p._probe_expr_ctxs[i]->clone(state, _probe_expr_ctxs[i])); - } - _other_join_conjuncts.resize(p._other_join_conjuncts.size()); - for (size_t i = 0; i < _other_join_conjuncts.size(); i++) { - RETURN_IF_ERROR(p._other_join_conjuncts[i]->clone(state, _other_join_conjuncts[i])); - } - - _mark_join_conjuncts.resize(p._mark_join_conjuncts.size()); - for (size_t i = 0; i < _mark_join_conjuncts.size(); i++) { - RETURN_IF_ERROR(p._mark_join_conjuncts[i]->clone(state, _mark_join_conjuncts[i])); - } - - _construct_mutable_join_block(); - _probe_column_disguise_null.reserve(_probe_expr_ctxs.size()); _process_hashtable_ctx_variants = std::make_unique(); + auto& p = _parent->cast(); std::visit( [&](auto&& join_op_variants, auto have_other_join_conjunct) { using JoinOpType = std::decay_t; @@ -434,8 +434,7 @@ Status HashJoinProbeLocalState::filter_data_and_build_output(RuntimeState* state } auto output_rows = temp_block->rows(); if (check_rows_count) { - DCHECK(output_rows <= state->batch_size()) - << "Reason: " << output_rows << " > " << state->batch_size(); + DCHECK(output_rows <= state->batch_size()); } { SCOPED_TIMER(_join_filter_timer); diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 31331568aacd6f..c78e5423709cf5 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -28,18 +28,6 @@ template 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); - - return Status::OK(); -} - -template -Status JoinProbeLocalState::open(RuntimeState* state) { - RETURN_IF_ERROR(Base::open(state)); auto& p = Base::_parent->template cast(); // only use in outer join as the bool column to mark for function of `tuple_is_null` if (p._is_outer_join) { @@ -51,6 +39,11 @@ Status JoinProbeLocalState::open(RuntimeState* state) { RETURN_IF_ERROR(p._output_expr_ctxs[i]->clone(state, _output_expr_ctxs[i])); } + _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); + return Status::OK(); } diff --git a/be/src/pipeline/exec/join_probe_operator.h b/be/src/pipeline/exec/join_probe_operator.h index a3d1e0e6548c1b..4072baa72fcbae 100644 --- a/be/src/pipeline/exec/join_probe_operator.h +++ b/be/src/pipeline/exec/join_probe_operator.h @@ -32,7 +32,6 @@ class JoinProbeLocalState : public PipelineXLocalState { using Base = PipelineXLocalState; Status init(RuntimeState* state, LocalStateInfo& info) override; Status close(RuntimeState* state) override; - Status open(RuntimeState* state) override; virtual void add_tuple_is_null_column(vectorized::Block* block) = 0; protected: 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 17db27042c9f66..c40af83bd58ad6 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp @@ -134,28 +134,19 @@ MultiCastDataStreamSourceLocalState::MultiCastDataStreamSourceLocalState(Runtime Status MultiCastDataStreamSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); RETURN_IF_ERROR(RuntimeFilterConsumer::init(state)); - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); - auto& p = _parent->cast(); - _wait_for_rf_timer = ADD_TIMER(_runtime_profile, "WaitForRuntimeFilter"); - // init profile for runtime filter - RuntimeFilterConsumer::_init_profile(profile()); - init_runtime_filter_dependency(_filter_dependencies, p.operator_id(), p.node_id(), - p.get_name() + "_FILTER_DEPENDENCY"); - _shared_state->multi_cast_data_streamer.set_dep_by_sender_idx(p._consumer_id, _dependency); - return Status::OK(); -} - -Status MultiCastDataStreamSourceLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); - RETURN_IF_ERROR(_acquire_runtime_filter(true)); auto& p = _parent->cast(); + _shared_state->multi_cast_data_streamer.set_dep_by_sender_idx(p._consumer_id, _dependency); _output_expr_contexts.resize(p._output_expr_contexts.size()); for (size_t i = 0; i < p._output_expr_contexts.size(); i++) { RETURN_IF_ERROR(p._output_expr_contexts[i]->clone(state, _output_expr_contexts[i])); } + _wait_for_rf_timer = ADD_TIMER(_runtime_profile, "WaitForRuntimeFilter"); + // init profile for runtime filter + RuntimeFilterConsumer::_init_profile(profile()); + init_runtime_filter_dependency(_filter_dependencies, p.operator_id(), p.node_id(), + p.get_name() + "_FILTER_DEPENDENCY"); return Status::OK(); } 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 8d14b4f266bec9..fd1f6f2c033667 100644 --- a/be/src/pipeline/exec/multi_cast_data_stream_source.h +++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h @@ -102,7 +102,11 @@ class MultiCastDataStreamSourceLocalState final : public PipelineXLocalState( parent, state) {} -Status NestedLoopJoinBuildSinkLocalState::open(RuntimeState* state) { - RETURN_IF_ERROR(JoinBuildSinkLocalState::open(state)); +Status NestedLoopJoinBuildSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(JoinBuildSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); diff --git a/be/src/pipeline/exec/nested_loop_join_build_operator.h b/be/src/pipeline/exec/nested_loop_join_build_operator.h index 3c4f670903c670..52f723b13aed15 100644 --- a/be/src/pipeline/exec/nested_loop_join_build_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_build_operator.h @@ -55,7 +55,7 @@ class NestedLoopJoinBuildSinkLocalState final NestedLoopJoinBuildSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state); ~NestedLoopJoinBuildSinkLocalState() = default; - Status open(RuntimeState* state) override; + Status init(RuntimeState* state, LocalSinkStateInfo& info) override; vectorized::VExprContextSPtrs& filter_src_expr_ctxs() { return _filter_src_expr_ctxs; } RuntimeProfile::Counter* runtime_filter_compute_timer() { 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 c7afa2c399c249..271891709b093c 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.cpp @@ -52,14 +52,6 @@ NestedLoopJoinProbeLocalState::NestedLoopJoinProbeLocalState(RuntimeState* state Status NestedLoopJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(JoinProbeLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); - _loop_join_timer = ADD_TIMER(profile(), "LoopGenerateJoin"); - return Status::OK(); -} - -Status NestedLoopJoinProbeLocalState::open(RuntimeState* state) { - RETURN_IF_ERROR(JoinProbeLocalState::open(state)); - SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); _join_conjuncts.resize(p._join_conjuncts.size()); @@ -67,6 +59,8 @@ Status NestedLoopJoinProbeLocalState::open(RuntimeState* state) { RETURN_IF_ERROR(p._join_conjuncts[i]->clone(state, _join_conjuncts[i])); } _construct_mutable_join_block(); + + _loop_join_timer = ADD_TIMER(profile(), "LoopGenerateJoin"); return Status::OK(); } 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 de9f11b437eece..7a8be87d922b90 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.h @@ -66,7 +66,6 @@ class NestedLoopJoinProbeLocalState final block->get_by_position(i).column->assume_mutable()->clear(); \ } Status init(RuntimeState* state, LocalStateInfo& info) override; - Status open(RuntimeState* state) override; Status close(RuntimeState* state) override; template Status generate_join_block_data(RuntimeState* state, JoinOpType& join_op_variants); diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index 4667986c3b3a65..f65d3e85b7b88b 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -398,10 +398,10 @@ TOlapScanNode& OlapScanLocalState::olap_scan_node() const { void OlapScanLocalState::set_scan_ranges(RuntimeState* state, const std::vector& scan_ranges) { - COUNTER_SET(_tablet_counter, (int64_t)scan_ranges.size()); for (auto& scan_range : scan_ranges) { DCHECK(scan_range.scan_range.__isset.palo_scan_range); _scan_ranges.emplace_back(new TPaloScanRange(scan_range.scan_range.palo_scan_range)); + COUNTER_UPDATE(_tablet_counter, 1); } } diff --git a/be/src/pipeline/exec/olap_table_sink_operator.cpp b/be/src/pipeline/exec/olap_table_sink_operator.cpp index faffaf99c112ed..7c9e71da56c587 100644 --- a/be/src/pipeline/exec/olap_table_sink_operator.cpp +++ b/be/src/pipeline/exec/olap_table_sink_operator.cpp @@ -29,6 +29,15 @@ OperatorPtr OlapTableSinkOperatorBuilder::build_operator() { return std::make_shared(this, _sink); } +Status OlapTableSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_open_timer); + auto& p = _parent->cast(); + RETURN_IF_ERROR(_writer->init_properties(p._pool)); + return Status::OK(); +} + Status OlapTableSinkLocalState::close(RuntimeState* state, Status exec_status) { if (Base::_closed) { return Status::OK(); diff --git a/be/src/pipeline/exec/olap_table_sink_operator.h b/be/src/pipeline/exec/olap_table_sink_operator.h index d09004a10f3633..c688660e2621a9 100644 --- a/be/src/pipeline/exec/olap_table_sink_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_operator.h @@ -52,6 +52,12 @@ class OlapTableSinkLocalState final ENABLE_FACTORY_CREATOR(OlapTableSinkLocalState); OlapTableSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : Base(parent, state) {}; + Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + Status open(RuntimeState* state) override { + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_open_timer); + return Base::open(state); + } Status close(RuntimeState* state, Status exec_status) override; friend class OlapTableSinkOperatorX; diff --git a/be/src/pipeline/exec/olap_table_sink_v2_operator.cpp b/be/src/pipeline/exec/olap_table_sink_v2_operator.cpp index 4b31edb091c4c3..0f43111ef55498 100644 --- a/be/src/pipeline/exec/olap_table_sink_v2_operator.cpp +++ b/be/src/pipeline/exec/olap_table_sink_v2_operator.cpp @@ -25,6 +25,15 @@ OperatorPtr OlapTableSinkV2OperatorBuilder::build_operator() { return std::make_shared(this, _sink); } +Status OlapTableSinkV2LocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_open_timer); + auto& p = _parent->cast(); + RETURN_IF_ERROR(_writer->init_properties(p._pool)); + return Status::OK(); +} + Status OlapTableSinkV2LocalState::close(RuntimeState* state, Status exec_status) { if (Base::_closed) { return Status::OK(); diff --git a/be/src/pipeline/exec/olap_table_sink_v2_operator.h b/be/src/pipeline/exec/olap_table_sink_v2_operator.h index 1fcd4716268119..595009cfc946de 100644 --- a/be/src/pipeline/exec/olap_table_sink_v2_operator.h +++ b/be/src/pipeline/exec/olap_table_sink_v2_operator.h @@ -52,6 +52,13 @@ class OlapTableSinkV2LocalState final ENABLE_FACTORY_CREATOR(OlapTableSinkV2LocalState); OlapTableSinkV2LocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : Base(parent, state) {}; + Status init(RuntimeState* state, LocalSinkStateInfo& info) override; + Status open(RuntimeState* state) override { + SCOPED_TIMER(exec_time_counter()); + SCOPED_TIMER(_open_timer); + return Base::open(state); + } + Status close(RuntimeState* state, Status exec_status) override; friend class OlapTableSinkV2OperatorX; diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.cpp b/be/src/pipeline/exec/partition_sort_sink_operator.cpp index 7b0f17335e1d7b..abe2fde555e164 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_sink_operator.cpp @@ -29,20 +29,8 @@ OperatorPtr PartitionSortSinkOperatorBuilder::build_operator() { Status PartitionSortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); - _hash_table_size_counter = ADD_COUNTER(_profile, "HashTableSize", TUnit::UNIT); - _build_timer = ADD_TIMER(_profile, "HashTableBuildTime"); - _selector_block_timer = ADD_TIMER(_profile, "SelectorBlockTime"); - _emplace_key_timer = ADD_TIMER(_profile, "EmplaceKeyTime"); - _passthrough_rows_counter = ADD_COUNTER(_profile, "PassThroughRowsCounter", TUnit::UNIT); - return Status::OK(); -} - -Status PartitionSortSinkLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(PipelineXSinkLocalState::open(state)); auto& p = _parent->cast(); RETURN_IF_ERROR(p._vsort_exec_exprs.clone(state, _vsort_exec_exprs)); _partition_expr_ctxs.resize(p._partition_expr_ctxs.size()); @@ -53,6 +41,11 @@ Status PartitionSortSinkLocalState::open(RuntimeState* state) { _partition_exprs_num = p._partition_exprs_num; _partitioned_data = std::make_unique(); _agg_arena_pool = std::make_unique(); + _hash_table_size_counter = ADD_COUNTER(_profile, "HashTableSize", TUnit::UNIT); + _build_timer = ADD_TIMER(_profile, "HashTableBuildTime"); + _selector_block_timer = ADD_TIMER(_profile, "SelectorBlockTime"); + _emplace_key_timer = ADD_TIMER(_profile, "EmplaceKeyTime"); + _passthrough_rows_counter = ADD_COUNTER(_profile, "PassThroughRowsCounter", TUnit::UNIT); _partition_sort_info = std::make_shared( &_vsort_exec_exprs, p._limit, 0, p._pool, p._is_asc_order, p._nulls_first, p._child_x->row_desc(), state, _profile, p._has_global_limit, p._partition_inner_limit, diff --git a/be/src/pipeline/exec/partition_sort_sink_operator.h b/be/src/pipeline/exec/partition_sort_sink_operator.h index c3468e180fee66..8602b096f516a2 100644 --- a/be/src/pipeline/exec/partition_sort_sink_operator.h +++ b/be/src/pipeline/exec/partition_sort_sink_operator.h @@ -59,7 +59,6 @@ class PartitionSortSinkLocalState : public PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; - Status open(RuntimeState* state) override; private: friend class PartitionSortSinkOperatorX; diff --git a/be/src/pipeline/exec/partition_sort_source_operator.cpp b/be/src/pipeline/exec/partition_sort_source_operator.cpp index 25f6af382ca143..7fd03a11f7ac37 100644 --- a/be/src/pipeline/exec/partition_sort_source_operator.cpp +++ b/be/src/pipeline/exec/partition_sort_source_operator.cpp @@ -32,7 +32,7 @@ OperatorPtr PartitionSortSourceOperatorBuilder::build_operator() { Status PartitionSortSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); _get_sorted_timer = ADD_TIMER(profile(), "GetSortedTime"); return Status::OK(); } diff --git a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp index 81d051e20eb461..4ea531bade0de6 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_aggregation_sink_operator.cpp @@ -37,7 +37,7 @@ Status PartitionedAggSinkLocalState::init(doris::RuntimeState* state, RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_init_timer); + SCOPED_TIMER(Base::_open_timer); _init_counters(); diff --git a/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp b/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp index 24be471278b359..a2484cd6db4ed0 100644 --- a/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/partitioned_aggregation_source_operator.cpp @@ -34,7 +34,7 @@ PartitionedAggLocalState::PartitionedAggLocalState(RuntimeState* state, Operator Status PartitionedAggLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); _init_counters(); return Status::OK(); } diff --git a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp index 3aa4b541ec8fc8..0f837f8bbdad1a 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_probe_operator.cpp @@ -30,8 +30,6 @@ PartitionedHashJoinProbeLocalState::PartitionedHashJoinProbeLocalState(RuntimeSt Status PartitionedHashJoinProbeLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXSpillLocalState::init(state, info)); - SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_init_timer); _internal_runtime_profile.reset(new RuntimeProfile("internal_profile")); auto& p = _parent->cast(); @@ -143,8 +141,6 @@ void PartitionedHashJoinProbeLocalState::update_probe_profile(RuntimeProfile* ch #undef UPDATE_PROFILE Status PartitionedHashJoinProbeLocalState::open(RuntimeState* state) { - SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_open_timer); RETURN_IF_ERROR(PipelineXSpillLocalState::open(state)); return _partitioner->open(state); } diff --git a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp index 21fe5be6426bb0..370606b190454c 100644 --- a/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp +++ b/be/src/pipeline/exec/partitioned_hash_join_sink_operator.cpp @@ -26,8 +26,6 @@ namespace doris::pipeline { Status PartitionedHashJoinSinkLocalState::init(doris::RuntimeState* state, doris::pipeline::LocalSinkStateInfo& info) { RETURN_IF_ERROR(PipelineXSpillSinkLocalState::init(state, info)); - SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_init_timer); auto& p = _parent->cast(); _shared_state->partitioned_build_blocks.resize(p._partition_count); _shared_state->spilled_streams.resize(p._partition_count); @@ -46,8 +44,6 @@ Status PartitionedHashJoinSinkLocalState::init(doris::RuntimeState* state, } Status PartitionedHashJoinSinkLocalState::open(RuntimeState* state) { - SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_open_timer); RETURN_IF_ERROR(PipelineXSpillSinkLocalState::open(state)); return _partitioner->open(state); } diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp index 42d009f0e76b7d..0f9cf93b3f56d3 100644 --- a/be/src/pipeline/exec/repeat_operator.cpp +++ b/be/src/pipeline/exec/repeat_operator.cpp @@ -47,10 +47,10 @@ RepeatLocalState::RepeatLocalState(RuntimeState* state, OperatorXBase* parent) _child_block(vectorized::Block::create_unique()), _repeat_id_idx(0) {} -Status RepeatLocalState::open(RuntimeState* state) { +Status RepeatLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); auto& p = _parent->cast(); _expr_ctxs.resize(p._expr_ctxs.size()); for (size_t i = 0; i < _expr_ctxs.size(); i++) { diff --git a/be/src/pipeline/exec/repeat_operator.h b/be/src/pipeline/exec/repeat_operator.h index 208b3d1e00565c..9cb671fccb0271 100644 --- a/be/src/pipeline/exec/repeat_operator.h +++ b/be/src/pipeline/exec/repeat_operator.h @@ -54,7 +54,7 @@ class RepeatLocalState final : public PipelineXLocalState { using Base = PipelineXLocalState; RepeatLocalState(RuntimeState* state, OperatorXBase* parent); - Status open(RuntimeState* state) override; + Status init(RuntimeState* state, LocalStateInfo& info) override; Status get_repeated_block(vectorized::Block* child_block, int repeat_id_idx, vectorized::Block* output_block); diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp index d0cd130cc8a8be..b89ce4adb2e2d3 100644 --- a/be/src/pipeline/exec/result_sink_operator.cpp +++ b/be/src/pipeline/exec/result_sink_operator.cpp @@ -53,18 +53,17 @@ bool ResultSinkOperator::can_write() { Status ResultSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); 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); - _rows_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "RowsProduced", TUnit::UNIT, 1); - // create sender + std::shared_ptr sender = nullptr; RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender( state->fragment_instance_id(), vectorized::RESULT_SINK_BUFFER_SIZE, &_sender, true, state->execution_timeout())); + _blocks_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "BlocksProduced", TUnit::UNIT, 1); + _rows_sent_counter = ADD_COUNTER_WITH_LEVEL(_profile, "RowsProduced", TUnit::UNIT, 1); ((PipBufferControlBlock*)_sender.get())->set_dependency(_dependency->shared_from_this()); return Status::OK(); } diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h index aed9961a6d6771..564f05eef43659 100644 --- a/be/src/pipeline/exec/result_sink_operator.h +++ b/be/src/pipeline/exec/result_sink_operator.h @@ -62,8 +62,8 @@ class ResultSinkLocalState final : public PipelineXSinkLocalState _sender = nullptr; - std::shared_ptr _writer = nullptr; + std::shared_ptr _sender; + std::shared_ptr _writer; RuntimeProfile::Counter* _blocks_sent_counter = nullptr; RuntimeProfile::Counter* _rows_sent_counter = nullptr; }; diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index 94d07f8c0d6d28..baff3df2d37c2c 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -110,9 +110,19 @@ Status ScanLocalState::init(RuntimeState* state, LocalStateInfo& info) _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL( _runtime_profile, "WaitForDependency[" + _scan_dependency->name() + "]Time", 1); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); RETURN_IF_ERROR(RuntimeFilterConsumer::init(state, p.ignore_data_distribution())); + + _common_expr_ctxs_push_down.resize(p._common_expr_ctxs_push_down.size()); + for (size_t i = 0; i < _common_expr_ctxs_push_down.size(); i++) { + RETURN_IF_ERROR( + p._common_expr_ctxs_push_down[i]->clone(state, _common_expr_ctxs_push_down[i])); + } + _stale_expr_ctxs.resize(p._stale_expr_ctxs.size()); + for (size_t i = 0; i < _stale_expr_ctxs.size(); i++) { + RETURN_IF_ERROR(p._stale_expr_ctxs[i]->clone(state, _stale_expr_ctxs[i])); + } // init profile for runtime filter RuntimeFilterConsumer::_init_profile(profile()); init_runtime_filter_dependency(_filter_dependencies, p.operator_id(), p.node_id(), @@ -139,18 +149,7 @@ Status ScanLocalState::open(RuntimeState* state) { if (_opened) { return Status::OK(); } - RETURN_IF_ERROR(PipelineXLocalState<>::open(state)); - auto& p = _parent->cast(); - _common_expr_ctxs_push_down.resize(p._common_expr_ctxs_push_down.size()); - for (size_t i = 0; i < _common_expr_ctxs_push_down.size(); i++) { - RETURN_IF_ERROR( - p._common_expr_ctxs_push_down[i]->clone(state, _common_expr_ctxs_push_down[i])); - } RETURN_IF_ERROR(_acquire_runtime_filter(true)); - _stale_expr_ctxs.resize(p._stale_expr_ctxs.size()); - for (size_t i = 0; i < _stale_expr_ctxs.size(); i++) { - RETURN_IF_ERROR(p._stale_expr_ctxs[i]->clone(state, _stale_expr_ctxs[i])); - } RETURN_IF_ERROR(_process_conjuncts()); auto status = _eos ? Status::OK() : _prepare_scanners(); diff --git a/be/src/pipeline/exec/schema_scan_operator.cpp b/be/src/pipeline/exec/schema_scan_operator.cpp index 2d32e21d991960..20eec02bc138c5 100644 --- a/be/src/pipeline/exec/schema_scan_operator.cpp +++ b/be/src/pipeline/exec/schema_scan_operator.cpp @@ -48,7 +48,7 @@ Status SchemaScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); _scanner_param.common_param = p._common_scanner_param; // init schema scanner profile @@ -69,9 +69,6 @@ Status SchemaScanLocalState::init(RuntimeState* state, LocalStateInfo& info) { } Status SchemaScanLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(PipelineXLocalState<>::open(state)); return _schema_scanner->start(state); } diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index 9ae40930d5f40a..76248f2c75c97f 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -137,31 +137,23 @@ Status SetProbeSinkOperatorX::sink(RuntimeState* state, vectorized template Status SetProbeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(Base::init(state, info)); + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); Parent& parent = _parent->cast(); _shared_state->probe_finished_children_dependency[parent._cur_child_id] = _dependency; _dependency->block(); - _child_exprs.resize(parent._child_exprs.size()); for (size_t i = 0; i < _child_exprs.size(); i++) { RETURN_IF_ERROR(parent._child_exprs[i]->clone(state, _child_exprs[i])); } + auto& child_exprs_lists = _shared_state->child_exprs_lists; child_exprs_lists[parent._cur_child_id] = _child_exprs; - return Status::OK(); -} - -template -Status SetProbeSinkLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); // Add the if check only for compatible with old optimiser - if (_shared_state->child_quantity > 1) { - _probe_columns.resize(_child_exprs.size()); + if (child_exprs_lists.size() > 1) { + _probe_columns.resize(child_exprs_lists[1].size()); } return Status::OK(); } diff --git a/be/src/pipeline/exec/set_probe_sink_operator.h b/be/src/pipeline/exec/set_probe_sink_operator.h index 9f80f03966b1f1..6b4197ea94b890 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.h +++ b/be/src/pipeline/exec/set_probe_sink_operator.h @@ -81,7 +81,6 @@ class SetProbeSinkLocalState final : public PipelineXSinkLocalStatevalid_element_in_hash_tbl; } private: diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index 3b6de314060efb..7ef4871555d303 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -162,38 +162,28 @@ template Status SetSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); _build_timer = ADD_TIMER(_profile, "BuildTime"); + auto& parent = _parent->cast(); _shared_state->probe_finished_children_dependency[parent._cur_child_id] = _dependency; - DCHECK(parent._cur_child_id == 0); - auto& child_exprs_lists = _shared_state->child_exprs_lists; - DCHECK(child_exprs_lists.empty() || child_exprs_lists.size() == parent._child_quantity); - if (child_exprs_lists.empty()) { - child_exprs_lists.resize(parent._child_quantity); - } _child_exprs.resize(parent._child_exprs.size()); for (size_t i = 0; i < _child_exprs.size(); i++) { RETURN_IF_ERROR(parent._child_exprs[i]->clone(state, _child_exprs[i])); } - child_exprs_lists[parent._cur_child_id] = _child_exprs; - _shared_state->child_quantity = parent._child_quantity; - return Status::OK(); -} -template -Status SetSinkLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(PipelineXSinkLocalState::open(state)); + _shared_state->child_quantity = parent._child_quantity; - auto& parent = _parent->cast(); - DCHECK(parent._cur_child_id == 0); auto& child_exprs_lists = _shared_state->child_exprs_lists; + DCHECK(child_exprs_lists.empty() || child_exprs_lists.size() == parent._child_quantity); + if (child_exprs_lists.empty()) { + child_exprs_lists.resize(parent._child_quantity); + } + child_exprs_lists[parent._cur_child_id] = _child_exprs; _shared_state->hash_table_variants = std::make_unique(); - for (const auto& ctx : child_exprs_lists[parent._cur_child_id]) { + for (const auto& ctx : child_exprs_lists[0]) { _shared_state->build_not_ignore_null.push_back(ctx->root()->is_nullable()); } _shared_state->hash_table_init(); diff --git a/be/src/pipeline/exec/set_sink_operator.h b/be/src/pipeline/exec/set_sink_operator.h index 2a6bb63c02e815..24f23593ea08e6 100644 --- a/be/src/pipeline/exec/set_sink_operator.h +++ b/be/src/pipeline/exec/set_sink_operator.h @@ -73,7 +73,6 @@ class SetSinkLocalState final : public PipelineXSinkLocalState { SetSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : Base(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; - Status open(RuntimeState* state) override; private: friend class SetSinkOperatorX; diff --git a/be/src/pipeline/exec/set_source_operator.cpp b/be/src/pipeline/exec/set_source_operator.cpp index 97ad66a867eebd..15524a25a7b7b3 100644 --- a/be/src/pipeline/exec/set_source_operator.cpp +++ b/be/src/pipeline/exec/set_source_operator.cpp @@ -53,7 +53,7 @@ template Status SetSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); _shared_state->probe_finished_children_dependency.resize( _parent->cast>()._child_quantity, nullptr); return Status::OK(); @@ -63,7 +63,7 @@ template Status SetSourceLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); + RETURN_IF_ERROR(PipelineXLocalState::open(state)); auto& child_exprs_lists = _shared_state->child_exprs_lists; auto output_data_types = vectorized::VectorizedUtils::get_data_types( diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp index d89e54614d14c6..687332e1aec669 100644 --- a/be/src/pipeline/exec/sort_sink_operator.cpp +++ b/be/src/pipeline/exec/sort_sink_operator.cpp @@ -29,18 +29,9 @@ namespace doris::pipeline { OPERATOR_CODE_GENERATOR(SortSinkOperator, StreamingOperator) Status SortSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { - RETURN_IF_ERROR(Base::init(state, info)); - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); - _sort_blocks_memory_usage = - ADD_CHILD_COUNTER_WITH_LEVEL(_profile, "SortBlocks", TUnit::BYTES, "MemoryUsage", 1); - return Status::OK(); -} - -Status SortSinkLocalState::open(RuntimeState* state) { + RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); auto& p = _parent->cast(); RETURN_IF_ERROR(p._vsort_exec_exprs.clone(state, _vsort_exec_exprs)); @@ -71,6 +62,9 @@ Status SortSinkLocalState::open(RuntimeState* state) { _shared_state->sorter->init_profile(_profile); _profile->add_info_string("TOP-N", p._limit == -1 ? "false" : "true"); + + _sort_blocks_memory_usage = + ADD_CHILD_COUNTER_WITH_LEVEL(_profile, "SortBlocks", TUnit::BYTES, "MemoryUsage", 1); return Status::OK(); } diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h index ad9c23401b4c69..f23437f72f9796 100644 --- a/be/src/pipeline/exec/sort_sink_operator.h +++ b/be/src/pipeline/exec/sort_sink_operator.h @@ -51,13 +51,12 @@ class SortSinkOperatorX; class SortSinkLocalState : public PipelineXSinkLocalState { ENABLE_FACTORY_CREATOR(SortSinkLocalState); - using Base = PipelineXSinkLocalState; public: - SortSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) : Base(parent, state) {} + SortSinkLocalState(DataSinkOperatorXBase* parent, RuntimeState* state) + : PipelineXSinkLocalState(parent, state) {} Status init(RuntimeState* state, LocalSinkStateInfo& info) override; - Status open(RuntimeState* state) override; private: friend class SortSinkOperatorX; diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.cpp b/be/src/pipeline/exec/spill_sort_sink_operator.cpp index e83434250e0f19..78c5c9f51e948b 100644 --- a/be/src/pipeline/exec/spill_sort_sink_operator.cpp +++ b/be/src/pipeline/exec/spill_sort_sink_operator.cpp @@ -27,12 +27,11 @@ SpillSortSinkLocalState::SpillSortSinkLocalState(DataSinkOperatorXBase* parent, parent->operator_id(), parent->node_id(), parent->get_name() + "_FINISH_DEPENDENCY", state->get_query_ctx()); } - Status SpillSortSinkLocalState::init(doris::RuntimeState* state, doris::pipeline::LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); _init_counters(); @@ -46,7 +45,6 @@ Status SpillSortSinkLocalState::init(doris::RuntimeState* state, } return Status::OK(); } - void SpillSortSinkLocalState::_init_counters() { _internal_runtime_profile = std::make_unique("internal_profile"); @@ -74,7 +72,10 @@ void SpillSortSinkLocalState::update_profile(RuntimeProfile* child_profile) { UPDATE_PROFILE(_merge_block_timer, "MergeBlockTime"); UPDATE_PROFILE(_sort_blocks_memory_usage, "SortBlocks"); } - +Status SpillSortSinkLocalState::open(RuntimeState* state) { + RETURN_IF_ERROR(Base::open(state)); + return Status::OK(); +} Status SpillSortSinkLocalState::close(RuntimeState* state, Status execsink_status) { auto& parent = Base::_parent->template cast(); if (parent._enable_spill) { diff --git a/be/src/pipeline/exec/spill_sort_sink_operator.h b/be/src/pipeline/exec/spill_sort_sink_operator.h index 4604696eff2011..d66215411aae12 100644 --- a/be/src/pipeline/exec/spill_sort_sink_operator.h +++ b/be/src/pipeline/exec/spill_sort_sink_operator.h @@ -35,6 +35,7 @@ class SpillSortSinkLocalState : public PipelineXSpillSinkLocalState("internal_profile"); _spill_timer = ADD_CHILD_TIMER_WITH_LEVEL(Base::profile(), "SpillMergeSortTime", "Spill", 1); _spill_merge_sort_timer = @@ -51,8 +49,6 @@ Status SpillSortLocalState::init(RuntimeState* state, LocalStateInfo& info) { } Status SpillSortLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); if (_opened) { return Status::OK(); } diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_operator.cpp index dfcfb0ebc450b4..1ec283bdc1f765 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/streaming_aggregation_operator.cpp @@ -83,7 +83,15 @@ StreamingAggLocalState::StreamingAggLocalState(RuntimeState* state, OperatorXBas Status StreamingAggLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_init_timer); + SCOPED_TIMER(Base::_open_timer); + auto& p = Base::_parent->template cast(); + for (auto& evaluator : p._aggregate_evaluators) { + _aggregate_evaluators.push_back(evaluator->clone(state, p._pool)); + } + _probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); + for (size_t i = 0; i < _probe_expr_ctxs.size(); i++) { + RETURN_IF_ERROR(p._probe_expr_ctxs[i]->clone(state, _probe_expr_ctxs[i])); + } _hash_table_memory_usage = ADD_CHILD_COUNTER_WITH_LEVEL(Base::profile(), "HashTable", TUnit::BYTES, "MemoryUsage", 1); _serialize_key_arena_memory_usage = Base::profile()->AddHighWaterMarkCounter( @@ -111,23 +119,7 @@ Status StreamingAggLocalState::init(RuntimeState* state, LocalStateInfo& info) { _serialize_result_timer = ADD_TIMER(profile(), "SerializeResultTime"); _hash_table_iterate_timer = ADD_TIMER(profile(), "HashTableIterateTime"); _insert_keys_to_column_timer = ADD_TIMER(profile(), "InsertKeysToColumnTime"); - - return Status::OK(); -} - -Status StreamingAggLocalState::open(RuntimeState* state) { - SCOPED_TIMER(Base::exec_time_counter()); - SCOPED_TIMER(Base::_open_timer); - RETURN_IF_ERROR(Base::open(state)); - - auto& p = Base::_parent->template cast(); - for (auto& evaluator : p._aggregate_evaluators) { - _aggregate_evaluators.push_back(evaluator->clone(state, p._pool)); - } - _probe_expr_ctxs.resize(p._probe_expr_ctxs.size()); - for (size_t i = 0; i < _probe_expr_ctxs.size(); i++) { - RETURN_IF_ERROR(p._probe_expr_ctxs[i]->clone(state, _probe_expr_ctxs[i])); - } + COUNTER_SET(_max_row_size_counter, (int64_t)0); for (auto& evaluator : _aggregate_evaluators) { evaluator->set_timer(_merge_timer, _expr_timer); diff --git a/be/src/pipeline/exec/streaming_aggregation_operator.h b/be/src/pipeline/exec/streaming_aggregation_operator.h index 2895fc63f39589..1ccb7e31d0f59d 100644 --- a/be/src/pipeline/exec/streaming_aggregation_operator.h +++ b/be/src/pipeline/exec/streaming_aggregation_operator.h @@ -42,7 +42,6 @@ class StreamingAggLocalState final : public PipelineXLocalState ~StreamingAggLocalState() override = default; Status init(RuntimeState* state, LocalStateInfo& info) override; - Status open(RuntimeState* state) override; Status close(RuntimeState* state) override; Status do_pre_agg(vectorized::Block* input_block, vectorized::Block* output_block); void make_nullable_output_key(vectorized::Block* block); diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index b4d993ef035acb..bd4bc3f90d65d8 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -44,10 +44,8 @@ Status TableFunctionOperator::close(doris::RuntimeState* state) { TableFunctionLocalState::TableFunctionLocalState(RuntimeState* state, OperatorXBase* parent) : PipelineXLocalState<>(state, parent), _child_block(vectorized::Block::create_unique()) {} -Status TableFunctionLocalState::open(RuntimeState* state) { - SCOPED_TIMER(PipelineXLocalState<>::exec_time_counter()); - SCOPED_TIMER(PipelineXLocalState<>::_open_timer); - RETURN_IF_ERROR(PipelineXLocalState<>::open(state)); +Status TableFunctionLocalState::init(RuntimeState* state, LocalStateInfo& info) { + RETURN_IF_ERROR(PipelineXLocalState<>::init(state, info)); auto& p = _parent->cast(); _vfn_ctxs.resize(p._vfn_ctxs.size()); for (size_t i = 0; i < _vfn_ctxs.size(); i++) { diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index 49dd242bfe78d9..3379a8f5b4a2d1 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -55,7 +55,7 @@ class TableFunctionLocalState final : public PipelineXLocalState<> { TableFunctionLocalState(RuntimeState* state, OperatorXBase* parent); ~TableFunctionLocalState() override = default; - Status open(RuntimeState* state) override; + Status init(RuntimeState* state, LocalStateInfo& info) override; void process_next_child_row(); Status get_expanded_block(RuntimeState* state, vectorized::Block* output_block, bool* eos); diff --git a/be/src/pipeline/exec/union_sink_operator.cpp b/be/src/pipeline/exec/union_sink_operator.cpp index 5acf6c8e1a29e5..ce1195f042bca9 100644 --- a/be/src/pipeline/exec/union_sink_operator.cpp +++ b/be/src/pipeline/exec/union_sink_operator.cpp @@ -95,24 +95,16 @@ Status UnionSinkOperator::close(RuntimeState* state) { Status UnionSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); - auto& p = _parent->cast(); - _shared_state->data_queue.set_sink_dependency(_dependency, p._cur_child_id); - return Status::OK(); -} - -Status UnionSinkLocalState::open(RuntimeState* state) { SCOPED_TIMER(exec_time_counter()); SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); auto& p = _parent->cast(); _child_expr.resize(p._child_expr.size()); + _shared_state->data_queue.set_sink_dependency(_dependency, p._cur_child_id); for (size_t i = 0; i < p._child_expr.size(); i++) { RETURN_IF_ERROR(p._child_expr[i]->clone(state, _child_expr[i])); } return Status::OK(); -} +}; UnionSinkOperatorX::UnionSinkOperatorX(int child_id, int sink_id, ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs) diff --git a/be/src/pipeline/exec/union_sink_operator.h b/be/src/pipeline/exec/union_sink_operator.h index 97b704078c63ec..6d79d3f2a9fcdf 100644 --- a/be/src/pipeline/exec/union_sink_operator.h +++ b/be/src/pipeline/exec/union_sink_operator.h @@ -73,7 +73,6 @@ class UnionSinkLocalState final : public PipelineXSinkLocalState; using Parent = UnionSinkOperatorX; diff --git a/be/src/pipeline/exec/union_source_operator.cpp b/be/src/pipeline/exec/union_source_operator.cpp index 10f98a8d1cbc49..de0348de508b65 100644 --- a/be/src/pipeline/exec/union_source_operator.cpp +++ b/be/src/pipeline/exec/union_source_operator.cpp @@ -110,7 +110,7 @@ Status UnionSourceOperator::get_block(RuntimeState* state, vectorized::Block* bl Status UnionSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); auto& p = _parent->cast(); if (p.get_child_count() != 0) { ((UnionSharedState*)_dependency->shared_state()) @@ -124,18 +124,6 @@ Status UnionSourceLocalState::init(RuntimeState* state, LocalStateInfo& info) { _runtime_profile, "WaitForDependency[" + _dependency->name() + "]Time", 1); } - if (p.get_child_count() == 0) { - _dependency->set_ready(); - } - return Status::OK(); -} - -Status UnionSourceLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); - - auto& p = _parent->cast(); // Const exprs materialized by this node. These exprs don't refer to any children. // Only materialized by the first fragment instance to avoid duplication. if (state->per_fragment_instance_idx() == 0) { @@ -155,6 +143,9 @@ Status UnionSourceLocalState::open(RuntimeState* state) { } } + if (p.get_child_count() == 0) { + _dependency->set_ready(); + } return Status::OK(); } diff --git a/be/src/pipeline/exec/union_source_operator.h b/be/src/pipeline/exec/union_source_operator.h index 60530521ec0a82..023e6363d48052 100644 --- a/be/src/pipeline/exec/union_source_operator.h +++ b/be/src/pipeline/exec/union_source_operator.h @@ -78,7 +78,6 @@ class UnionSourceLocalState final : public PipelineXLocalState UnionSourceLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent) {}; Status init(RuntimeState* state, LocalStateInfo& info) override; - Status open(RuntimeState* state) override; [[nodiscard]] std::string debug_string(int indentation_level = 0) const override; diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.cpp b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.cpp index e0b47699f7144c..e540aeb54b94a0 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.cpp +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.cpp @@ -24,16 +24,9 @@ namespace doris::pipeline { Status LocalExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_init_timer); + SCOPED_TIMER(_open_timer); _compute_hash_value_timer = ADD_TIMER(profile(), "ComputeHashValueTime"); _distribute_timer = ADD_TIMER(profile(), "DistributeDataTime"); - return Status::OK(); -} - -Status LocalExchangeSinkLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); _exchanger = _shared_state->exchanger.get(); DCHECK(_exchanger != nullptr); diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h index b3ecf29736fda6..b7e0d75465540f 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_sink_operator.h @@ -37,7 +37,6 @@ class LocalExchangeSinkLocalState final : public PipelineXSinkLocalStatemem_trackers[_channel_id] = _mem_tracker.get(); - return Status::OK(); -} - -Status LocalExchangeSourceLocalState::open(RuntimeState* state) { - SCOPED_TIMER(exec_time_counter()); - SCOPED_TIMER(_open_timer); - RETURN_IF_ERROR(Base::open(state)); - _exchanger = _shared_state->exchanger.get(); DCHECK(_exchanger != nullptr); _get_block_failed_counter = diff --git a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h index 7d416b10c19a9c..53e6aef3327a9f 100644 --- a/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h +++ b/be/src/pipeline/pipeline_x/local_exchange/local_exchange_source_operator.h @@ -35,7 +35,6 @@ class LocalExchangeSourceLocalState final : public PipelineXLocalState::init(RuntimeState* state, LocalState } } - _rows_returned_counter = - ADD_COUNTER_WITH_LEVEL(_runtime_profile, "RowsProduced", TUnit::UNIT, 1); - _blocks_returned_counter = - ADD_COUNTER_WITH_LEVEL(_runtime_profile, "BlocksProduced", TUnit::UNIT, 1); - _projection_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "ProjectionTime", 1); - _open_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "OpenTime", 1); - _init_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "InitTime", 1); - _close_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "CloseTime", 1); - _exec_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "ExecTime", 1); - _mem_tracker = std::make_unique("PipelineXLocalState:" + _runtime_profile->name()); - _memory_used_counter = ADD_LABEL_COUNTER_WITH_LEVEL(_runtime_profile, "MemoryUsage", 1); - _peak_memory_usage_counter = _runtime_profile->AddHighWaterMarkCounter( - "PeakMemoryUsage", TUnit::BYTES, "MemoryUsage", 1); - return Status::OK(); -} - -template -Status PipelineXLocalState::open(RuntimeState* state) { _conjuncts.resize(_parent->_conjuncts.size()); _projections.resize(_parent->_projections.size()); for (size_t i = 0; i < _conjuncts.size(); i++) { @@ -428,6 +410,18 @@ Status PipelineXLocalState::open(RuntimeState* state) { } } + _rows_returned_counter = + ADD_COUNTER_WITH_LEVEL(_runtime_profile, "RowsProduced", TUnit::UNIT, 1); + _blocks_returned_counter = + ADD_COUNTER_WITH_LEVEL(_runtime_profile, "BlocksProduced", TUnit::UNIT, 1); + _projection_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "ProjectionTime", 1); + _open_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "OpenTime", 1); + _close_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "CloseTime", 1); + _exec_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, "ExecTime", 1); + _mem_tracker = std::make_unique("PipelineXLocalState:" + _runtime_profile->name()); + _memory_used_counter = ADD_LABEL_COUNTER_WITH_LEVEL(_runtime_profile, "MemoryUsage", 1); + _peak_memory_usage_counter = _runtime_profile->AddHighWaterMarkCounter( + "PeakMemoryUsage", TUnit::BYTES, "MemoryUsage", 1); return Status::OK(); } @@ -474,7 +468,6 @@ Status PipelineXSinkLocalState::init(RuntimeState* state, LocalSink } _rows_input_counter = ADD_COUNTER_WITH_LEVEL(_profile, "InputRows", TUnit::UNIT, 1); _open_timer = ADD_TIMER_WITH_LEVEL(_profile, "OpenTime", 1); - _init_timer = ADD_TIMER_WITH_LEVEL(_profile, "InitTime", 1); _close_timer = ADD_TIMER_WITH_LEVEL(_profile, "CloseTime", 1); _exec_timer = ADD_TIMER_WITH_LEVEL(_profile, "ExecTime", 1); info.parent_profile->add_child(_profile, true, nullptr); @@ -544,10 +537,16 @@ template requires(std::is_base_of_v) Status AsyncWriterSink::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(Base::init(state, info)); + _output_vexpr_ctxs.resize(_parent->cast()._output_vexpr_ctxs.size()); + for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) { + RETURN_IF_ERROR( + _parent->cast()._output_vexpr_ctxs[i]->clone(state, _output_vexpr_ctxs[i])); + } _writer.reset(new Writer(info.tsink, _output_vexpr_ctxs)); _async_writer_dependency = AsyncWriterDependency::create_shared( _parent->operator_id(), _parent->node_id(), state->get_query_ctx()); _writer->set_dependency(_async_writer_dependency.get(), _finish_dependency.get()); + _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL( _profile, "WaitForDependency[" + _async_writer_dependency->name() + "]Time", 1); return Status::OK(); @@ -557,11 +556,6 @@ template requires(std::is_base_of_v) Status AsyncWriterSink::open(RuntimeState* state) { RETURN_IF_ERROR(Base::open(state)); - _output_vexpr_ctxs.resize(_parent->cast()._output_vexpr_ctxs.size()); - for (size_t i = 0; i < _output_vexpr_ctxs.size(); i++) { - RETURN_IF_ERROR( - _parent->cast()._output_vexpr_ctxs[i]->clone(state, _output_vexpr_ctxs[i])); - } RETURN_IF_ERROR(_writer->start_writer(state, _profile)); return Status::OK(); } diff --git a/be/src/pipeline/pipeline_x/operator.h b/be/src/pipeline/pipeline_x/operator.h index b470c9237e8402..7a0a5d121724a6 100644 --- a/be/src/pipeline/pipeline_x/operator.h +++ b/be/src/pipeline/pipeline_x/operator.h @@ -73,7 +73,7 @@ class PipelineXLocalStateBase { virtual Status init(RuntimeState* state, LocalStateInfo& info) = 0; // Do initialization. This step can be executed multiple times, so we should make sure it is // idempotent (e.g. wait for runtime filters). - virtual Status open(RuntimeState* state) = 0; + virtual Status open(RuntimeState* state) { return Status::OK(); } virtual Status close(RuntimeState* state) = 0; // If use projection, we should clear `_origin_block`. @@ -128,7 +128,6 @@ class PipelineXLocalStateBase { RuntimeProfile::Counter* _exec_timer = nullptr; // Account for peak memory used by this node RuntimeProfile::Counter* _peak_memory_usage_counter = nullptr; - RuntimeProfile::Counter* _init_timer = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; RuntimeProfile::Counter* _close_timer = nullptr; @@ -403,7 +402,6 @@ class PipelineXLocalState : public PipelineXLocalStateBase { ~PipelineXLocalState() override = default; Status init(RuntimeState* state, LocalStateInfo& info) override; - Status open(RuntimeState* state) override; virtual std::string name_suffix() const { return " (id=" + std::to_string(_parent->node_id()) + ")"; @@ -544,7 +542,6 @@ class PipelineXSinkLocalStateBase { std::make_unique("faker profile"); RuntimeProfile::Counter* _rows_input_counter = nullptr; - RuntimeProfile::Counter* _init_timer = nullptr; RuntimeProfile::Counter* _open_timer = nullptr; RuntimeProfile::Counter* _close_timer = nullptr; RuntimeProfile::Counter* _wait_for_dependency_timer = nullptr; diff --git a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp index 6644da7b49ad44..d08b05f3c45977 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_task.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_task.cpp @@ -105,11 +105,9 @@ Status PipelineXTask::prepare(const TPipelineInstanceParams& local_params, const query_ctx->register_query_statistics( _state->get_local_state(op->operator_id())->get_query_statistics_ptr()); } - { - const auto& deps = _state->get_local_state(_source->operator_id())->filter_dependencies(); - std::copy(deps.begin(), deps.end(), - std::inserter(_filter_dependencies, _filter_dependencies.end())); - } + + _block = doris::vectorized::Block::create_unique(); + RETURN_IF_ERROR(_extract_dependencies()); // We should make sure initial state for task are runnable so that we can do some preparation jobs (e.g. initialize runtime filters). set_state(PipelineTaskState::RUNNABLE); _prepared = true; @@ -141,6 +139,11 @@ Status PipelineXTask::_extract_dependencies() { _finish_dependencies.push_back(fin_dep); } } + { + const auto& deps = _state->get_local_state(_source->operator_id())->filter_dependencies(); + std::copy(deps.begin(), deps.end(), + std::inserter(_filter_dependencies, _filter_dependencies.end())); + } return Status::OK(); } @@ -196,8 +199,6 @@ Status PipelineXTask::_open() { RETURN_IF_ERROR(st); } RETURN_IF_ERROR(_state->get_sink_local_state()->open(_state)); - RETURN_IF_ERROR(_extract_dependencies()); - _block = doris::vectorized::Block::create_unique(); _opened = true; return Status::OK(); } diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp index 56fdf3dc0352ea..682f9cc2a1cafa 100644 --- a/be/src/vec/sink/vdata_stream_sender.cpp +++ b/be/src/vec/sink/vdata_stream_sender.cpp @@ -80,53 +80,6 @@ Status Channel::init(RuntimeState* state) { _brpc_timeout_ms = std::min(3600, state->execution_timeout()) * 1000; - if (state->query_options().__isset.enable_local_exchange) { - _is_local &= state->query_options().enable_local_exchange; - } - - if (_is_local) { - _local_recvr = _parent->state()->exec_env()->vstream_mgr()->find_recvr( - _fragment_instance_id, _dest_node_id); - } else { - if (_brpc_dest_addr.hostname == BackendOptions::get_localhost()) { - _brpc_stub = state->exec_env()->brpc_internal_client_cache()->get_client( - "127.0.0.1", _brpc_dest_addr.port); - } else { - _brpc_stub = - state->exec_env()->brpc_internal_client_cache()->get_client(_brpc_dest_addr); - } - - if (!_brpc_stub) { - std::string msg = fmt::format("Get rpc stub failed, dest_addr={}:{}", - _brpc_dest_addr.hostname, _brpc_dest_addr.port); - LOG(WARNING) << msg; - return Status::InternalError(msg); - } - } - - _serializer.set_is_local(_is_local); - - // In bucket shuffle join will set fragment_instance_id (-1, -1) - // to build a camouflaged empty channel. the ip and port is '0.0.0.0:0" - // so the empty channel not need call function close_internal() - _need_close = (_fragment_instance_id.hi != -1 && _fragment_instance_id.lo != -1); - _state = state; - return Status::OK(); -} - -template -Status Channel::init_stub(RuntimeState* state) { - if (_brpc_dest_addr.hostname.empty()) { - LOG(WARNING) << "there is no brpc destination address's hostname" - ", maybe version is not compatible."; - return Status::InternalError("no brpc destination"); - } - if (state->query_options().__isset.enable_local_exchange) { - _is_local &= state->query_options().enable_local_exchange; - } - if (_is_local) { - return Status::OK(); - } if (_brpc_dest_addr.hostname == BackendOptions::get_localhost()) { _brpc_stub = state->exec_env()->brpc_internal_client_cache()->get_client( "127.0.0.1", _brpc_dest_addr.port); @@ -140,27 +93,10 @@ Status Channel::init_stub(RuntimeState* state) { LOG(WARNING) << msg; return Status::InternalError(msg); } - return Status::OK(); -} -template -Status Channel::open(RuntimeState* state) { - _be_number = state->be_number(); - _brpc_request = std::make_shared(); - // initialize brpc request - _brpc_request->mutable_finst_id()->set_hi(_fragment_instance_id.hi); - _brpc_request->mutable_finst_id()->set_lo(_fragment_instance_id.lo); - _finst_id = _brpc_request->finst_id(); - - _brpc_request->mutable_query_id()->set_hi(state->query_id().hi); - _brpc_request->mutable_query_id()->set_lo(state->query_id().lo); - _query_id = _brpc_request->query_id(); - - _brpc_request->set_node_id(_dest_node_id); - _brpc_request->set_sender_id(_parent->sender_id()); - _brpc_request->set_be_number(_be_number); - - _brpc_timeout_ms = std::min(3600, state->execution_timeout()) * 1000; + if (state->query_options().__isset.enable_local_exchange) { + _is_local &= state->query_options().enable_local_exchange; + } if (_is_local) { _local_recvr = _parent->state()->exec_env()->vstream_mgr()->find_recvr( diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h index 2771eeaa2a0586..8119b5a35f9772 100644 --- a/be/src/vec/sink/vdata_stream_sender.h +++ b/be/src/vec/sink/vdata_stream_sender.h @@ -286,9 +286,6 @@ class Channel { // Returns OK if successful, error indication otherwise. Status init(RuntimeState* state); - Status init_stub(RuntimeState* state); - Status open(RuntimeState* state); - // Asynchronously sends a row batch. // Returns the status of the most recently finished transmit_data // rpc (or OK if there wasn't one that hasn't been reported yet). @@ -399,8 +396,8 @@ class Channel { PUniqueId _finst_id; PUniqueId _query_id; PBlock _pb_block; - std::shared_ptr _brpc_request = nullptr; - std::shared_ptr _brpc_stub = nullptr; + std::shared_ptr _brpc_request; + std::shared_ptr _brpc_stub; std::shared_ptr> _send_remote_block_callback; Status _receiver_status; int32_t _brpc_timeout_ms = 500; diff --git a/be/src/vec/sink/volap_table_sink.cpp b/be/src/vec/sink/volap_table_sink.cpp index 8da183cd61fff4..7d607d5f003bfd 100644 --- a/be/src/vec/sink/volap_table_sink.cpp +++ b/be/src/vec/sink/volap_table_sink.cpp @@ -122,7 +122,13 @@ namespace vectorized { VOlapTableSink::VOlapTableSink(ObjectPool* pool, const RowDescriptor& row_desc, const std::vector& texprs) - : AsyncWriterSink(row_desc, texprs) {} + : AsyncWriterSink(row_desc, texprs), _pool(pool) {} + +Status VOlapTableSink::init(const TDataSink& t_sink) { + RETURN_IF_ERROR(AsyncWriterSink::init(t_sink)); + RETURN_IF_ERROR(_writer->init_properties(_pool)); + return Status::OK(); +} Status VOlapTableSink::close(RuntimeState* state, Status exec_status) { if (_closed) { diff --git a/be/src/vec/sink/volap_table_sink.h b/be/src/vec/sink/volap_table_sink.h index 0de1cebe5a63af..add285cdfdd7df 100644 --- a/be/src/vec/sink/volap_table_sink.h +++ b/be/src/vec/sink/volap_table_sink.h @@ -86,9 +86,14 @@ class VOlapTableSink final : public AsyncWriterSink& texprs); + // the real writer will construct in (actually, father's) init but not constructor + Status init(const TDataSink& sink) override; + Status close(RuntimeState* state, Status exec_status) override; private: + ObjectPool* _pool = nullptr; + Status _close_status = Status::OK(); }; diff --git a/be/src/vec/sink/volap_table_sink_v2.cpp b/be/src/vec/sink/volap_table_sink_v2.cpp index fbc57fc83e1b5a..a73ee483bd3886 100644 --- a/be/src/vec/sink/volap_table_sink_v2.cpp +++ b/be/src/vec/sink/volap_table_sink_v2.cpp @@ -39,10 +39,16 @@ namespace vectorized { VOlapTableSinkV2::VOlapTableSinkV2(ObjectPool* pool, const RowDescriptor& row_desc, const std::vector& texprs) - : AsyncWriterSink(row_desc, texprs) {} + : AsyncWriterSink(row_desc, texprs), _pool(pool) {} VOlapTableSinkV2::~VOlapTableSinkV2() = default; +Status VOlapTableSinkV2::init(const TDataSink& t_sink) { + RETURN_IF_ERROR(AsyncWriterSink::init(t_sink)); + RETURN_IF_ERROR(_writer->init_properties(_pool)); + return Status::OK(); +} + Status VOlapTableSinkV2::close(RuntimeState* state, Status exec_status) { SCOPED_TIMER(_exec_timer); if (_closed) { diff --git a/be/src/vec/sink/volap_table_sink_v2.h b/be/src/vec/sink/volap_table_sink_v2.h index 33e50adeb1125a..8257d83bfc1ef5 100644 --- a/be/src/vec/sink/volap_table_sink_v2.h +++ b/be/src/vec/sink/volap_table_sink_v2.h @@ -56,9 +56,13 @@ class VOlapTableSinkV2 final : public AsyncWriterSinkinit(table_sink.schema)); _schema->set_timestamp_ms(state->timestamp_ms()); _schema->set_timezone(state->timezone()); - _pool = state->obj_pool(); _location = _pool->add(new OlapTableLocationParam(table_sink.location)); _nodes_info = _pool->add(new DorisNodesInfo(table_sink.nodes_info)); if (table_sink.__isset.write_single_replica && table_sink.write_single_replica) { diff --git a/be/src/vec/sink/writer/vtablet_writer.h b/be/src/vec/sink/writer/vtablet_writer.h index 514b67f95e9143..b44ce7aabfbac7 100644 --- a/be/src/vec/sink/writer/vtablet_writer.h +++ b/be/src/vec/sink/writer/vtablet_writer.h @@ -520,6 +520,8 @@ class VTabletWriter final : public AsyncResultWriter { public: VTabletWriter(const TDataSink& t_sink, const VExprContextSPtrs& output_exprs); + Status init_properties(ObjectPool* pool); + Status write(Block& block) override; Status close(Status) override; diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.cpp b/be/src/vec/sink/writer/vtablet_writer_v2.cpp index 8cac63f51107bf..a7467758dc6507 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.cpp +++ b/be/src/vec/sink/writer/vtablet_writer_v2.cpp @@ -135,8 +135,12 @@ Status VTabletWriterV2::_init_row_distribution() { return _row_distribution.open(_output_row_desc); } +Status VTabletWriterV2::init_properties(ObjectPool* pool) { + _pool = pool; + return Status::OK(); +} + Status VTabletWriterV2::_init(RuntimeState* state, RuntimeProfile* profile) { - _pool = state->obj_pool(); auto& table_sink = _t_sink.olap_table_sink; _load_id.set_hi(table_sink.load_id.hi); _load_id.set_lo(table_sink.load_id.lo); diff --git a/be/src/vec/sink/writer/vtablet_writer_v2.h b/be/src/vec/sink/writer/vtablet_writer_v2.h index 20952229bbbe63..c04cff15cf4b37 100644 --- a/be/src/vec/sink/writer/vtablet_writer_v2.h +++ b/be/src/vec/sink/writer/vtablet_writer_v2.h @@ -106,6 +106,8 @@ class VTabletWriterV2 final : public AsyncResultWriter { ~VTabletWriterV2() override; + Status init_properties(ObjectPool* pool); + Status write(Block& block) override; Status open(RuntimeState* state, RuntimeProfile* profile) override; From 8a1f579ff0e178f738b40d173768344dc8f744b6 Mon Sep 17 00:00:00 2001 From: Pxl Date: Thu, 11 Apr 2024 18:39:48 +0800 Subject: [PATCH 04/71] [Improvementation](stream-load) improve streamLoadPut log warning detail (#33535) improve streamLoadPut log warning detail --- .../java/org/apache/doris/service/FrontendServiceImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 8d5ebb0fc37a88..683ad847872727 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -1948,11 +1948,11 @@ public TStreamLoadPutResult streamLoadPut(TStreamLoadPutRequest request) { status.setStatusCode(TStatusCode.NOT_FOUND); status.addToErrorMsgs(e.getMessage()); } catch (UserException e) { - LOG.warn("failed to get stream load plan: {}", e.getMessage()); + LOG.warn("failed to get stream load plan, label: {}", request.getLabel(), e); status.setStatusCode(TStatusCode.ANALYSIS_ERROR); status.addToErrorMsgs(e.getMessage()); } catch (Throwable e) { - LOG.warn("catch unknown result.", e); + LOG.warn("stream load catch unknown result, label: {}", request.getLabel(), e); status.setStatusCode(TStatusCode.INTERNAL_ERROR); status.addToErrorMsgs(e.getClass().getSimpleName() + ": " + Strings.nullToEmpty(e.getMessage())); return result; From c866e401886362b30e5df346256fe5d6c2c375e6 Mon Sep 17 00:00:00 2001 From: zhengyu Date: Thu, 11 Apr 2024 20:45:15 +0800 Subject: [PATCH 05/71] [fix](regression) fix accidental failure by sample analyze (#33498) sample analyze will delay the update the show stats result. check for outdated value may cause cloud regression to failure. Signed-off-by: freemandealer --- .../suites/statistics/analyze_stats_partition_first_load.groovy | 1 + regression-test/suites/statistics/test_basic_statistics.groovy | 1 + 2 files changed, 2 insertions(+) diff --git a/regression-test/suites/statistics/analyze_stats_partition_first_load.groovy b/regression-test/suites/statistics/analyze_stats_partition_first_load.groovy index f5d2d33d928a6a..d25ff8086c43f5 100644 --- a/regression-test/suites/statistics/analyze_stats_partition_first_load.groovy +++ b/regression-test/suites/statistics/analyze_stats_partition_first_load.groovy @@ -20,6 +20,7 @@ import java.util.stream.Collectors suite("test_analyze_partition_first_load") { String tbl = "partition_first_load_test" + sql """set global force_sample_analyze=false""" sql """ DROP TABLE IF EXISTS `$tbl` diff --git a/regression-test/suites/statistics/test_basic_statistics.groovy b/regression-test/suites/statistics/test_basic_statistics.groovy index b6cbc25cc8b443..f5bf5ec341aa22 100644 --- a/regression-test/suites/statistics/test_basic_statistics.groovy +++ b/regression-test/suites/statistics/test_basic_statistics.groovy @@ -18,6 +18,7 @@ suite("test_basic_statistics") { String db = "test_basic_statistics" String tbl = "test_table_1" + sql """set global force_sample_analyze=false""" sql """ DROP DATABASE IF EXISTS `${db}` From 3aafe68a759faeb3ab96e7ed195c547a185bba5f Mon Sep 17 00:00:00 2001 From: airborne12 Date: Thu, 11 Apr 2024 21:26:00 +0800 Subject: [PATCH 06/71] [Fix](inverted index) fix build index error status when batch_next (#33532) --- be/src/olap/task/index_builder.cpp | 6 +++++ .../test_build_index_fault.out | 6 +++++ .../test_build_index_fault.groovy | 24 +++++++++++++++---- 3 files changed, 32 insertions(+), 4 deletions(-) diff --git a/be/src/olap/task/index_builder.cpp b/be/src/olap/task/index_builder.cpp index 85ba65820ebc76..0fb0f2c59d9c48 100644 --- a/be/src/olap/task/index_builder.cpp +++ b/be/src/olap/task/index_builder.cpp @@ -27,6 +27,7 @@ #include "olap/segment_loader.h" #include "olap/storage_engine.h" #include "olap/tablet_schema.h" +#include "util/debug_points.h" #include "util/trace.h" namespace doris { @@ -396,6 +397,10 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta output_rowset_schema->create_block(return_columns)); while (true) { auto status = iter->next_batch(block.get()); + DBUG_EXECUTE_IF("IndexBuilder::handle_single_rowset", { + status = Status::Error( + "next_batch fault injection"); + }); if (!status.ok()) { if (status.is()) { break; @@ -403,6 +408,7 @@ Status IndexBuilder::handle_single_rowset(RowsetMetaSharedPtr output_rowset_meta LOG(WARNING) << "failed to read next block when schema change for inverted index." << ", err=" << status.to_string(); + return status; } // write inverted index data diff --git a/regression-test/data/fault_injection_p0/test_build_index_fault.out b/regression-test/data/fault_injection_p0/test_build_index_fault.out index 543ca7ae5ff596..3d5d60cfcaa7a2 100644 --- a/regression-test/data/fault_injection_p0/test_build_index_fault.out +++ b/regression-test/data/fault_injection_p0/test_build_index_fault.out @@ -17,3 +17,9 @@ -- !count6 -- 1000000 +-- !count7 -- +1000000 + +-- !count8 -- +1000000 + diff --git a/regression-test/suites/fault_injection_p0/test_build_index_fault.groovy b/regression-test/suites/fault_injection_p0/test_build_index_fault.groovy index 988db48c6a79e9..84e0bae24ad364 100644 --- a/regression-test/suites/fault_injection_p0/test_build_index_fault.groovy +++ b/regression-test/suites/fault_injection_p0/test_build_index_fault.groovy @@ -192,7 +192,7 @@ suite("test_build_index_fault", "inverted_index, nonConcurrent,p2"){ // BUILD INDEX and expect state is FINISHED sql """ BUILD INDEX idx_comment ON ${tableName}; """ state = wait_for_last_build_index_on_table_finish(tableName, timeout) - assertEquals(state, "FINISHED") + assertEquals("FINISHED", state) // check data qt_count3 """ SELECT COUNT() from ${tableName}; """ @@ -209,7 +209,7 @@ suite("test_build_index_fault", "inverted_index, nonConcurrent,p2"){ // BUILD INDEX again and expect state is FINISHED sql """ BUILD INDEX idx_comment ON ${tableName}; """ state = wait_for_last_build_index_on_table_finish(tableName, timeout) - assertEquals(state, "FINISHED") + assertEquals("FINISHED", state) // check data qt_count4 """ SELECT COUNT() from ${tableName}; """ @@ -219,7 +219,7 @@ suite("test_build_index_fault", "inverted_index, nonConcurrent,p2"){ GetDebugPoint().enableDebugPointForAllBEs("fault_inject::BetaRowset::link_files_to::_link_inverted_index_file") sql """ BUILD INDEX idx_title ON ${tableName}; """ state = wait_for_last_build_index_on_table_finish(tableName, timeout) - assertEquals(state, "wait_timeout") + assertEquals("wait_timeout", state) // check data qt_count5 """ SELECT COUNT() from ${tableName}; """ @@ -227,7 +227,23 @@ suite("test_build_index_fault", "inverted_index, nonConcurrent,p2"){ GetDebugPoint().disableDebugPointForAllBEs("fault_inject::BetaRowset::link_files_to::_link_inverted_index_file") // timeout * 10 for possible fe schedule delay state = wait_for_last_build_index_on_table_finish(tableName, timeout * 10) - assertEquals(state, "FINISHED") + assertEquals("FINISHED", state) // check data qt_count6 """ SELECT COUNT() from ${tableName}; """ + + // BUILD INDEX with error injection + sql """ ALTER TABLE ${tableName} ADD INDEX idx_url (`url`) USING INVERTED """ + GetDebugPoint().enableDebugPointForAllBEs("IndexBuilder::handle_single_rowset") + sql """ BUILD INDEX idx_url ON ${tableName}; """ + state = wait_for_last_build_index_on_table_finish(tableName, timeout) + assertEquals("wait_timeout", state) + // check data + qt_count7 """ SELECT COUNT() from ${tableName}; """ + + GetDebugPoint().disableDebugPointForAllBEs("IndexBuilder::handle_single_rowset") + // timeout * 10 for possible fe schedule delay + state = wait_for_last_build_index_on_table_finish(tableName, timeout * 10) + assertEquals("FINISHED", state) + // check data + qt_count8 """ SELECT COUNT() from ${tableName}; """ } From 550f9bcb6f5861ebb91c4440cbf8ce94519aa5f5 Mon Sep 17 00:00:00 2001 From: zhangstar333 <87313068+zhangstar333@users.noreply.github.com> Date: Thu, 11 Apr 2024 22:45:29 +0800 Subject: [PATCH 07/71] [bug](not in) if not in (null) could eos early (#33482) * [bug](not in) if not in (null) could eos early --- be/src/vec/exec/scan/vscan_node.cpp | 3 +- .../conditional_functions/test_query_in.out | 2 + .../test_query_in.groovy | 46 +++++++++++++++++++ 3 files changed, 50 insertions(+), 1 deletion(-) diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp index b3aeebb4e5b3c9..e75214dcdf83e3 100644 --- a/be/src/vec/exec/scan/vscan_node.cpp +++ b/be/src/vec/exec/scan/vscan_node.cpp @@ -863,7 +863,8 @@ Status VScanNode::_normalize_not_in_and_not_eq_predicate(VExpr* expr, VExprConte HybridSetBase::IteratorBase* iter = state->hybrid_set->begin(); auto fn_name = std::string(""); - if (!is_fixed_range && state->hybrid_set->contain_null()) { + // https://github.com/apache/doris/pull/15466 + if (state->hybrid_set->contain_null()) { _eos = true; } while (iter->has_next()) { diff --git a/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_query_in.out b/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_query_in.out index 57ae1887c080a5..7783afd86313cf 100644 --- a/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_query_in.out +++ b/regression-test/data/nereids_p0/sql_functions/conditional_functions/test_query_in.out @@ -104,3 +104,5 @@ jj -28532 false 1 1989 1001 11011902 123.123 true 1989-03-21 1989-03-21T13:00 wangjuoo4 0.1 6.333 string12345 170141183460469231731687303715884105727 false 3 1989 1002 11011905 24453.325 false 2012-03-14 2000-01-01T00:00 yunlj8@nk 78945.0 3654.0 string12345 0 +-- !in33 -- + diff --git a/regression-test/suites/nereids_p0/sql_functions/conditional_functions/test_query_in.groovy b/regression-test/suites/nereids_p0/sql_functions/conditional_functions/test_query_in.groovy index d65e969b42b278..1de3321f0fd358 100644 --- a/regression-test/suites/nereids_p0/sql_functions/conditional_functions/test_query_in.groovy +++ b/regression-test/suites/nereids_p0/sql_functions/conditional_functions/test_query_in.groovy @@ -64,4 +64,50 @@ suite("test_query_in", "query,p0") { qt_in31 """select * from (select 'jj' as kk1, sum(k2) from ${tableName2} where k10 = '2015-04-02' group by kk1)tt where kk1 = 'jj'""" qt_in32 """select * from ${tableName1} where cast(k1 as char) in (1, -1, 5, 0.1, 3.000) order by k1, k2, k3, k4""" + + sql """DROP TABLE IF EXISTS table_200_undef_partitions2_keys3_properties4_distributed_by56""" + sql """ + CREATE TABLE `table_200_undef_partitions2_keys3_properties4_distributed_by56` ( + `col_int_undef_signed_not_null` int(11) NOT NULL, + `col_date_undef_signed_not_null` date NOT NULL, + `col_bigint_undef_signed_not_null` bigint(20) NOT NULL, + `col_int_undef_signed` int(11) NULL, + `col_bigint_undef_signed` bigint(20) NULL, + `col_date_undef_signed` date NULL, + `col_varchar_10__undef_signed` varchar(10) NULL, + `col_varchar_10__undef_signed_not_null` varchar(10) NOT NULL, + `col_varchar_1024__undef_signed` varchar(1024) NULL, + `col_varchar_1024__undef_signed_not_null` varchar(1024) NOT NULL, + `pk` int(11) NULL + ) ENGINE=OLAP + duplicate KEY(`col_int_undef_signed_not_null`, `col_date_undef_signed_not_null`, `col_bigint_undef_signed_not_null`) + COMMENT 'OLAP' + PARTITION BY RANGE(`col_int_undef_signed_not_null`, `col_date_undef_signed_not_null`) + (PARTITION p VALUES [("-2147483648", '0000-01-01'), ("-1", '1997-12-11')), + PARTITION p0 VALUES [("-1", '1997-12-11'), ("4", '2023-12-11')), + PARTITION p1 VALUES [("4", '2023-12-11'), ("6", '2023-12-15')), + PARTITION p2 VALUES [("6", '2023-12-15'), ("7", '2023-12-16')), + PARTITION p3 VALUES [("7", '2023-12-16'), ("8", '2023-12-25')), + PARTITION p4 VALUES [("8", '2023-12-25'), ("8", '2024-01-18')), + PARTITION p5 VALUES [("8", '2024-01-18'), ("10", '2024-02-18')), + PARTITION p6 VALUES [("10", '2024-02-18'), ("1147483647", '2056-12-31')), + PARTITION p100 VALUES [("1147483647", '2056-12-31'), ("2147483647", '9999-12-31'))) + DISTRIBUTED BY HASH(`col_bigint_undef_signed_not_null`) BUCKETS 30 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ + insert into table_200_undef_partitions2_keys3_properties4_distributed_by56(pk,col_int_undef_signed,col_int_undef_signed_not_null,col_bigint_undef_signed,col_bigint_undef_signed_not_null,col_date_undef_signed,col_date_undef_signed_not_null,col_varchar_10__undef_signed,col_varchar_10__undef_signed_not_null,col_varchar_1024__undef_signed,col_varchar_1024__undef_signed_not_null) values (0,-10,-10,null,858959258950266384,'2024-02-18','2027-01-09',null,'a','b','j'),(1,-10,7,null,6742353775637329154,'2023-12-12','2025-02-17','d','o','l','p'),(2,2,7,-5728564124670569750,1263875535185196830,'2023-12-12','2026-01-18','q','a','v','w'),(3,-10,1,7051047447732837092,2539536806670127406,'2023-12-09','2024-01-17','g','a',null,'q'),(4,7,7,-8634733270957194673,40291659134025903,'2024-01-17','2026-01-18','o','j','k','m'),(5,8,9,-8556392990684241659,8465118127227018191,'2024-01-09','2023-12-09',null,'d','e','j'),(6,8,3,null,-2017332532079385575,'2023-12-18','2023-12-16','d','y','h','b'),(7,9,9,5596192543704853388,267638791183929783,null,'2023-12-19',null,'d','t','l'),(8,5,-4,-6113554641783986996,-5658484838681634530,'2023-12-14','2023-12-19','i','c','a','k'),(9,null,-10,null,4991457615093257177,'2027-01-09','2023-12-19',null,'n','y','f'),(10,9,3,-7301049681081113761,-4781699936467294820,'2024-01-31','2024-01-09','u','d','c','n'),(11,-4,-4,-7086337865184828373,4315532519968709226,'2023-12-17','2023-12-10',null,'o',null,'u'),(12,0,9,-8757684043812948141,616531940395766973,null,'2024-01-31','h','d','l','y'),(13,2,-4,null,5554227672494943664,'2025-02-18','2023-12-14','q','w','f','p'),(14,-10,-10,-4561433444690964781,-8444020181581535329,'2023-12-12','2023-12-15','z','c','w','n'),(15,-10,7,-8802827632747131651,-5481795095278885584,'2023-12-17','2024-02-18','e','t','p','d'),(16,6,6,-5639219943760685155,490175745438454548,'2024-01-19','2025-06-18',null,'e','s','m'),(17,-4,6,1274523274892000841,-1666908142973027113,'2025-06-18','2025-02-18','m','j',null,'t'),(18,0,-10,-829309147977962633,-1448984827777801939,'2023-12-10','2025-06-18','q','f','p','d'),(19,4,-10,5017465451318631070,1485374261466392807,'2023-12-12','2024-02-18','o','u','n','r'),(20,-10,-4,null,-8472667734325532219,null,'2024-01-09',null,'l','t','s'),(21,-4,-10,4724543116023491216,3069741180880099025,'2026-02-18','2024-02-18','h','n','p','w'),(22,8,7,-8225548122477525987,-7403275139540774667,'2024-01-09','2026-01-18','w','u','z','y'),(23,7,4,-8308238561244451420,-4737114215802511670,'2023-12-09','2026-02-18','h','g',null,'j'),(24,8,0,-3445293753181337298,-4671072044529490912,'2025-06-18','2024-01-31','h','z',null,'h'),(25,9,3,-643703259719937453,-4467933225693972253,'2023-12-19','2026-02-18','h','k','a','p'),(26,-10,5,-4360065276518573247,6071515314723843321,'2024-02-18','2023-12-11','b','e','y','u'),(27,-10,5,7277432098525098029,7324870441313849762,'2023-12-16','2023-12-13',null,'o','d','m'),(28,4,1,null,-3401829705651503439,'2023-12-11','2024-01-19',null,'m','b','o'),(29,5,9,1595806217868219791,2396625106748754274,'2023-12-12','2026-01-18','y','i','v','k'),(30,8,-4,-8296263444527129454,-1200655730727433306,'2023-12-18','2023-12-16','x','w','e','g'),(31,8,9,null,-1057786787217140446,'2024-02-18','2025-02-18',null,'r',null,'n'),(32,2,6,null,-3794687203717519885,'2024-01-17','2023-12-11','s','o','o','p'),(33,8,-4,745035089749481716,6870210688431692264,'2023-12-16','2024-02-18','u','v','r','c'),(34,0,-4,-6725072017628066450,-8684294917367566361,'2023-12-18','2023-12-10','u','x','s','f'),(35,9,9,2062232271317540255,801582713562034622,'2023-12-12','2027-01-09','f','e','t','m'),(36,7,6,6833793247704001187,1810376613825059765,null,'2023-12-17','s','x',null,'c'),(37,3,-4,null,-5441663581149252353,'2023-12-19','2023-12-12','j','p','n','p'),(38,-4,2,null,-2168470328237974425,'2024-02-18','2026-02-18','l','y','z','z'),(39,1,-10,-2207119652858390693,2589212604181409488,null,'2023-12-11',null,'c','h','r'),(40,1,-4,-5120586939953271680,-5115443745232748649,'2025-06-18','2023-12-18',null,'z',null,'g'),(41,-4,9,-2705550486017092761,953166616065633786,null,'2024-01-08','d','u','z','k'),(42,1,8,528231752347363899,3778913804645926289,null,'2024-01-17','s','y','k','w'),(43,2,-10,null,2959039737583266404,'2023-12-12','2026-02-18','x','a','o','i'),(44,5,1,4479688968809054722,7900165947449530851,'2023-12-17','2023-12-19','o','m','m','k'),(45,null,3,null,3853519400061892820,'2023-12-19','2027-01-09',null,'m','j','u'),(46,null,2,-2308621156442800431,7150891722109935935,null,'2023-12-09','r','b','p','r'),(47,-10,7,null,-5384774129746079660,'2026-01-18','2026-02-18','i','y','f','l'),(48,7,-10,2518046378930975040,8009748340888023422,'2024-01-17','2023-12-14','a','y','m','r'),(49,0,1,-855757624678856701,6770116622679381583,'2023-12-11','2023-12-20','s','p','w','x'),(50,5,-4,null,-6219498513513783540,'2024-02-18','2023-12-09','l','d','y','a'),(51,null,9,-2453700456069426683,-82738410074789208,'2027-01-16','2023-12-10','o','w','f','w'),(52,-4,0,119899203407625900,-4661476900528916913,'2026-02-18','2023-12-10','j','u','g','g'),(53,1,2,-1453070963768828719,6267227367755586537,'2023-12-17','2023-12-10','o','l','o','f'),(54,0,-4,5021492283879788057,-8876089608130965603,'2025-02-17','2024-01-09','f','n','c','y'),(55,-4,1,1122238061013285565,308537169947542049,'2024-02-18','2023-12-12','f','z',null,'b'),(56,-4,2,-4640444469732044136,-5543784356823664941,null,'2024-01-19','j','w','z','e'),(57,5,-4,3373002253627578009,-1165501306547040863,'2027-01-09','2026-02-18','w','r','p','c'),(58,1,-4,8808393723467177119,-5346257761475098062,'2023-12-15','2024-02-18',null,'j','i','w'),(59,4,0,6552414885194925805,-3534775325743454515,'2026-01-18','2023-12-13','e','f',null,'r'),(60,6,-10,-4000277373668369461,-3111045443815568668,'2024-02-18','2025-06-18',null,'w','v','m'),(61,2,1,null,-8233172593971051412,'2023-12-18','2024-02-18','h','k','s','t'),(62,-10,0,null,-4491694283352502,'2023-12-17','2024-01-31','y','d','g','c'),(63,7,9,1464936785266490950,-2418183835907994133,'2025-06-18','2023-12-14',null,'z','y','m'),(64,3,5,-7706297176485718179,933626871060619190,'2023-12-20','2023-12-18','i','k','k','g'),(65,4,5,-8995970132111077734,8154004529141856099,null,'2025-06-18','w','n','j','w'),(66,9,-10,2294807974922328614,5767191679135413635,'2025-06-18','2025-02-17','w','j','f','l'),(67,-4,-10,587090899228213425,3507177411126691631,null,'2023-12-20','w','z','t','b'),(68,-10,-10,-4926187957454428492,-8376602544614160532,'2023-12-17','2026-02-18','x','u','k','z'),(69,-10,0,null,8496150289250370341,'2026-02-18','2026-01-18',null,'o','b','y'),(70,1,-10,-1962993669054805646,-6008622308929669427,'2024-01-09','2023-12-14',null,'a','d','x'),(71,3,0,null,-5808769432222500599,'2023-12-15','2024-02-18','u','f','y','b'),(72,-10,-10,-5219650699813656115,452002228724588510,'2023-12-18','2024-01-19',null,'c','j','y'),(73,0,8,-8474540639839757820,6411085865775239406,'2025-02-18','2024-01-19',null,'z','z','n'),(74,-4,8,-2199433898224071335,8363644609820182223,'2027-01-09','2023-12-09','p','s','m','y'),(75,3,-10,-2522181458998583891,917421047128804747,'2023-12-11','2023-12-13','t','u','r','c'),(76,0,4,4399813326725698376,1373458130594935549,'2024-02-18','2024-01-17','h','p','n','t'),(77,-4,-4,null,1788584297854554832,'2025-02-17','2024-01-09','w','r','u','r'),(78,0,-10,165950310516640650,-8133159160941577613,'2025-06-18','2024-02-18','n','c',null,'p'),(79,0,6,-2829447613177748553,-4770797190584838628,'2023-12-20','2023-12-20','f','w',null,'g'),(80,3,1,null,6702519360356810260,'2023-12-12','2023-12-19','d','y','n','i'),(81,null,3,4248574368647718721,-8514779561337794294,'2023-12-15','2024-01-17','h','r','n','z'),(82,2,1,320280761780398808,4314609622203347175,null,'2023-12-20','x','n','y','j'),(83,1,0,-6200587512045784782,-4498885757862052232,'2027-01-16','2023-12-18','u','d','g','f'),(84,5,-4,-4277349083685494679,-3593280850611947078,'2025-06-18','2025-02-17',null,'v','w','f'),(85,5,7,2883282944364027996,6420277952201969882,'2026-01-18','2024-02-18','u','j','t','q'),(86,4,-10,-1120247934953844457,1663931126940365825,'2023-12-11','2027-01-16','q','a','z','t'),(87,9,8,-1426751146107022190,1441141349683396036,'2023-12-16','2023-12-17','q','y','h','i'),(88,8,-10,-4349974916742880190,8256435217108546627,'2023-12-14','2024-01-09','z','c','o','b'),(89,-10,8,-8338928813651598785,7977286452179410459,null,'2024-01-19','n','k','j','w'),(90,null,0,244807790468898978,-3801880175864777196,null,'2023-12-17','l','y','h','i'),(91,null,9,-6408133915102679384,7511711843888882940,'2026-01-18','2027-01-16',null,'v','m','a'),(92,null,-4,3739649918962133441,3076240410999154553,'2023-12-17','2024-01-08','z','f','u','y'),(93,null,-10,null,5205086860864014631,null,'2023-12-13','m','y',null,'e'),(94,4,3,7271842280143827859,-1246168252444258268,'2025-02-17','2023-12-14','c','q','w','m'),(95,-10,1,null,-1956081584567562286,'2023-12-11','2024-02-18','y','g','u','p'),(96,null,0,-7842733493711584669,2232657695829197601,'2026-01-18','2023-12-20',null,'x','r','u'),(97,null,1,null,-8604813143528815146,'2023-12-20','2024-01-08','z','v','r','i'),(98,-4,4,null,2825934924130026445,'2024-01-09','2023-12-09','k','r','x','u'),(99,1,0,null,-1181926554319033971,'2024-02-18','2023-12-13','l','g','e','r'),(100,4,4,null,-8117404878456129464,'2023-12-16','2024-01-08','l','q','n','e'),(101,9,-10,null,-8162239741142259898,null,'2024-01-31','s','l','v','e'),(102,8,4,4428460829238539547,4665293281033362886,'2024-01-19','2025-06-18','y','a',null,'j'),(103,8,2,-1223671322140090504,6977230827413685849,'2025-02-18','2024-02-18','h','s','q','m'),(104,2,-10,null,-2682464990487136210,'2027-01-16','2026-01-18','a','u','t','a'),(105,8,3,1844228284799324033,7396769740370937616,'2024-01-08','2026-02-18','l','q','m','w'),(106,null,0,-4635982079725022381,-1559819812914917184,'2025-06-18','2025-06-18','x','b',null,'w'),(107,8,-4,-5968215346736716838,9136666151714541291,'2024-02-18','2024-01-08','w','r','g','v'),(108,5,5,null,5858275700878532942,null,'2024-01-19','p','c','v','t'),(109,4,-10,7788561359573057073,3796991113318205212,'2024-01-19','2027-01-16','s','z','y','u'),(110,9,8,-4936498559299264714,3987489369204641984,'2023-12-16','2024-01-08','m','n','l','u'),(111,0,-10,null,-1839332505961199897,'2025-02-17','2023-12-16','i','r','x','y'),(112,1,1,7588479964786885990,-3518894912918244447,'2023-12-10','2024-02-18','b','u','t','a'),(113,-4,-10,3503740574835224983,-8376857573915349696,'2023-12-11','2023-12-15','o','g','k','d'),(114,9,3,null,2623523588167267085,'2023-12-11','2024-01-08','m','q','z','u'),(115,4,-10,null,1141333940801996130,'2023-12-09','2025-02-18','a','k',null,'c'),(116,1,1,1301860584665365272,-3137133797699518743,'2023-12-13','2023-12-11','o','f','i','g'),(117,-10,7,2061572628480164506,-941565778547616976,'2025-02-18','2024-02-18','y','b','d','v'),(118,4,-4,5905457525257274831,-3537379062861041102,'2025-06-18','2023-12-15',null,'p','b','a'),(119,6,0,5203103959331291655,4553687495326142230,'2024-01-17','2024-02-18','k','q','x','y'),(120,-4,9,-8635697863181382818,7636960765531229395,'2023-12-09','2025-02-18','t','c','w','m'),(121,0,0,-8318602828209501850,-4943371026882157173,'2024-01-17','2023-12-16','r','x','n','f'),(122,1,7,2423501569419443281,-6589241835404749979,'2025-02-18','2027-01-09','l','e','x','x'),(123,-4,6,null,3729608390680606227,'2023-12-15','2026-02-18','x','o','m','l'),(124,null,-10,4987191974521539793,7099886620557264771,null,'2023-12-19','b','q','g','j'),(125,-10,-10,null,-8031447195377137816,null,'2024-02-18','r','p','u','z'),(126,7,1,6676484990875474971,2888513296794301991,'2024-02-18','2023-12-09','p','n','g','b'),(127,8,2,-2907338527699127177,-4546369298413621144,'2024-01-31','2025-06-18',null,'h','c','w'),(128,-4,-4,-725621622182332157,-4345970020462840706,'2027-01-16','2027-01-09','v','v','w','r'),(129,5,5,null,-7250528589413808433,'2023-12-12','2023-12-17','a','g','z','d'),(130,null,0,-7201945181697281727,4076316632344864610,'2024-01-09','2026-02-18','e','g','p','x'),(131,8,2,4039015488782675948,-7627575764896379667,null,'2024-02-18',null,'x','x','u'),(132,-4,8,582950484149896044,-8546974684347671801,'2023-12-15','2023-12-16',null,'c',null,'s'),(133,4,-10,8472160150721707654,-8121377870390656734,'2023-12-16','2027-01-16','f','t',null,'c'),(134,8,3,null,-7286212975735329414,null,'2023-12-16','e','e','v','v'),(135,8,5,8893169871966679916,-3528815577223369116,'2024-01-09','2023-12-18','a','u','a','b'),(136,2,-10,null,-600117778267872813,'2025-02-17','2026-01-18','v','o','u','a'),(137,3,-10,null,3875574420073502874,'2027-01-16','2023-12-15','d','h',null,'m'),(138,6,3,-7092907178026955224,-7458173286579166661,'2025-02-18','2023-12-20','k','s',null,'v'),(139,4,-4,null,8066978303437641605,'2023-12-14','2025-02-17','x','o','o','c'),(140,3,0,-5283347581061457496,-3904748897731435093,'2024-01-09','2023-12-20','g','o','o','t'),(141,null,3,null,1915421260870512382,'2023-12-11','2025-02-17','l','y','e','k'),(142,-10,-10,8989121510646360393,-5554479379366875651,'2025-06-18','2023-12-20','l','z',null,'w'),(143,-10,-4,5103258104632957393,9108626379079271426,'2025-06-18','2023-12-11','o','t',null,'v'),(144,8,5,null,7089037888036198838,'2024-02-18','2023-12-12','w','e','z','g'),(145,2,0,-2530114892096232328,-3029426656119524867,'2026-02-18','2023-12-20','q','n','w','f'),(146,2,0,-1167184901984734011,9197712261028249733,'2027-01-16','2026-01-18','y','g','d','w'),(147,0,-10,-3151820955481808490,6084406231058480929,null,'2026-02-18','m','z','e','k'),(148,null,7,-5620657663724505253,-5569893795944384789,'2024-02-18','2023-12-15','u','r','t','n'),(149,null,7,1932155420012171131,-7307674080602904438,'2025-06-18','2023-12-18',null,'w','r','a'),(150,null,2,-843675053631435346,794239051911249282,'2023-12-15','2027-01-16','j','a','r','h'),(151,2,3,3099780186868693967,-3225999145687125620,'2024-01-19','2024-01-08','k','c','t','k'),(152,4,2,1770467068126077319,-9055195259747766247,'2025-06-18','2027-01-09','v','a',null,'h'),(153,4,0,-2754737360318041457,-4243320690494160068,'2024-02-18','2026-02-18','h','s','k','t'),(154,null,1,3047341609933447747,-5430978432065487602,'2023-12-10','2025-06-18','r','t','e','w'),(155,null,6,9002764758580821577,4301472086716260964,'2023-12-14','2023-12-17','s','f','u','j'),(156,-10,7,5766282996410787024,-6842767888259184512,'2025-06-18','2024-01-08',null,'i','t','t'),(157,8,0,null,-2971991946957950726,'2023-12-11','2026-01-18','e','n','x','u'),(158,0,5,3486248075304478637,7151930690139263019,'2027-01-09','2023-12-17','s','w','w','h'),(159,null,1,null,8198990090355964665,'2024-01-19','2024-01-31','d','g','y','r'),(160,9,6,null,-2148224693875975660,'2023-12-16','2025-02-18','a','k','b','q'),(161,-10,-4,5488734119027589165,7513618645834418897,'2024-01-09','2023-12-14',null,'s','c','w'),(162,9,8,-7672565124538576135,-5841834824122770876,'2025-02-17','2027-01-09','n','k','c','v'),(163,-4,5,null,7364114271450150187,null,'2023-12-14',null,'j','h','n'),(164,2,-10,null,-2194630486679592764,'2023-12-14','2026-01-18','u','y','p','x'),(165,2,2,-1880918634814261976,1232284826891731722,'2025-06-18','2024-02-18','c','l','p','p'),(166,1,3,null,2074110522301738224,'2026-02-18','2024-01-17','u','c',null,'f'),(167,-10,-4,null,-1170888295489186999,'2023-12-19','2023-12-20','j','q','m','v'),(168,-10,-4,null,-5537440894255523447,null,'2024-02-18',null,'q','j','f'),(169,-10,6,-3451103407491045129,5150722079806394405,'2025-06-18','2024-01-09','v','l',null,'k'),(170,3,4,-5818977070284761683,-5371459815538469672,'2026-02-18','2023-12-14',null,'q','b','m'),(171,1,5,4168395638427775128,5611668983964341559,null,'2024-01-17','w','s','q','h'),(172,1,0,3458544995986307787,-4149888810199163471,'2024-02-18','2024-01-19',null,'q',null,'z'),(173,1,-10,4142592142963263928,-909410561626911386,'2025-06-18','2023-12-09','a','a','d','l'),(174,null,4,-518273896821184968,1288957583256666554,null,'2025-02-17','c','f','x','p'),(175,-10,3,-3032449689171348418,-2741600948103211533,'2024-01-17','2024-01-09','q','y','f','u'),(176,5,0,null,-8712847991493189369,'2026-01-18','2023-12-10',null,'y','o','k'),(177,-10,-10,6505905262846026612,-2663878103275066151,'2023-12-16','2023-12-13','d','i','b','t'),(178,-10,1,2979439245250192393,-4004139275935906983,'2025-02-18','2024-02-18','b','a',null,'f'),(179,null,3,-5246999956982673330,4693551967836520193,'2027-01-16','2023-12-16','x','r','q','t'),(180,-4,6,6129616258684699357,-7442537280431110769,'2026-01-18','2023-12-14','i','p','u','m'),(181,8,-4,null,3559171006199606037,'2025-06-18','2024-02-18',null,'a','o','a'),(182,-10,0,576423267430397537,2065938004294521372,'2024-01-31','2027-01-09','z','f','m','z'),(183,8,9,-46478079095765773,2258977652029773198,'2023-12-14','2023-12-13','y','h',null,'s'),(184,3,-4,755103508423070586,9018243765720151284,'2023-12-12','2023-12-14','n','w','f','z'),(185,7,3,6433137528287040546,-7371938000327475691,'2024-01-08','2025-06-18','f','z','i','b'),(186,-10,8,1177751778138866228,7659149130558923595,'2023-12-12','2027-01-09','r','c',null,'c'),(187,0,9,1288191403351998836,-6549118763715205452,'2023-12-12','2023-12-20','e','i','p','i'),(188,8,-4,null,253602092357870614,'2024-02-18','2027-01-16','c','h','l','d'),(189,-4,3,5718357978282980367,869765378314899903,'2023-12-16','2024-01-17','h','l','e','m'),(190,4,9,2418044860521148649,770875792276394729,'2026-02-18','2025-02-18','o','h',null,'s'),(191,2,4,-1120990051628321869,-5502418519216245814,null,'2023-12-12','d','i','j','b'),(192,5,-10,null,-9098364149107256069,'2023-12-17','2024-01-09','c','i','g','y'),(193,8,-10,null,6154296133843956869,'2023-12-16','2024-02-18','t','r','v','i'),(194,-4,8,-2387697740444229124,-4420658912264770524,'2024-01-17','2023-12-16','f','e','u','c'),(195,-10,6,-2772463034040194217,-7907822037637823718,'2023-12-12','2025-06-18','r','j','c','y'),(196,-4,2,8554448119524477807,5158562227277810813,null,'2024-01-19','z','p','w','k'),(197,4,1,-5392061529260431139,3278574395762126118,null,'2023-12-09','a','s','d','z'),(198,null,-10,-4067083055147098114,2685481298193076314,'2024-02-18','2023-12-11','t','g','v','k'),(199,1,1,-8261512111679927590,-1534216194492202538,'2024-01-31','2023-12-12','t','i','w','e'); + """ + + qt_in33 """ + SELECT + col_varchar_1024__undef_signed + FROM + table_200_undef_partitions2_keys3_properties4_distributed_by56 AS table1 + WHERE + table1.`col_varchar_1024__undef_signed` NOT IN (NULL, 'c') + AND table1.`col_varchar_1024__undef_signed` IN (NULL, 'u', 'd'); + """ } From d8c2db72daa912ffd0f66bc24ef83103c661fd24 Mon Sep 17 00:00:00 2001 From: Xinyi Zou Date: Thu, 11 Apr 2024 23:06:36 +0800 Subject: [PATCH 08/71] [fix](memory) Fix compaction destructor memory tracking #33549 --- be/src/exec/rowid_fetcher.cpp | 2 -- be/src/olap/compaction.cpp | 10 +++++++++- be/src/service/internal_service.cpp | 1 + 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/be/src/exec/rowid_fetcher.cpp b/be/src/exec/rowid_fetcher.cpp index 3892aac966a3f8..c921be9509f56a 100644 --- a/be/src/exec/rowid_fetcher.cpp +++ b/be/src/exec/rowid_fetcher.cpp @@ -327,8 +327,6 @@ struct IteratorItem { Status RowIdStorageReader::read_by_rowids(const PMultiGetRequest& request, PMultiGetResponse* response) { - SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER( - ExecEnv::GetInstance()->rowid_storage_reader_tracker()); // read from storage engine row id by row id OlapReaderStatistics stats; vectorized::Block result_block; diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 81b41d76ee61ae..dec407894ef8a7 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -67,6 +67,7 @@ #include "olap/txn_manager.h" #include "olap/utils.h" #include "runtime/memory/mem_tracker_limiter.h" +#include "runtime/thread_context.h" #include "util/time.h" #include "util/trace.h" @@ -120,7 +121,14 @@ Compaction::Compaction(BaseTabletSPtr tablet, const std::string& label) init_profile(label); } -Compaction::~Compaction() = default; +Compaction::~Compaction() { + SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(_mem_tracker); + _output_rs_writer.reset(); + _tablet.reset(); + _input_rowsets.clear(); + _output_rowset.reset(); + _cur_tablet_schema.reset(); +} void Compaction::init_profile(const std::string& label) { _profile = std::make_unique(label); diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index adcd81689c13a3..9918980514c753 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -1926,6 +1926,7 @@ void PInternalService::multiget_data(google::protobuf::RpcController* controller watch.start(); brpc::ClosureGuard closure_guard(done); response->mutable_status()->set_status_code(0); + SCOPED_ATTACH_TASK(ExecEnv::GetInstance()->rowid_storage_reader_tracker()); Status st = RowIdStorageReader::read_by_rowids(*request, response); st.to_protobuf(response->mutable_status()); LOG(INFO) << "multiget_data finished, cost(us):" << watch.elapsed_time() / 1000; From 838e31146f095cc53d4b38a8ec7e078cbc7b226c Mon Sep 17 00:00:00 2001 From: zhangstar333 <87313068+zhangstar333@users.noreply.github.com> Date: Thu, 11 Apr 2024 23:40:50 +0800 Subject: [PATCH 09/71] [improve](fold) support complex type for constant folding (#32867) --- be/src/runtime/fold_constant_executor.cpp | 61 ++- .../org/apache/doris/catalog/ScalarType.java | 4 + .../org/apache/doris/analysis/CastExpr.java | 5 + .../org/apache/doris/analysis/MapLiteral.java | 2 +- .../rules/FoldConstantRuleOnBE.java | 406 ++++++++++++++++-- .../trees/expressions/literal/MapLiteral.java | 5 +- .../expressions/literal/StructLiteral.java | 5 +- .../doris/rewrite/FoldConstantsRule.java | 1 + .../apache/doris/analysis/MapLiteralTest.java | 12 +- .../doris/analysis/StructLiteralTest.java | 4 +- .../trees/expressions/LiteralTest.java | 265 ++++++++++++ gensrc/thrift/PaloInternalService.thrift | 1 + .../push_filter_inside_join.groovy | 1 + .../push_filter_through.groovy | 1 + 14 files changed, 696 insertions(+), 77 deletions(-) diff --git a/be/src/runtime/fold_constant_executor.cpp b/be/src/runtime/fold_constant_executor.cpp index 69fe217da7802a..5ba384baf84b46 100644 --- a/be/src/runtime/fold_constant_executor.cpp +++ b/be/src/runtime/fold_constant_executor.cpp @@ -48,19 +48,20 @@ #include "util/runtime_profile.h" #include "util/uid_util.h" #include "vec/columns/column.h" +#include "vec/columns/column_array.h" #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" #include "vec/common/string_ref.h" #include "vec/core/block.h" #include "vec/core/column_with_type_and_name.h" +#include "vec/core/field.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type_array.h" #include "vec/data_types/data_type_number.h" #include "vec/exprs/vexpr.h" #include "vec/exprs/vexpr_context.h" #include "vec/runtime/vdatetime_value.h" -using std::string; -using std::map; - namespace doris { static std::unordered_set PRIMITIVE_TYPE_SET { @@ -72,7 +73,7 @@ static std::unordered_set PRIMITIVE_TYPE_SET { Status FoldConstantExecutor::fold_constant_vexpr(const TFoldConstantParams& params, PConstantExprResult* response) { const auto& expr_map = params.expr_map; - auto expr_result_map = response->mutable_expr_result_map(); + auto* expr_result_map = response->mutable_expr_result_map(); TQueryGlobals query_globals = params.query_globals; _query_id = params.query_id; @@ -107,31 +108,45 @@ Status FoldConstantExecutor::fold_constant_vexpr(const TFoldConstantParams& para string result; const auto& column_ptr = tmp_block.get_by_position(result_column).column; const auto& column_type = tmp_block.get_by_position(result_column).type; - if (column_ptr->is_null_at(0)) { - expr_result.set_success(false); - } else { + // 4 from fe: Config.be_exec_version maybe need remove after next version, now in 2.1 + if (_runtime_state->be_exec_version() >= 4 && params.__isset.is_nereids && + params.is_nereids) { + auto* p_type_desc = expr_result.mutable_type_desc(); + auto* p_values = expr_result.mutable_result_content(); + res_type.to_protobuf(p_type_desc); + auto datatype_serde = column_type->get_serde(); + RETURN_IF_ERROR(datatype_serde->write_column_to_pb( + *column_ptr->convert_to_full_column_if_const(), *p_values, 0, 1)); expr_result.set_success(true); - StringRef string_ref; - auto type = ctx->root()->type().type; - //eg: strcut, array, map VARIANT... will not impl get_data_at, so could use column->to_string() - if (PRIMITIVE_TYPE_SET.contains(type)) { - string_ref = column_ptr->get_data_at(0); + // after refactor, this field is useless, but it's required + expr_result.set_content("ERROR"); + expr_result.mutable_type()->set_type(t_type); + pexpr_result_map.mutable_map()->insert({n.first, expr_result}); + } else { + if (column_ptr->is_null_at(0)) { + expr_result.set_success(false); + } else { + expr_result.set_success(true); + StringRef string_ref; + auto type = ctx->root()->type().type; + //eg: strcut, array, map VARIANT... will not impl get_data_at, so could use column->to_string() + if (PRIMITIVE_TYPE_SET.contains(type)) { + string_ref = column_ptr->get_data_at(0); + } + RETURN_IF_ERROR(_get_result((void*)string_ref.data, string_ref.size, + ctx->root()->type(), column_ptr, column_type, + result)); } - RETURN_IF_ERROR(_get_result((void*)string_ref.data, string_ref.size, - ctx->root()->type(), column_ptr, column_type, result)); + expr_result.set_content(std::move(result)); + expr_result.mutable_type()->set_type(t_type); + expr_result.mutable_type()->set_scale(res_type.scale); + expr_result.mutable_type()->set_precision(res_type.precision); + expr_result.mutable_type()->set_len(res_type.len); + pexpr_result_map.mutable_map()->insert({n.first, expr_result}); } - - expr_result.set_content(std::move(result)); - expr_result.mutable_type()->set_type(t_type); - expr_result.mutable_type()->set_scale(res_type.scale); - expr_result.mutable_type()->set_precision(res_type.precision); - expr_result.mutable_type()->set_len(res_type.len); - pexpr_result_map.mutable_map()->insert({n.first, expr_result}); } expr_result_map->insert({m.first, pexpr_result_map}); } - //TODO: will be delete the debug log after find problem of timeout. - LOG(INFO) << "finish fold_query_id: " << query_id_string(); return Status::OK(); } diff --git a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java index 83072ad97f5d5b..67346fc21609c1 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java +++ b/fe/fe-common/src/main/java/org/apache/doris/catalog/ScalarType.java @@ -216,6 +216,10 @@ public static ScalarType createType(PrimitiveType type) { return DEFAULT_DECIMALV2; case LARGEINT: return LARGEINT; + case IPV4: + return IPV4; + case IPV6: + return IPV6; case ALL: return ALL; default: diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java index f069c1cd695dce..d2437fbd95c614 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java @@ -132,6 +132,11 @@ public CastExpr(Type targetType, Expr e, Void v) { if (type.isDecimalV2() && e.type.isDecimalV2()) { getChild(0).setType(type); } + // as the targetType have struct field name, if use the default name will be + // like col1,col2, col3... in struct, and the filed name is import in BE. + if (type.isStructType() && e.type.isStructType()) { + getChild(0).setType(type); + } analysisDone(); return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/MapLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/MapLiteral.java index e55519101cf77b..633478f3f84410 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/MapLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/MapLiteral.java @@ -188,7 +188,7 @@ public String getStringValueInFe() { // map key type do not support complex type throw new UnsupportedOperationException("Unsupport key type for MAP: " + children.get(i).getType()); } - list.add(children.get(i).getStringValueForArray() + list.add(getStringLiteralForComplexType(children.get(i)) + ":" + getStringLiteralForComplexType(children.get(i + 1))); } return "{" + StringUtils.join(list, ", ") + "}"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java index 09e9bbe0b91e37..ac9d7c4427b0b0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnBE.java @@ -22,7 +22,9 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.Config; import org.apache.doris.common.IdGenerator; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.TimeUtils; @@ -33,18 +35,44 @@ import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.generator.TableGeneratingFunction; import org.apache.doris.nereids.trees.expressions.functions.scalar.Sleep; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Tokenize; +import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal; +import org.apache.doris.nereids.trees.expressions.literal.DateV2Literal; +import org.apache.doris.nereids.trees.expressions.literal.DecimalLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DecimalV3Literal; +import org.apache.doris.nereids.trees.expressions.literal.DoubleLiteral; +import org.apache.doris.nereids.trees.expressions.literal.FloatLiteral; +import org.apache.doris.nereids.trees.expressions.literal.IPv4Literal; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.expressions.literal.JsonLiteral; +import org.apache.doris.nereids.trees.expressions.literal.LargeIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.MapLiteral; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.literal.NumericLiteral; -import org.apache.doris.nereids.types.CharType; +import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; +import org.apache.doris.nereids.trees.expressions.literal.StructLiteral; +import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; +import org.apache.doris.nereids.types.ArrayType; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.DateTimeV2Type; -import org.apache.doris.nereids.types.DecimalV2Type; import org.apache.doris.nereids.types.DecimalV3Type; -import org.apache.doris.nereids.types.VarcharType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.StructField; +import org.apache.doris.nereids.types.StructType; import org.apache.doris.proto.InternalService; import org.apache.doris.proto.InternalService.PConstantExprResult; import org.apache.doris.proto.Types.PScalarType; +import org.apache.doris.proto.Types.PStructField; +import org.apache.doris.proto.Types.PTypeDesc; +import org.apache.doris.proto.Types.PTypeNode; +import org.apache.doris.proto.Types.PValues; import org.apache.doris.qe.ConnectContext; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.system.Backend; @@ -55,12 +83,18 @@ import org.apache.doris.thrift.TQueryGlobals; import org.apache.doris.thrift.TQueryOptions; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; +import com.google.common.net.InetAddresses; +import com.google.protobuf.ByteString; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.Inet4Address; +import java.time.DateTimeException; +import java.time.LocalDate; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Collections; @@ -68,7 +102,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Objects; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -84,8 +117,8 @@ public class FoldConstantRuleOnBE implements ExpressionPatternRuleFactory { public List> buildRules() { return ImmutableList.of( root(Expression.class) - .whenCtx(FoldConstantRuleOnBE::isEnableFoldByBe) - .thenApply(FoldConstantRuleOnBE::foldByBE) + .whenCtx(FoldConstantRuleOnBE::isEnableFoldByBe) + .thenApply(FoldConstantRuleOnBE::foldByBE) ); } @@ -156,12 +189,23 @@ private static void collectConst(Expression expr, Map constM return; } // eg: avg_state(1) return is agg function serialize data - if (expr.getDataType().isAggStateType()) { + // and some type can't find a literal to represent. + // time type: need add a time literal in nereids + // IPv6 type: need get a library to output the compressed address format + if (expr.getDataType().isAggStateType() || expr.getDataType().isObjectType() + || expr.getDataType().isVariantType() || expr.getDataType().isTimeLikeType() + || expr.getDataType().isIPv6Type()) { return; } - if (skipSleepFunction(expr)) { + if (skipSleepFunction(expr) || (expr instanceof TableGeneratingFunction)) { return; } + // Tokenize function want check the second child literal must be string type + // and properties format, it's a little special, + // maybe check in checkLegalityBeforeTypeCoercion function? + if (expr instanceof Tokenize) { + expr.checkLegalityAfterRewrite(); + } String id = idGenerator.getNextId().toString(); constMap.put(id, expr); Expr staleExpr; @@ -219,14 +263,14 @@ private static Map evalOnBE(Map> TQueryOptions tQueryOptions = new TQueryOptions(); tQueryOptions.setRepeatMaxNum(context.getSessionVariable().repeatMaxNum); + tQueryOptions.setBeExecVersion(Config.be_exec_version); TFoldConstantParams tParams = new TFoldConstantParams(paramMap, queryGlobals); tParams.setVecExec(true); tParams.setQueryOptions(tQueryOptions); tParams.setQueryId(context.queryId()); + tParams.setIsNereids(true); - // TODO: will be delete the debug log after find problem of timeout. - LOG.info("fold query {} ", DebugUtil.printId(context.queryId())); Future future = BackendServiceProxy.getInstance().foldConstantExpr(brpcAddress, tParams); PConstantExprResult result = future.get(5, TimeUnit.SECONDS); @@ -234,42 +278,18 @@ private static Map evalOnBE(Map> if (result.getStatus().getStatusCode() == 0) { for (Entry e : result.getExprResultMapMap().entrySet()) { for (Entry e1 : e.getValue().getMapMap().entrySet()) { - PScalarType pScalarType = e1.getValue().getType(); - TPrimitiveType tPrimitiveType = TPrimitiveType.findByValue(pScalarType.getType()); - PrimitiveType primitiveType = PrimitiveType.fromThrift(Objects.requireNonNull(tPrimitiveType)); Expression ret; if (e1.getValue().getSuccess()) { - DataType type; - if (PrimitiveType.ARRAY == primitiveType - || PrimitiveType.MAP == primitiveType - || PrimitiveType.STRUCT == primitiveType - || PrimitiveType.AGG_STATE == primitiveType) { + PTypeDesc pTypeDesc = e1.getValue().getTypeDesc(); + DataType type = convertToNereidsType(pTypeDesc.getTypesList(), 0).key(); + PValues resultContent = e1.getValue().getResultContent(); + List resultExpression = getResultExpression(type, resultContent); + if (resultExpression.isEmpty()) { ret = constMap.get(e1.getKey()); + LOG.debug("Be constant folding convert {} to {} failed query_id: {}", e1.getKey(), ret, + DebugUtil.printId(context.queryId())); } else { - if (primitiveType == PrimitiveType.CHAR) { - Preconditions.checkState(pScalarType.hasLen(), - "be return char type without len"); - type = CharType.createCharType(pScalarType.getLen()); - } else if (primitiveType == PrimitiveType.VARCHAR) { - Preconditions.checkState(pScalarType.hasLen(), - "be return varchar type without len"); - type = VarcharType.createVarcharType(pScalarType.getLen()); - } else if (primitiveType == PrimitiveType.DECIMALV2) { - type = DecimalV2Type.createDecimalV2Type( - pScalarType.getPrecision(), pScalarType.getScale()); - } else if (primitiveType == PrimitiveType.DATETIMEV2) { - type = DateTimeV2Type.of(pScalarType.getScale()); - } else if (primitiveType == PrimitiveType.DECIMAL32 - || primitiveType == PrimitiveType.DECIMAL64 - || primitiveType == PrimitiveType.DECIMAL128 - || primitiveType == PrimitiveType.DECIMAL256) { - type = DecimalV3Type.createDecimalV3TypeLooseCheck( - pScalarType.getPrecision(), pScalarType.getScale()); - } else { - type = DataType.fromCatalogType(ScalarType.createType( - PrimitiveType.fromThrift(tPrimitiveType))); - } - ret = Literal.of(e1.getValue().getContent()).castTo(type); + ret = resultExpression.get(0); } } else { ret = constMap.get(e1.getKey()); @@ -291,4 +311,304 @@ private static Map evalOnBE(Map> } return resultMap; } + + /** + * convert PValues which from BE to Expression of nereids + */ + public static List getResultExpression(DataType type, PValues resultContent) { + List res = new ArrayList<>(); + if (type.isNullType()) { + int num = resultContent.getNullMapCount(); + for (int i = 0; i < num; ++i) { + Literal literal = new NullLiteral(type); + res.add(literal); + } + } else if (type.isBooleanType()) { + int num = resultContent.getUint32ValueCount(); + for (int i = 0; i < num; ++i) { + Literal literal = BooleanLiteral.of(resultContent.getUint32Value(i) == 1); + res.add(literal); + } + } else if (type.isTinyIntType()) { + int num = resultContent.getInt32ValueCount(); + for (int i = 0; i < num; ++i) { + Literal literal = new TinyIntLiteral((byte) resultContent.getInt32Value(i)); + res.add(literal); + } + } else if (type.isSmallIntType()) { + int num = resultContent.getInt32ValueCount(); + for (int i = 0; i < num; ++i) { + Literal literal = new SmallIntLiteral((short) resultContent.getInt32Value(i)); + res.add(literal); + } + } else if (type.isIntegerType()) { + int num = resultContent.getInt32ValueCount(); + for (int i = 0; i < num; ++i) { + Literal literal = new IntegerLiteral(resultContent.getInt32Value(i)); + res.add(literal); + } + } else if (type.isBigIntType()) { + int num = resultContent.getInt64ValueCount(); + for (int i = 0; i < num; ++i) { + Literal literal = new BigIntLiteral(resultContent.getInt64Value(i)); + res.add(literal); + } + } else if (type.isLargeIntType()) { + int num = resultContent.getBytesValueCount(); + for (int i = 0; i < num; ++i) { + ByteString bytesValue = resultContent.getBytesValue(i); + byte[] bytes = convertByteOrder(bytesValue.toByteArray()); + BigInteger convertedBigInteger = new BigInteger(bytes); + Literal literal = new LargeIntLiteral(convertedBigInteger); + res.add(literal); + } + } else if (type.isFloatType()) { + int num = resultContent.getFloatValueCount(); + for (int i = 0; i < num; ++i) { + float value = resultContent.getFloatValue(i); + Literal literal = null; + if (Float.isNaN(value)) { + literal = new NullLiteral(type); + } else { + literal = new FloatLiteral(value); + } + res.add(literal); + } + } else if (type.isDoubleType()) { + int num = resultContent.getDoubleValueCount(); + for (int i = 0; i < num; ++i) { + double value = resultContent.getDoubleValue(i); + Literal literal = null; + if (Double.isNaN(value)) { + literal = new NullLiteral(type); + } else { + literal = new DoubleLiteral(value); + } + res.add(literal); + } + } else if (type.isDecimalV2Type()) { + int num = resultContent.getBytesValueCount(); + for (int i = 0; i < num; ++i) { + ByteString bytesValue = resultContent.getBytesValue(i); + byte[] bytes = convertByteOrder(bytesValue.toByteArray()); + BigInteger value = new BigInteger(bytes); + BigDecimal bigDecimal = new BigDecimal(value, 9); // decimalv2 scale always 9 + Literal literal = new DecimalLiteral(bigDecimal); + res.add(literal); + } + } else if (type.isDecimalV3Type()) { + int num = resultContent.getBytesValueCount(); + DecimalV3Type decimalV3Type = (DecimalV3Type) type; + for (int i = 0; i < num; ++i) { + ByteString bytesValue = resultContent.getBytesValue(i); + byte[] bytes = convertByteOrder(bytesValue.toByteArray()); + BigInteger value = new BigInteger(bytes); + BigDecimal bigDecimal = new BigDecimal(value, decimalV3Type.getScale()); + Literal literal = new DecimalV3Literal(decimalV3Type, bigDecimal); + res.add(literal); + } + } else if (type.isDateTimeV2Type()) { + int num = resultContent.getUint64ValueCount(); + for (int i = 0; i < num; ++i) { + long uint64Value = resultContent.getUint64Value(i); + LocalDateTime dateTimeV2 = convertToJavaDateTimeV2(uint64Value); + Literal literal = new DateTimeV2Literal((DateTimeV2Type) type, dateTimeV2.getYear(), + dateTimeV2.getMonthValue(), dateTimeV2.getDayOfMonth(), dateTimeV2.getHour(), + dateTimeV2.getMinute(), dateTimeV2.getSecond(), dateTimeV2.getNano() / 1000); + res.add(literal); + } + } else if (type.isDateV2Type()) { + int num = resultContent.getUint32ValueCount(); + for (int i = 0; i < num; ++i) { + int uint32Value = resultContent.getUint32Value(i); + LocalDate localDate = convertToJavaDateV2(uint32Value); + DateV2Literal dateV2Literal = new DateV2Literal(localDate.getYear(), localDate.getMonthValue(), + localDate.getDayOfMonth()); + res.add(dateV2Literal); + } + } else if (type.isIPv4Type()) { + int num = resultContent.getUint32ValueCount(); + for (int i = 0; i < num; ++i) { + Inet4Address inet4Address = InetAddresses.fromInteger(resultContent.getUint32Value(i)); + IPv4Literal iPv4Literal = new IPv4Literal(inet4Address.getHostAddress()); + res.add(iPv4Literal); + } + } else if (type.isJsonType()) { + int num = resultContent.getStringValueCount(); + for (int i = 0; i < num; ++i) { + String stringValue = resultContent.getStringValue(i); + // maybe need handle NULL_IN_CSV_FOR_ORDINARY_TYPE = "\\N"; + JsonLiteral jsonLiteral = new JsonLiteral(stringValue); + res.add(jsonLiteral); + } + } else if (type.isStringLikeType()) { + int num = resultContent.getStringValueCount(); + for (int i = 0; i < num; ++i) { + Literal literal = new StringLiteral(resultContent.getStringValue(i)); + res.add(literal); + } + } else if (type.isArrayType()) { + ArrayType arrayType = (ArrayType) type; + int childCount = resultContent.getChildElementCount(); + List allLiterals = new ArrayList<>(); + for (int i = 0; i < childCount; ++i) { + allLiterals.addAll(getResultExpression(arrayType.getItemType(), + resultContent.getChildElement(i))); + } + int offsetCount = resultContent.getChildOffsetCount(); + if (offsetCount == 1) { + ArrayLiteral arrayLiteral = new ArrayLiteral(allLiterals, arrayType); + res.add(arrayLiteral); + } else { + for (int i = 0; i < offsetCount; ++i) { + List childLiteral = new ArrayList<>(); + int startOffset = (int) ((i == 0) ? 0 : resultContent.getChildOffset(i - 1)); + int endOffset = (int) resultContent.getChildOffset(i); + for (int off = startOffset; off < endOffset; ++off) { + childLiteral.add(allLiterals.get(off)); + } + ArrayLiteral arrayLiteral = new ArrayLiteral(childLiteral, arrayType); + res.add(arrayLiteral); + } + } + } else if (type.isMapType()) { + MapType mapType = (MapType) type; + int childCount = resultContent.getChildElementCount(); + List allKeys = new ArrayList<>(); + List allValues = new ArrayList<>(); + for (int i = 0; i < childCount; i = i + 2) { + allKeys.addAll(getResultExpression(mapType.getKeyType(), + resultContent.getChildElement(i))); + allValues.addAll(getResultExpression(mapType.getValueType(), + resultContent.getChildElement(i + 1))); + } + int offsetCount = resultContent.getChildOffsetCount(); + if (offsetCount == 1) { + MapLiteral mapLiteral = new MapLiteral(allKeys, allValues, mapType); + res.add(mapLiteral); + } else { + for (int i = 0; i < offsetCount; ++i) { + List keyLiteral = new ArrayList<>(); + List valueLiteral = new ArrayList<>(); + int startOffset = (int) ((i == 0) ? 0 : resultContent.getChildOffset(i - 1)); + int endOffset = (int) resultContent.getChildOffset(i); + for (int off = startOffset; off < endOffset; ++off) { + keyLiteral.add(allKeys.get(off)); + valueLiteral.add(allValues.get(off)); + } + MapLiteral mapLiteral = new MapLiteral(keyLiteral, valueLiteral, mapType); + res.add(mapLiteral); + } + } + } else if (type.isStructType()) { + StructType structType = (StructType) type; + int childCount = resultContent.getChildElementCount(); + List> allFields = new ArrayList<>(); + for (int i = 0; i < childCount; ++i) { + allFields.add(getResultExpression(structType.getFields().get(i).getDataType(), + resultContent.getChildElement(i))); + } + for (int i = 0; i < allFields.get(0).size(); ++i) { + List fields = new ArrayList<>(); + for (int child = 0; child < childCount; ++child) { + fields.add(allFields.get(child).get(i)); + } + StructLiteral structLiteral = new StructLiteral(fields, structType); + res.add(structLiteral); + } + } else { + LOG.warn("the type: {} is not support, should implement it", type.toString()); + } + if (resultContent.hasHasNull()) { + for (int i = 0; i < resultContent.getNullMapCount(); ++i) { + if (resultContent.getNullMap(i)) { + res.set(i, new NullLiteral(type)); + } + } + } + return res; + } + + private static Pair convertToNereidsType(List typeNodes, int start) { + PScalarType pScalarType = typeNodes.get(start).getScalarType(); + boolean containsNull = typeNodes.get(start).getContainsNull(); + TPrimitiveType tPrimitiveType = TPrimitiveType.findByValue(pScalarType.getType()); + DataType type; + int parsedNodes; + if (tPrimitiveType == TPrimitiveType.ARRAY) { + Pair itemType = convertToNereidsType(typeNodes, start + 1); + type = ArrayType.of(itemType.key(), containsNull); + parsedNodes = 1 + itemType.value(); + } else if (tPrimitiveType == TPrimitiveType.MAP) { + Pair keyType = convertToNereidsType(typeNodes, start + 1); + Pair valueType = convertToNereidsType(typeNodes, start + 1 + keyType.value()); + type = MapType.of(keyType.key(), valueType.key()); + parsedNodes = 1 + keyType.value() + valueType.value(); + } else if (tPrimitiveType == TPrimitiveType.STRUCT) { + parsedNodes = 1; + ArrayList fields = new ArrayList<>(); + for (int i = 0; i < typeNodes.get(start).getStructFieldsCount(); ++i) { + Pair fieldType = convertToNereidsType(typeNodes, start + parsedNodes); + PStructField structField = typeNodes.get(start).getStructFields(i); + fields.add(new StructField(structField.getName(), fieldType.key(), + structField.getContainsNull(), + structField.getComment() == null ? "" : structField.getComment())); + parsedNodes += fieldType.value(); + } + type = new StructType(fields); + } else if (tPrimitiveType == TPrimitiveType.DECIMALV2) { + type = DataType.fromCatalogType(ScalarType.createDecimalType(PrimitiveType.fromThrift(tPrimitiveType), + pScalarType.getPrecision(), pScalarType.getScale())); + parsedNodes = 1; + } else { + type = DataType.fromCatalogType(ScalarType.createType(PrimitiveType.fromThrift(tPrimitiveType), + pScalarType.getLen(), pScalarType.getPrecision(), pScalarType.getScale())); + parsedNodes = 1; + } + return Pair.of(type, parsedNodes); + } + + private static LocalDateTime convertToJavaDateTimeV2(long time) { + int year = (int) (time >> 46); + int yearMonth = (int) (time >> 42); + int yearMonthDay = (int) (time >> 37); + + int month = (yearMonth & 0XF); + int day = (yearMonthDay & 0X1F); + + int hour = (int) ((time >> 32) & 0X1F); + int minute = (int) ((time >> 26) & 0X3F); + int second = (int) ((time >> 20) & 0X3F); + int microsecond = (int) (time & 0XFFFFF); + + try { + return LocalDateTime.of(year, month, day, hour, minute, second, microsecond * 1000); + } catch (DateTimeException e) { + return null; + } + } + + private static LocalDate convertToJavaDateV2(int date) { + int year = date >> 9; + int month = (date >> 5) & 0XF; + int day = date & 0X1F; + try { + return LocalDate.of(year, month, day); + } catch (DateTimeException e) { + return null; + } + } + + /** + * Change the order of the bytes, Because JVM is Big-Endian , x86 is Little-Endian. + */ + private static byte[] convertByteOrder(byte[] bytes) { + int length = bytes.length; + for (int i = 0; i < length / 2; ++i) { + byte temp = bytes[i]; + bytes[i] = bytes[length - 1 - i]; + bytes[length - 1 - i] = temp; + } + return bytes; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/MapLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/MapLiteral.java index 7dab827509bed4..c57bd3a04875e1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/MapLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/MapLiteral.java @@ -48,7 +48,10 @@ public MapLiteral(List keys, List values) { this(keys, values, computeDataType(keys, values)); } - private MapLiteral(List keys, List values, DataType dataType) { + /** + * create MAP Literal with keys, values and datatype + */ + public MapLiteral(List keys, List values, DataType dataType) { super(dataType); this.keys = ImmutableList.copyOf(Objects.requireNonNull(keys, "keys should not be null")); this.values = ImmutableList.copyOf(Objects.requireNonNull(values, "values should not be null")); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StructLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StructLiteral.java index 2fd2186f4965eb..3a46f1f5b83e7e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StructLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/StructLiteral.java @@ -49,7 +49,10 @@ public StructLiteral(List fields) { this(fields, computeDataType(fields)); } - private StructLiteral(List fields, DataType dataType) { + /** + * create Struct Literal with fields and datatype + */ + public StructLiteral(List fields, DataType dataType) { super(dataType); this.fields = ImmutableList.copyOf(Objects.requireNonNull(fields, "fields should not be null")); Preconditions.checkArgument(dataType instanceof StructType, diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FoldConstantsRule.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FoldConstantsRule.java index 3ef20a5651d258..dd37c2fc7b178a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rewrite/FoldConstantsRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/FoldConstantsRule.java @@ -384,6 +384,7 @@ private Map> calcConstExpr(Map future = BackendServiceProxy.getInstance().foldConstantExpr(brpcAddress, tParams); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/MapLiteralTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/MapLiteralTest.java index 700c54253e4972..61e77dbc907016 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/MapLiteralTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/MapLiteralTest.java @@ -116,26 +116,26 @@ public void testGetStringValueForArray() throws AnalysisException { @Test public void testGetStringInFe() throws AnalysisException { MapLiteral mapLiteral1 = new MapLiteral(intLiteral1, floatLiteral); - Assert.assertEquals("{\"1\":2.15}", mapLiteral1.getStringValueInFe()); + Assert.assertEquals("{1:2.15}", mapLiteral1.getStringValueInFe()); MapLiteral mapLiteral11 = new MapLiteral(intLiteral1, floatLiteral1); - Assert.assertEquals("{\"1\":\"11:22:33\"}", mapLiteral11.getStringValueInFe()); + Assert.assertEquals("{1:\"11:22:33\"}", mapLiteral11.getStringValueInFe()); MapLiteral mapLiteral2 = new MapLiteral(boolLiteral, stringLiteral); - Assert.assertEquals("{\"1\":\"shortstring\"}", mapLiteral2.getStringValueInFe()); + Assert.assertEquals("{1:\"shortstring\"}", mapLiteral2.getStringValueInFe()); MapLiteral mapLiteral3 = new MapLiteral(largeIntLiteral, dateLiteral); - Assert.assertEquals("{\"1000000000000000000000\":\"2022-10-10\"}", mapLiteral3.getStringValueInFe()); + Assert.assertEquals("{1000000000000000000000:\"2022-10-10\"}", mapLiteral3.getStringValueInFe()); MapLiteral mapLiteral4 = new MapLiteral(floatLiteral1, nullLiteral); Assert.assertEquals("{\"11:22:33\":null}", mapLiteral4.getStringValueInFe()); MapLiteral mapLiteral5 = new MapLiteral(datetimeLiteral, dateLiteral); Assert.assertEquals("{\"2022-10-10 12:10:10\":\"2022-10-10\"}", mapLiteral5.getStringValueInFe()); MapLiteral mapLiteral6 = new MapLiteral(decimalLiteral1, decimalLiteral2); - Assert.assertEquals("{\"1.0\":2}", mapLiteral6.getStringValueInFe()); + Assert.assertEquals("{1.0:2}", mapLiteral6.getStringValueInFe()); MapLiteral mapLiteral7 = new MapLiteral(); Assert.assertEquals("{}", mapLiteral7.getStringValueInFe()); MapLiteral mapLiteral8 = new MapLiteral(nullLiteral, intLiteral1); Assert.assertEquals("{null:1}", mapLiteral8.getStringValueInFe()); MapLiteral mapLiteral9 = new MapLiteral(intLiteral1, nullLiteral); - Assert.assertEquals("{\"1\":null}", mapLiteral9.getStringValueInFe()); + Assert.assertEquals("{1:null}", mapLiteral9.getStringValueInFe()); MapLiteral mapLiteral10 = new MapLiteral(intLiteral1, arrayLiteral); Assert.assertEquals("{\"1\":[\"1\", \"2.15\"]}", mapLiteral10.getStringValueForArray()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/StructLiteralTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/StructLiteralTest.java index b7f2577ad632d9..c48ad5d5785e4a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/StructLiteralTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/StructLiteralTest.java @@ -86,9 +86,9 @@ public void testGetStringInFe() throws AnalysisException { stringLiteral, largeIntLiteral, decimalLiteral1, decimalLiteral2, dateLiteral, datetimeLiteral); Assert.assertEquals("{\"col1\": 1, \"col2\": 2.15, \"col3\": \"11:22:33\", \"col4\": 1, \"col5\": " + "\"shortstring\", \"col6\": 1000000000000000000000, \"col7\": 1.0, \"col8\": 2, \"col9\": \"2022-10-10\", \"col10\": \"2022-10-10 12:10:10\"}", - structLiteral1.getStringValueInFe()); + structLiteral1.getStringValueInFe()); StructLiteral structLiteral2 = new StructLiteral(arrayLiteral, mapLiteral, structLiteral); - Assert.assertEquals("{\"col1\": [1.0, 2.15], \"col2\": {\"1\":2.15}, \"col3\": " + Assert.assertEquals("{\"col1\": [1.0, 2.15], \"col2\": {1:2.15}, \"col3\": " + "{\"col1\": 1, \"col2\": 2.15, \"col3\": 1.0, \"col4\": \"2022-10-10\"}}", structLiteral2.getStringValueInFe()); StructLiteral structLiteral3 = new StructLiteral(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/LiteralTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/LiteralTest.java index 842846bbd1bd8d..fcb64ff0bface9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/LiteralTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/LiteralTest.java @@ -17,15 +17,29 @@ package org.apache.doris.nereids.trees.expressions; +import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; +import org.apache.doris.nereids.types.ArrayType; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.StructField; +import org.apache.doris.nereids.types.StructType; +import org.apache.doris.proto.Types.PGenericType; +import org.apache.doris.proto.Types.PGenericType.TypeId; +import org.apache.doris.proto.Types.PValues; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + class LiteralTest { @Test @@ -42,4 +56,255 @@ public void testEqual() { Assertions.assertTrue(Literal.of(1) instanceof IntegerLiteral); Assertions.assertTrue(Literal.of(false) instanceof BooleanLiteral); } + + @Test + public void testGetResultExpressionArrayInt() { + int num = 10; + Integer[] elementsArray = new Integer[num]; + for (int i = 0; i < elementsArray.length; ++i) { + elementsArray[i] = i; + } + DataType arrayType = ArrayType.of(IntegerType.INSTANCE, true); + PGenericType.Builder childTypeBuilder = PGenericType.newBuilder(); + childTypeBuilder.setId(TypeId.INT32); + PGenericType.Builder typeBuilder = PGenericType.newBuilder(); + typeBuilder.setId(TypeId.LIST); + + PValues.Builder childBuilder = PValues.newBuilder(); + PValues.Builder resultContentBuilder = PValues.newBuilder(); + for (int value : elementsArray) { + childBuilder.addInt32Value(value); + } + childBuilder.setType(childTypeBuilder.build()); + PValues childValues = childBuilder.build(); + resultContentBuilder.setType(typeBuilder.build()); + resultContentBuilder.addChildElement(childValues); + resultContentBuilder.addChildOffset(10); + PValues resultContent = resultContentBuilder.build(); + List resultExpression + = org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnBE.getResultExpression(arrayType, + resultContent); + Assertions.assertTrue(resultExpression.get(0).isLiteral()); + } + + @Test + public void testGetResultExpressionArrayArrayInt() { + int num = 10; + Integer[] elementsArray = new Integer[num]; + for (int i = 0; i < elementsArray.length; ++i) { + elementsArray[i] = i; + } + DataType nestedArrayType = ArrayType.of(IntegerType.INSTANCE, true); + DataType outArrayType = ArrayType.of(nestedArrayType, true); + PGenericType.Builder childTypeBuilder = PGenericType.newBuilder(); + childTypeBuilder.setId(TypeId.INT32); + PGenericType.Builder typeBuilder = PGenericType.newBuilder(); + typeBuilder.setId(TypeId.LIST); + PGenericType.Builder outTypeBuilder = PGenericType.newBuilder(); + outTypeBuilder.setId(TypeId.LIST); + + PValues.Builder childBuilder = PValues.newBuilder(); + PValues.Builder nestedContentBuilder = PValues.newBuilder(); + PValues.Builder outContentBuilder = PValues.newBuilder(); + for (int value : elementsArray) { + childBuilder.addInt32Value(value); + } + childBuilder.setType(childTypeBuilder.build()); + PValues childValues = childBuilder.build(); + nestedContentBuilder.setType(typeBuilder.build()); + nestedContentBuilder.addChildElement(childValues); + nestedContentBuilder.addChildOffset(10); + PValues nestedResultContent = nestedContentBuilder.build(); + outContentBuilder.setType(outTypeBuilder.build()); + outContentBuilder.addChildElement(nestedResultContent); + outContentBuilder.addChildOffset(1); + PValues resultContent = outContentBuilder.build(); + List resultExpression + = org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnBE.getResultExpression(outArrayType, + resultContent); + // [[0, 1, 2, 3, 4, 5, 6, 7, 8, 9]] + Assertions.assertTrue(resultExpression.get(0) instanceof ArrayLiteral); + System.out.println(resultExpression.get(0).toString()); + } + + @Test + public void testGetResultExpressionArrayArrayInt2() { + int num = 10; + Integer[] elementsArray = new Integer[num]; + for (int i = 0; i < elementsArray.length; ++i) { + elementsArray[i] = i; + } + DataType nestedArrayType = ArrayType.of(IntegerType.INSTANCE, true); + DataType outArrayType = ArrayType.of(nestedArrayType, true); + PGenericType.Builder childTypeBuilder = PGenericType.newBuilder(); + childTypeBuilder.setId(TypeId.INT32); + PGenericType.Builder typeBuilder = PGenericType.newBuilder(); + typeBuilder.setId(TypeId.LIST); + PGenericType.Builder outTypeBuilder = PGenericType.newBuilder(); + outTypeBuilder.setId(TypeId.LIST); + + PValues.Builder childBuilder1 = PValues.newBuilder(); + PValues.Builder childBuilder2 = PValues.newBuilder(); + PValues.Builder nestedContentBuilder = PValues.newBuilder(); + PValues.Builder outContentBuilder = PValues.newBuilder(); + for (int i = 0; i < elementsArray.length; i = i + 2) { + childBuilder1.addInt32Value(elementsArray[i]); + childBuilder2.addInt32Value(elementsArray[i + 1]); + } + childBuilder1.setType(childTypeBuilder.build()); + childBuilder2.setType(childTypeBuilder.build()); + PValues childValues1 = childBuilder1.build(); + PValues childValues2 = childBuilder2.build(); + nestedContentBuilder.setType(typeBuilder.build()); + nestedContentBuilder.addChildElement(childValues1); + nestedContentBuilder.addChildElement(childValues2); + nestedContentBuilder.addChildOffset(5); + nestedContentBuilder.addChildOffset(10); + PValues nestedResultContent = nestedContentBuilder.build(); + outContentBuilder.setType(outTypeBuilder.build()); + outContentBuilder.addChildElement(nestedResultContent); + outContentBuilder.addChildOffset(2); + PValues resultContent = outContentBuilder.build(); + List resultExpression + = org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnBE.getResultExpression(outArrayType, + resultContent); + // [[0, 2, 4, 6, 8], [1, 3, 5, 7, 9]] + Assertions.assertTrue(resultExpression.get(0) instanceof ArrayLiteral); + System.out.println(resultExpression.get(0).toString()); + } + + @Test + public void testGetResultExpressionArrayNull() { + int num = 10; + Integer[] elementsArray = new Integer[num]; + Boolean[] nullMap = new Boolean[num]; + for (int i = 0; i < elementsArray.length; ++i) { + elementsArray[i] = i; + nullMap[i] = (i % 2 == 1); + } + DataType arrayType = ArrayType.of(IntegerType.INSTANCE, true); + PGenericType.Builder childTypeBuilder = PGenericType.newBuilder(); + childTypeBuilder.setId(TypeId.INT32); + PGenericType.Builder typeBuilder = PGenericType.newBuilder(); + typeBuilder.setId(TypeId.LIST); + + PValues.Builder childBuilder = PValues.newBuilder(); + PValues.Builder resultContentBuilder = PValues.newBuilder(); + for (int value : elementsArray) { + childBuilder.addInt32Value(value); + } + childBuilder.setType(childTypeBuilder.build()); + childBuilder.setHasNull(true); + childBuilder.addAllNullMap(Arrays.asList(nullMap)); + PValues childValues = childBuilder.build(); + resultContentBuilder.setType(typeBuilder.build()); + resultContentBuilder.addChildElement(childValues); + resultContentBuilder.addChildOffset(10); + PValues resultContent = resultContentBuilder.build(); + List resultExpression + = org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnBE.getResultExpression(arrayType, + resultContent); + // [0, NULL, 2, NULL, 4, NULL, 6, NULL, 8, NULL] + Assertions.assertTrue(resultExpression.get(0).isLiteral()); + System.out.println(resultExpression.get(0).toString()); + } + + @Test + public void testGetResultExpressionStruct() { + int num = 10; + Integer[] elementsArray = new Integer[num]; + for (int i = 0; i < elementsArray.length; ++i) { + elementsArray[i] = i; + } + List typeFields = new ArrayList<>(); + typeFields.add(new StructField("col1", IntegerType.INSTANCE, true, "comment1")); + typeFields.add(new StructField("col2", StringType.INSTANCE, true, "comment1")); + + DataType structType = new StructType(typeFields); + PGenericType.Builder childTypeBuilder1 = PGenericType.newBuilder(); + childTypeBuilder1.setId(TypeId.INT32); + PGenericType.Builder childTypeBuilder2 = PGenericType.newBuilder(); + childTypeBuilder2.setId(TypeId.STRING); + PGenericType.Builder typeBuilder = PGenericType.newBuilder(); + typeBuilder.setId(TypeId.STRUCT); + + PValues.Builder childBuilder1 = PValues.newBuilder(); + PValues.Builder childBuilder2 = PValues.newBuilder(); + PValues.Builder resultContentBuilder = PValues.newBuilder(); + for (int i = 0; i < elementsArray.length; i = i + 2) { + childBuilder1.addInt32Value(elementsArray[i]); + childBuilder2.addStringValue("str" + (i + 1)); + } + childBuilder1.setType(childTypeBuilder1.build()); + childBuilder2.setType(childTypeBuilder2.build()); + PValues childValues1 = childBuilder1.build(); + PValues childValues2 = childBuilder2.build(); + + resultContentBuilder.setType(typeBuilder.build()); + resultContentBuilder.addChildElement(childValues1); + resultContentBuilder.addChildElement(childValues2); + PValues resultContent = resultContentBuilder.build(); + List resultExpression + = org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnBE.getResultExpression(structType, + resultContent); + Assertions.assertTrue(resultExpression.get(0).isLiteral()); + // STRUCT('col1':0,'col2':'str1') + System.out.println(resultExpression.get(0).toString()); + } + + @Test + public void testGetResultExpressionStructArray() { + int num = 10; + Integer[] elementsArray = new Integer[num]; + for (int i = 0; i < elementsArray.length; ++i) { + elementsArray[i] = i; + } + List typeFields = new ArrayList<>(); + typeFields.add(new StructField("col1", ArrayType.of(IntegerType.INSTANCE), true, "comment1")); + typeFields.add(new StructField("col2", ArrayType.of(StringType.INSTANCE), true, "comment1")); + + DataType structType = new StructType(typeFields); + PGenericType.Builder childTypeBuilder1 = PGenericType.newBuilder(); + childTypeBuilder1.setId(TypeId.INT32); + PGenericType.Builder childTypeBuilder2 = PGenericType.newBuilder(); + childTypeBuilder2.setId(TypeId.STRING); + PGenericType.Builder childTypeBuilder3 = PGenericType.newBuilder(); + childTypeBuilder3.setId(TypeId.LIST); + PGenericType.Builder typeBuilder = PGenericType.newBuilder(); + typeBuilder.setId(TypeId.STRUCT); + + PValues.Builder childBuilder1 = PValues.newBuilder(); + PValues.Builder childBuilder2 = PValues.newBuilder(); + PValues.Builder arrayChildBuilder1 = PValues.newBuilder(); + PValues.Builder arrayChildBuilder2 = PValues.newBuilder(); + PValues.Builder resultContentBuilder = PValues.newBuilder(); + for (int i = 0; i < elementsArray.length; i = i + 2) { + childBuilder1.addInt32Value(elementsArray[i]); + childBuilder2.addStringValue("str" + (i + 1)); + } + childBuilder1.setType(childTypeBuilder1.build()); + childBuilder2.setType(childTypeBuilder2.build()); + arrayChildBuilder1.setType(childTypeBuilder3.build()); + arrayChildBuilder2.setType(childTypeBuilder3.build()); + + PValues childValues1 = childBuilder1.build(); + PValues childValues2 = childBuilder2.build(); + arrayChildBuilder1.addChildElement(childValues1); + arrayChildBuilder1.addChildOffset(5); + arrayChildBuilder2.addChildElement(childValues2); + arrayChildBuilder2.addChildOffset(5); + PValues arrayChildValues1 = arrayChildBuilder1.build(); + PValues arrayChildValues2 = arrayChildBuilder2.build(); + + resultContentBuilder.setType(typeBuilder.build()); + resultContentBuilder.addChildElement(arrayChildValues1); + resultContentBuilder.addChildElement(arrayChildValues2); + PValues resultContent = resultContentBuilder.build(); + List resultExpression + = org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnBE.getResultExpression(structType, + resultContent); + Assertions.assertTrue(resultExpression.get(0).isLiteral()); + // STRUCT('col1':[0, 2, 4, 6, 8],'col2':['str1', 'str3', 'str5', 'str7', 'str9']) + System.out.println(resultExpression.get(0).toString()); + } } diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 97bd7f5552e6b6..b71ddfa21a36eb 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -548,6 +548,7 @@ struct TFoldConstantParams { 3: optional bool vec_exec 4: optional TQueryOptions query_options 5: optional Types.TUniqueId query_id + 6: optional bool is_nereids } // TransmitData diff --git a/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_inside_join.groovy b/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_inside_join.groovy index 97fdc1c0b4f24e..cd8ed1a25a04ea 100644 --- a/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_inside_join.groovy +++ b/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_inside_join.groovy @@ -23,6 +23,7 @@ suite("push_filter_inside_join") { sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "SET disable_join_reorder=true" sql 'set be_number_for_test=3' + sql "set enable_fold_constant_by_be=false"//plan shape will be different // Push down > condition to cross join qt_pushdown_cross_join""" diff --git a/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_through.groovy b/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_through.groovy index 16c06b8edd96f3..d084d3240da198 100644 --- a/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_through.groovy +++ b/regression-test/suites/nereids_rules_p0/filter_push_down/push_filter_through.groovy @@ -24,6 +24,7 @@ suite("push_filter_through") { sql 'set be_number_for_test=3' sql "SET ignore_shape_nodes='PhysicalDistribute, PhysicalProject'" sql "set enable_fold_nondeterministic_fn=false" + sql "set enable_fold_constant_by_be=false"//plan shape will be different // push filter through alias qt_filter_project_alias""" From a49cce1c2f4ef76fb646ae3ad6e6ae0a04b8121e Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Fri, 12 Apr 2024 00:19:40 +0800 Subject: [PATCH 10/71] [Chore](code comment) add comment for #32999 (#33548) --- .../doris/nereids/glue/translator/PlanTranslatorContext.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java index eb0598d4894a68..31e54a92ab95a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java @@ -289,6 +289,8 @@ public SlotDescriptor createSlotDesc(TupleDescriptor tupleDesc, SlotReference sl slotRef = new SlotRef(slotDescriptor); if (slotReference.hasSubColPath()) { slotDescriptor.setSubColLables(slotReference.getSubColPath()); + // use lower case name for variant's root, since backend treat parent column as lower case + // see issue: https://github.com/apache/doris/pull/32999/commits slotDescriptor.setMaterializedColumnName(slotRef.getColumnName().toLowerCase() + "." + String.join(".", slotReference.getSubColPath())); } From 92ade6eb3e5eb0726cf122bfb2c69d4e36be8048 Mon Sep 17 00:00:00 2001 From: amory Date: Fri, 12 Apr 2024 09:49:32 +0800 Subject: [PATCH 11/71] [FIX](regresstest) fix array_range case for regress test (#33337) --- .../scalar_function/Array.groovy | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/regression-test/suites/nereids_function_p0/scalar_function/Array.groovy b/regression-test/suites/nereids_function_p0/scalar_function/Array.groovy index e1b7d5d8d4bdeb..14a9453632a906 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/Array.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/Array.groovy @@ -617,12 +617,12 @@ suite("nereids_scalar_fn_Array") { order_qt_sql_array_pushfront_DateV2_notnull "select array_pushfront(kadtv2, kdtv2) from fn_test_not_nullable" // array_range - order_qt_sql_array_range_one_param "select array_range(kint) from fn_test" - order_qt_sql_array_range_one_param_notnull "select array_range(kint) from fn_test_not_nullable" - order_qt_sql_array_range_two_param "select array_range(kint, 1000) from fn_test" - order_qt_sql_array_range_two_param_notnull "select array_range(kint, 1000) from fn_test_not_nullable" - order_qt_sql_array_range_three_param "select array_range(kint, 10000, ktint) from fn_test" - order_qt_sql_array_range_three_param_notnull "select array_range(kint, 10000, ktint) from fn_test_not_nullable" + order_qt_sql_array_range_one_param "select array_range(kint) from fn_test order by id" + order_qt_sql_array_range_one_param_notnull "select array_range(kint) from fn_test_not_nullable order by id" + order_qt_sql_array_range_two_param "select array_range(kint, 1000) from fn_test order by id" + order_qt_sql_array_range_two_param_notnull "select array_range(kint, 1000) from fn_test_not_nullable order by id" + order_qt_sql_array_range_three_param "select array_range(kint, 10000, ktint) from fn_test order by id" + order_qt_sql_array_range_three_param_notnull "select array_range(kint, 10000, ktint) from fn_test_not_nullable order by id" // array_remove order_qt_sql_array_remove_Double "select array_remove(kadbl, kdbl) from fn_test" @@ -1274,4 +1274,4 @@ suite("nereids_scalar_fn_Array") { qt_sequence_datetime_hour """select sequence(kdtmv2s1, date_add(kdtmv2s1, interval kint-3 hour), interval kint hour) from fn_test order by kdtmv2s1;""" qt_sequence_datetime_minute """select sequence(kdtmv2s1, date_add(kdtmv2s1, interval kint+1 minute), interval kint minute) from fn_test order by kdtmv2s1;""" qt_sequence_datetime_second """select sequence(kdtmv2s1, date_add(kdtmv2s1, interval kint second), interval kint-1 second) from fn_test order by kdtmv2s1;""" -} \ No newline at end of file +} From 9f8d4a5e61d1bee4c9960030d9daace90469f69a Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 12 Apr 2024 09:56:28 +0800 Subject: [PATCH 12/71] [refactor](heap sort) Simplify sorted block view (#33477) --- be/src/vec/common/sort/heap_sorter.cpp | 16 ++++----- be/src/vec/core/sort_cursor.h | 47 +++++--------------------- 2 files changed, 16 insertions(+), 47 deletions(-) diff --git a/be/src/vec/common/sort/heap_sorter.cpp b/be/src/vec/common/sort/heap_sorter.cpp index 19de3838301452..01db368e980a12 100644 --- a/be/src/vec/common/sort/heap_sorter.cpp +++ b/be/src/vec/common/sort/heap_sorter.cpp @@ -84,18 +84,18 @@ Status HeapSorter::append_block(Block* block) { Block tmp_block = block->clone_empty(); tmp_block.swap(*block); size_t num_rows = tmp_block.rows(); - HeapSortCursorBlockView block_view_val(std::move(tmp_block), _sort_description); - auto* block_view = new SharedHeapSortCursorBlockView(std::move(block_view_val)); - block_view->ref(); - Defer defer([&] { block_view->unref(); }); + auto block_view = + std::make_shared(std::move(tmp_block), _sort_description); + bool filtered = false; if (_heap_size == _heap->size()) { { SCOPED_TIMER(_topn_filter_timer); - _do_filter(block_view->value(), num_rows); + _do_filter(*block_view, num_rows); } - size_t remain_rows = block_view->value().block.rows(); + size_t remain_rows = block_view->block.rows(); _topn_filter_rows += (num_rows - remain_rows); COUNTER_SET(_topn_filter_rows_counter, _topn_filter_rows); + filtered = remain_rows == 0; for (size_t i = 0; i < remain_rows; ++i) { HeapSortCursorImpl cursor(i, block_view); _heap->replace_top_if_less(std::move(cursor)); @@ -114,8 +114,8 @@ Status HeapSorter::append_block(Block* block) { _heap->replace_top_if_less(std::move(cursor)); } } - if (block_view->ref_count() > 1) { - _data_size += block_view->value().block.allocated_bytes(); + if (!filtered) { + _data_size += block_view->block.allocated_bytes(); } return Status::OK(); } diff --git a/be/src/vec/core/sort_cursor.h b/be/src/vec/core/sort_cursor.h index a7be72eb9ab982..127ef35c4345c8 100644 --- a/be/src/vec/core/sort_cursor.h +++ b/be/src/vec/core/sort_cursor.h @@ -58,43 +58,16 @@ struct HeapSortCursorBlockView { } }; -// Use `SharedHeapSortCursorBlockView` for `HeapSortCursorBlockView` instead of shared_ptr because there will be no -// concurrent operation for `HeapSortCursorBlockView` and we don't need the lock inside shared_ptr -class SharedHeapSortCursorBlockView { -public: - SharedHeapSortCursorBlockView(HeapSortCursorBlockView&& reference) - : _ref_count(0), _reference(std::move(reference)) {} - SharedHeapSortCursorBlockView(const SharedHeapSortCursorBlockView&) = delete; - void unref() noexcept { - DCHECK_GT(_ref_count, 0); - _ref_count--; - if (_ref_count == 0) { - delete this; - } - } - void ref() noexcept { _ref_count++; } - - HeapSortCursorBlockView& value() { return _reference; } - - int ref_count() const { return _ref_count; } - -private: - ~SharedHeapSortCursorBlockView() noexcept = default; - int _ref_count; - HeapSortCursorBlockView _reference; -}; +using HeapSortCursorBlockSPtr = std::shared_ptr; struct HeapSortCursorImpl { public: - HeapSortCursorImpl(int row_id, SharedHeapSortCursorBlockView* block_view) - : _row_id(row_id), _block_view(block_view) { - block_view->ref(); - } + HeapSortCursorImpl(int row_id, HeapSortCursorBlockSPtr block_view) + : _row_id(row_id), _block_view(block_view) {} HeapSortCursorImpl(const HeapSortCursorImpl& other) { _row_id = other._row_id; _block_view = other._block_view; - _block_view->ref(); } HeapSortCursorImpl(HeapSortCursorImpl&& other) { @@ -109,19 +82,15 @@ struct HeapSortCursorImpl { return *this; } - ~HeapSortCursorImpl() { - if (_block_view) { - _block_view->unref(); - } - } + ~HeapSortCursorImpl() = default; size_t row_id() const { return _row_id; } - const ColumnRawPtrs& sort_columns() const { return _block_view->value().sort_columns; } + const ColumnRawPtrs& sort_columns() const { return _block_view->sort_columns; } - const Block* block() const { return &_block_view->value().block; } + const Block* block() const { return &_block_view->block; } - const SortDescription& sort_desc() const { return _block_view->value().desc; } + const SortDescription& sort_desc() const { return _block_view->desc; } bool operator<(const HeapSortCursorImpl& rhs) const { for (size_t i = 0; i < sort_desc().size(); ++i) { @@ -143,7 +112,7 @@ struct HeapSortCursorImpl { private: size_t _row_id; - SharedHeapSortCursorBlockView* _block_view; + HeapSortCursorBlockSPtr _block_view; }; /** Cursor allows to compare rows in different blocks (and parts). From 015cca5ef4c8f0156f4ceea175b3494a00b2f057 Mon Sep 17 00:00:00 2001 From: minghong Date: Fri, 12 Apr 2024 09:59:06 +0800 Subject: [PATCH 13/71] [fix](nereids) translate rf target expression using targetSlot #33531 --- .../translator/RuntimeFilterTranslator.java | 33 +++++++++---------- 1 file changed, 15 insertions(+), 18 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java index a19e97e3206b28..3dbd6cfcec7917 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java @@ -22,7 +22,6 @@ import org.apache.doris.analysis.SlotId; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TupleId; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.processor.post.RuntimeFilterContext; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; @@ -39,9 +38,9 @@ import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TRuntimeFilterType; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; import java.util.ArrayList; import java.util.List; @@ -73,20 +72,15 @@ public void translateRuntimeFilterTarget(Slot slot, ScanNode node, PlanTranslato } private class RuntimeFilterExpressionTranslator extends ExpressionTranslator { - Map slotExprMap; + SlotRef targetSlotRef; - RuntimeFilterExpressionTranslator(Map slotExprMap) { - this.slotExprMap = slotExprMap; + RuntimeFilterExpressionTranslator(SlotRef targetSlotRef) { + this.targetSlotRef = targetSlotRef; } @Override public Expr visitSlotReference(SlotReference slotReference, PlanTranslatorContext context) { - Expr expr = slotExprMap.get(slotReference); - if (expr instanceof SlotRef) { - return expr; - } else { - throw new AnalysisException("cannot find SlotRef for " + slotReference); - } + return targetSlotRef; } } @@ -110,8 +104,8 @@ public void createLegacyRuntimeFilter(RuntimeFilter filter, JoinNodeBase node, P for (int i = 0; i < filter.getTargetExpressions().size(); i++) { Slot curTargetSlot = filter.getTargetSlots().get(i); Expression curTargetExpression = filter.getTargetExpressions().get(i); - Expr target = context.getExprIdToOlapScanNodeSlotRef().get(curTargetSlot.getExprId()); - if (target == null) { + SlotRef targetSlotRef = context.getExprIdToOlapScanNodeSlotRef().get(curTargetSlot.getExprId()); + if (targetSlotRef == null) { context.setTargetNullCount(); hasInvalidTarget = true; break; @@ -119,12 +113,15 @@ public void createLegacyRuntimeFilter(RuntimeFilter filter, JoinNodeBase node, P ScanNode scanNode = context.getScanNodeOfLegacyRuntimeFilterTarget().get(curTargetSlot); Expr targetExpr; if (curTargetSlot.equals(curTargetExpression)) { - targetExpr = target; + targetExpr = targetSlotRef; } else { // map nereids target slot to original planner slot - Map slotMap = Maps.newHashMap(); - slotMap.put(curTargetSlot, target); - RuntimeFilterExpressionTranslator translator = new RuntimeFilterExpressionTranslator(slotMap); + Preconditions.checkArgument(curTargetExpression.getInputSlots().size() == 1, + "target expression is invalid, input slot num > 1; filter :" + filter); + Slot slotInTargetExpression = curTargetExpression.getInputSlots().iterator().next(); + Preconditions.checkArgument(slotInTargetExpression.equals(curTargetSlot) + || curTargetSlot.equals(context.getAliasTransferMap().get(slotInTargetExpression).second)); + RuntimeFilterExpressionTranslator translator = new RuntimeFilterExpressionTranslator(targetSlotRef); targetExpr = curTargetExpression.accept(translator, ctx); } @@ -132,7 +129,7 @@ public void createLegacyRuntimeFilter(RuntimeFilter filter, JoinNodeBase node, P if (!src.getType().equals(targetExpr.getType()) && filter.getType() != TRuntimeFilterType.BITMAP) { targetExpr = new CastExpr(src.getType(), targetExpr); } - SlotRef targetSlot = target.getSrcSlotRef(); + SlotRef targetSlot = targetSlotRef.getSrcSlotRef(); TupleId targetTupleId = targetSlot.getDesc().getParent().getId(); SlotId targetSlotId = targetSlot.getSlotId(); scanNodeList.add(scanNode); From 2ae4bfa7bb3fdc8845b6f9dc850bffada03cbb92 Mon Sep 17 00:00:00 2001 From: AlexYue Date: Fri, 12 Apr 2024 10:03:05 +0800 Subject: [PATCH 14/71] [feature](Cloud) Alter be to fetch vault info after creating storage vault (#33558) --- be/src/cloud/cloud_internal_service.cpp | 10 ++++ be/src/cloud/cloud_internal_service.h | 5 ++ be/src/cloud/cloud_storage_engine.cpp | 48 ++++++++++--------- be/src/cloud/cloud_storage_engine.h | 2 + .../java/org/apache/doris/catalog/Env.java | 2 +- .../apache/doris/catalog/StorageVaultMgr.java | 25 +++++++++- .../doris/rpc/BackendServiceClient.java | 5 ++ .../apache/doris/rpc/BackendServiceProxy.java | 14 ++++++ .../cloud/catalog/HdfsStorageVaultTest.java | 3 +- gensrc/proto/internal_service.proto | 7 +++ 10 files changed, 96 insertions(+), 25 deletions(-) diff --git a/be/src/cloud/cloud_internal_service.cpp b/be/src/cloud/cloud_internal_service.cpp index 0ebbe1c20c61f7..aba178bb34d852 100644 --- a/be/src/cloud/cloud_internal_service.cpp +++ b/be/src/cloud/cloud_internal_service.cpp @@ -17,6 +17,8 @@ #include "cloud/cloud_internal_service.h" +#include "cloud/cloud_storage_engine.h" + namespace doris { CloudInternalServiceImpl::CloudInternalServiceImpl(CloudStorageEngine& engine, ExecEnv* exec_env) @@ -24,4 +26,12 @@ CloudInternalServiceImpl::CloudInternalServiceImpl(CloudStorageEngine& engine, E CloudInternalServiceImpl::~CloudInternalServiceImpl() = default; +void CloudInternalServiceImpl::alter_vault_sync(google::protobuf::RpcController* controller, + const doris::PAlterVaultSyncRequest* request, + PAlterVaultSyncResponse* response, + google::protobuf::Closure* done) { + LOG(INFO) << "alter be to sync vault info from Meta Service"; + _engine.sync_storage_vault(); +} + } // namespace doris diff --git a/be/src/cloud/cloud_internal_service.h b/be/src/cloud/cloud_internal_service.h index 6399a8923fa052..1bc6806c57f635 100644 --- a/be/src/cloud/cloud_internal_service.h +++ b/be/src/cloud/cloud_internal_service.h @@ -31,6 +31,11 @@ class CloudInternalServiceImpl final : public PInternalService { // TODO(plat1ko): cloud internal service functions + void alter_vault_sync(google::protobuf::RpcController* controller, + const doris::PAlterVaultSyncRequest* request, + PAlterVaultSyncResponse* response, + google::protobuf::Closure* done) override; + private: [[maybe_unused]] CloudStorageEngine& _engine; }; diff --git a/be/src/cloud/cloud_storage_engine.cpp b/be/src/cloud/cloud_storage_engine.cpp index 96e336b3ef3de1..d3f4c05180c140 100644 --- a/be/src/cloud/cloud_storage_engine.cpp +++ b/be/src/cloud/cloud_storage_engine.cpp @@ -267,32 +267,36 @@ Status CloudStorageEngine::start_bg_threads() { return Status::OK(); } +void CloudStorageEngine::sync_storage_vault() { + cloud::StorageVaultInfos vault_infos; + auto st = _meta_mgr->get_storage_vault_info(&vault_infos); + if (!st.ok()) { + LOG(WARNING) << "failed to get storage vault info. err=" << st; + return; + } + + CHECK(!vault_infos.empty()) << "no s3 infos"; + for (auto& [id, vault_info] : vault_infos) { + auto fs = get_filesystem(id); + auto st = (fs == nullptr) + ? std::visit(VaultCreateFSVisitor {id}, vault_info) + : std::visit(RefreshFSVaultVisitor {id, std::move(fs)}, vault_info); + if (!st.ok()) [[unlikely]] { + LOG(WARNING) << vault_process_error(id, vault_info, std::move(st)); + } + } + + if (auto& id = std::get<0>(vault_infos.back()); + latest_fs() == nullptr || latest_fs()->id() != id) { + set_latest_fs(get_filesystem(id)); + } +} + // We should enable_java_support if we want to use hdfs vault void CloudStorageEngine::_refresh_storage_vault_info_thread_callback() { while (!_stop_background_threads_latch.wait_for( std::chrono::seconds(config::refresh_s3_info_interval_s))) { - cloud::StorageVaultInfos vault_infos; - auto st = _meta_mgr->get_storage_vault_info(&vault_infos); - if (!st.ok()) { - LOG(WARNING) << "failed to get storage vault info. err=" << st; - continue; - } - - CHECK(!vault_infos.empty()) << "no s3 infos"; - for (auto& [id, vault_info] : vault_infos) { - auto fs = get_filesystem(id); - auto st = (fs == nullptr) - ? std::visit(VaultCreateFSVisitor {id}, vault_info) - : std::visit(RefreshFSVaultVisitor {id, std::move(fs)}, vault_info); - if (!st.ok()) [[unlikely]] { - LOG(WARNING) << vault_process_error(id, vault_info, std::move(st)); - } - } - - if (auto& id = std::get<0>(vault_infos.back()); - latest_fs() == nullptr || latest_fs()->id() != id) { - set_latest_fs(get_filesystem(id)); - } + sync_storage_vault(); } } diff --git a/be/src/cloud/cloud_storage_engine.h b/be/src/cloud/cloud_storage_engine.h index 297050360dfc3b..e57d0059e974a8 100644 --- a/be/src/cloud/cloud_storage_engine.h +++ b/be/src/cloud/cloud_storage_engine.h @@ -109,6 +109,8 @@ class CloudStorageEngine final : public BaseStorageEngine { std::shared_ptr cumu_compaction_policy( std::string_view compaction_policy); + void sync_storage_vault(); + private: void _refresh_storage_vault_info_thread_callback(); void _vacuum_stale_rowsets_thread_callback(); 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 837a1e96ba59f6..cf72de4ebf7290 100755 --- 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 @@ -692,7 +692,7 @@ public Env(boolean isCheckpointCatalog) { this.brokerMgr = new BrokerMgr(); this.resourceMgr = new ResourceMgr(); - this.storageVaultMgr = new StorageVaultMgr(); + this.storageVaultMgr = new StorageVaultMgr(systemInfo); this.globalTransactionMgr = EnvFactory.getInstance().createGlobalTransactionMgr(this); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVaultMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVaultMgr.java index ddf10a52fffa25..b473afd6840957 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVaultMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVaultMgr.java @@ -24,12 +24,18 @@ import org.apache.doris.cloud.rpc.MetaServiceProxy; import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; +import org.apache.doris.proto.InternalService.PAlterVaultSyncRequest; +import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.RpcException; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TNetworkAddress; import com.google.common.annotations.VisibleForTesting; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -41,7 +47,12 @@ public class StorageVaultMgr { private ReadWriteLock rwLock = new ReentrantReadWriteLock(); - public StorageVaultMgr() { + private static final ExecutorService ALTER_BE_SYNC_THREAD_POOL = Executors.newFixedThreadPool(1); + + private final SystemInfoService systemInfoService; + + public StorageVaultMgr(SystemInfoService systemInfoService) { + this.systemInfoService = systemInfoService; } // TODO(ByteYue): The CreateStorageVault should only be handled by master @@ -125,6 +136,7 @@ public void createHdfsVault(StorageVault vault) throws DdlException { MetaServiceProxy.getInstance().alterObjStoreInfo(requestBuilder.build()); if (response.getStatus().getCode() == Cloud.MetaServiceCode.ALREADY_EXISTED && hdfsStorageVault.ifNotExists()) { + ALTER_BE_SYNC_THREAD_POOL.execute(() -> alterSyncVaultTask()); return; } if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { @@ -136,4 +148,15 @@ public void createHdfsVault(StorageVault vault) throws DdlException { throw new DdlException(e.getMessage()); } } + + private void alterSyncVaultTask() { + systemInfoService.getAllBackends().forEach(backend -> { + TNetworkAddress address = backend.getBrpcAddress(); + try { + BackendServiceProxy.getInstance().alterVaultSync(address, PAlterVaultSyncRequest.newBuilder().build()); + } catch (RpcException e) { + LOG.warn("failed to alter sync vault"); + } + }); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java index 50afc7c96bb109..4027cf6d9510b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java @@ -182,6 +182,11 @@ public Future getWalQueueSize( return stub.getWalQueueSize(request); } + public Future alterVaultSync( + InternalService.PAlterVaultSyncRequest request) { + return stub.alterVaultSync(request); + } + public void shutdown() { ConnectivityState state = channel.getState(false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java index e541b0eb689356..9db06eac4dedf4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java @@ -23,6 +23,8 @@ import org.apache.doris.metric.MetricRepo; import org.apache.doris.planner.PlanFragmentId; import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.InternalService.PAlterVaultSyncRequest; +import org.apache.doris.proto.InternalService.PAlterVaultSyncResponse; import org.apache.doris.proto.InternalService.PExecPlanFragmentStartRequest; import org.apache.doris.proto.InternalService.PGetWalQueueSizeRequest; import org.apache.doris.proto.InternalService.PGetWalQueueSizeResponse; @@ -527,6 +529,18 @@ public Future getWalQueueSize(TNetworkAddress address, } } + public Future alterVaultSync(TNetworkAddress address, + PAlterVaultSyncRequest request) throws RpcException { + try { + final BackendServiceClient client = getProxy(address); + return client.alterVaultSync(request); + } catch (Throwable e) { + LOG.warn("failed to alter vault sync from address={}:{}", address.getHostname(), + address.getPort(), e); + throw new RpcException(address.getHostname(), e.getMessage()); + } + } + public Future fetchRemoteTabletSchemaAsync( TNetworkAddress address, InternalService.PFetchRemoteSchemaRequest request) throws RpcException { try { diff --git a/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/HdfsStorageVaultTest.java b/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/HdfsStorageVaultTest.java index a77e86076d58ff..e3458e57a36317 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/HdfsStorageVaultTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/HdfsStorageVaultTest.java @@ -31,6 +31,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; import org.apache.doris.rpc.RpcException; +import org.apache.doris.system.SystemInfoService; import com.google.common.collect.ImmutableMap; import mockit.Mock; @@ -44,7 +45,7 @@ import java.util.Set; public class HdfsStorageVaultTest { - private StorageVaultMgr mgr = new StorageVaultMgr(); + private StorageVaultMgr mgr = new StorageVaultMgr(new SystemInfoService()); @Before public void setUp() throws Exception { diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 1a6dad5521bac6..59abc9adfb7b58 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -930,6 +930,12 @@ message PFetchRemoteSchemaResponse { optional TabletSchemaPB merged_schema = 2; } +message PAlterVaultSyncRequest { +} + +message PAlterVaultSyncResponse { +} + service PBackendService { rpc transmit_data(PTransmitDataParams) returns (PTransmitDataResult); rpc transmit_data_by_http(PEmptyRequest) returns (PTransmitDataResult); @@ -979,5 +985,6 @@ service PBackendService { rpc fetch_arrow_flight_schema(PFetchArrowFlightSchemaRequest) returns (PFetchArrowFlightSchemaResult); rpc fetch_remote_tablet_schema(PFetchRemoteSchemaRequest) returns (PFetchRemoteSchemaResponse); rpc test_jdbc_connection(PJdbcTestConnectionRequest) returns (PJdbcTestConnectionResult); + rpc alter_vault_sync(PAlterVaultSyncRequest) returns (PAlterVaultSyncResponse); }; From ee6d26b545f6ce7b9a18992816b73f89f8899783 Mon Sep 17 00:00:00 2001 From: Luzhijing <82810928+luzhijing@users.noreply.github.com> Date: Fri, 12 Apr 2024 10:15:08 +0800 Subject: [PATCH 15/71] [Community](readme)Update README.md (#33433) --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 9da62033960018..9a0f062a57d0f1 100644 --- a/README.md +++ b/README.md @@ -35,10 +35,10 @@ Apache Doris is an easy-to-use, high-performance and real-time analytical databa All this makes Apache Doris an ideal tool for scenarios including report analysis, ad-hoc query, unified data warehouse, and data lake query acceleration. On Apache Doris, users can build various applications, such as user behavior analysis, AB test platform, log retrieval analysis, user portrait analysis, and order analysis. -🎉 Version 2.1.0 released now. Check out the 🔗[Release Notes](https://doris.apache.org/docs/releasenotes/release-2.1.0) here. The 2.1 verison delivers exceptional performance with 100% higher out-of-the-box queries proven by TPC-DS 1TB tests, enhanced data lake analytics that are 4-6 times speedier than Trino and Spark, solid support for semi-structured data analysis with new Variant types and suite of analytical functions, asynchronous materialized views for query acceleration, optimized real-time writing at scale, and better workload management with stability and runtime SQL resource tracking. +🎉 Version 2.1.1 released now. Check out the 🔗[Release Notes](https://doris.apache.org/docs/releasenotes/release-2.1.1) here. The 2.1 verison delivers exceptional performance with 100% higher out-of-the-box queries proven by TPC-DS 1TB tests, enhanced data lake analytics that are 4-6 times speedier than Trino and Spark, solid support for semi-structured data analysis with new Variant types and suite of analytical functions, asynchronous materialized views for query acceleration, optimized real-time writing at scale, and better workload management with stability and runtime SQL resource tracking. -🎉 Version 2.0.6 is now released ! This fully evolved and stable release is ready for all users to upgrade. Check out the 🔗[Release Notes](https://doris.apache.org/docs/releasenotes/release-2.0.6) here. +🎉 Version 2.0.8 is now released ! This fully evolved and stable release is ready for all users to upgrade. Check out the 🔗[Release Notes](https://doris.apache.org/docs/releasenotes/release-2.0.8) here. 👀 Have a look at the 🔗[Official Website](https://doris.apache.org/) for a comprehensive list of Apache Doris's core features, blogs and user cases. From bad86a8fdb981fb971d1b419b79eddff30443097 Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Fri, 12 Apr 2024 10:32:41 +0800 Subject: [PATCH 16/71] [cloud](Variant) Optimize FDB kv size for Variant rowsets and update schema on compaction (#33298) Currently, Variant writes an `fdb kv` for each `rowset`'s `TabletSchema` upon `commit_rowset`. This approach is due to the schema's granularity being at the rowset level, with each rowset potentially having a unique schema. However, when the number of rowsets exceeds 1000, `get_rowset` operations may face FDB transaction timeouts due to large kv sizes (exceeding 100K), leading to read times over 10ms. To address this, there's a need to optimize the kv size to prevent timeouts, targeting a size reduction to <2k. The compaction process does not update the schema, potentially merging multiple rowset schemas into a new one without accurately reflecting the changes. This results in inconsistencies and incorrect schema representations in the output rowset. To reduce the space occupied by `TabletSchema`, which is largely consumed by the `column` and `index` fields, a dictionary compression method is proposed. By encoding these fields into dictionary keys, we can significantly decrease the storage space required. This involves modifications to the `RowsetMetaCloudPB` and the introduction of a `SchemaCloudDictionary` to manage the dictionary keys and ensure efficient storage utilization. ``` message RowsetMetaCloudPB { ... repeated int32 column_dict_key_list = 26; ... } message SchemaCloudDictionary { map column_dict = 1; optional int64 current_column_index = 2; map index_dict = 3; optional int64 current_index_index = 2; } ``` --- be/src/olap/rowset/segment_v2/segment.cpp | 4 +- be/src/olap/tablet_schema.cpp | 4 + cloud/src/common/config.h | 3 + cloud/src/meta-service/keys.cpp | 19 +- cloud/src/meta-service/keys.h | 11 +- cloud/src/meta-service/meta_service.cpp | 31 ++- .../src/meta-service/meta_service_schema.cpp | 254 ++++++++++++++++++ cloud/src/meta-service/meta_service_schema.h | 9 + cloud/src/recycler/recycler.cpp | 38 +-- cloud/test/keys_test.cpp | 30 +++ cloud/test/schema_kv_test.cpp | 65 ++++- gensrc/proto/olap_file.proto | 25 ++ .../data/variant_p0/concurrent_insert.out | 103 +++++++ .../variant_github_events_p0/load.groovy | 31 +++ .../suites/variant_p0/complexjson.groovy | 2 +- .../variant_p0/concurrent_insert.groovy | 54 ++++ 16 files changed, 612 insertions(+), 71 deletions(-) create mode 100644 regression-test/data/variant_p0/concurrent_insert.out create mode 100644 regression-test/suites/variant_p0/concurrent_insert.groovy diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp index 9a41171bdd8e21..9d7b573ca98fb6 100644 --- a/be/src/olap/rowset/segment_v2/segment.cpp +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -416,7 +416,9 @@ Status Segment::_create_column_readers(const SegmentFooterPB& footer) { if (!column.has_path_info()) { continue; } - auto iter = column_path_to_footer_ordinal.find(*column.path_info_ptr()); + auto path = column.has_path_info() ? *column.path_info_ptr() + : vectorized::PathInData(column.name_lower_case()); + auto iter = column_path_to_footer_ordinal.find(path); if (iter == column_path_to_footer_ordinal.end()) { continue; } diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp index 2197d78c21d5cc..33e18986b10847 100644 --- a/be/src/olap/tablet_schema.cpp +++ b/be/src/olap/tablet_schema.cpp @@ -560,6 +560,10 @@ void TabletColumn::init_from_pb(const ColumnPB& column) { _column_path->from_protobuf(column.column_path_info()); _parent_col_unique_id = column.column_path_info().parrent_column_unique_id(); } + if (is_variant_type() && !column.has_column_path_info()) { + // set path info for variant root column, to prevent from missing + _column_path = std::make_shared(_col_name_lower_case); + } for (auto& column_pb : column.sparse_columns()) { TabletColumn column; column.init_from_pb(column_pb); diff --git a/cloud/src/common/config.h b/cloud/src/common/config.h index 5640aef2cf95ab..280faf9014c8af 100644 --- a/cloud/src/common/config.h +++ b/cloud/src/common/config.h @@ -125,6 +125,9 @@ CONF_mBool(snapshot_get_tablet_stats, "true"); // Value codec version CONF_mInt16(meta_schema_value_version, "1"); +// Limit kv size of Schema SchemaDictKeyList, default 10MB +CONF_mInt32(schema_dict_kv_size_limit, "10485760"); + // For instance check interval CONF_Int64(reserved_buffer_days, "3"); diff --git a/cloud/src/meta-service/keys.cpp b/cloud/src/meta-service/keys.cpp index 6ad60d8aeac0c8..01b4b9906524e7 100644 --- a/cloud/src/meta-service/keys.cpp +++ b/cloud/src/meta-service/keys.cpp @@ -47,10 +47,10 @@ namespace doris::cloud { [[maybe_unused]] static const char* META_KEY_INFIX_TABLET = "tablet"; [[maybe_unused]] static const char* META_KEY_INFIX_TABLET_IDX = "tablet_index"; [[maybe_unused]] static const char* META_KEY_INFIX_SCHEMA = "schema"; -[[maybe_unused]] static const char* META_KEY_INFIX_ROWSET_SCHEMA = "rowset_schema"; [[maybe_unused]] static const char* META_KEY_INFIX_DELETE_BITMAP = "delete_bitmap"; [[maybe_unused]] static const char* META_KEY_INFIX_DELETE_BITMAP_LOCK = "delete_bitmap_lock"; [[maybe_unused]] static const char* META_KEY_INFIX_DELETE_BITMAP_PENDING = "delete_bitmap_pending"; +[[maybe_unused]] static const char* META_KEY_SCHEMA_PB_DICTIONARY = "tablet_schema_pb_dict"; [[maybe_unused]] static const char* RECYCLE_KEY_INFIX_INDEX = "index"; [[maybe_unused]] static const char* RECYCLE_KEY_INFIX_PART = "partition"; @@ -115,7 +115,7 @@ static void encode_prefix(const T& t, std::string* key) { RecycleIndexKeyInfo, RecyclePartKeyInfo, RecycleRowsetKeyInfo, RecycleTxnKeyInfo, RecycleStageKeyInfo, StatsTabletKeyInfo, TableVersionKeyInfo, JobTabletKeyInfo, JobRecycleKeyInfo, RLJobProgressKeyInfo, - CopyJobKeyInfo, CopyFileKeyInfo, MetaRowsetSchemaKeyInfo, StorageVaultKeyInfo>); + CopyJobKeyInfo, CopyFileKeyInfo, StorageVaultKeyInfo, MetaSchemaPBDictionaryInfo>); key->push_back(CLOUD_USER_KEY_SPACE01); // Prefixes for key families @@ -131,7 +131,7 @@ static void encode_prefix(const T& t, std::string* key) { || std::is_same_v || std::is_same_v || std::is_same_v - || std::is_same_v + || std::is_same_v || std::is_same_v || std::is_same_v || std::is_same_v) { @@ -282,13 +282,6 @@ void meta_schema_key(const MetaSchemaKeyInfo& in, std::string* out) { encode_int64(std::get<2>(in), out); // schema_version } -void meta_rowset_schema_key(const MetaRowsetSchemaKeyInfo& in, std::string* out) { - encode_prefix(in, out); // 0x01 "meta" ${instance_id} - encode_bytes(META_KEY_INFIX_ROWSET_SCHEMA, out); // "rowset_schema" - encode_int64(std::get<1>(in), out); // tablet_id - encode_bytes(std::get<2>(in), out); // rowset_id -} - void meta_delete_bitmap_key(const MetaDeleteBitmapInfo& in, std::string* out) { encode_prefix(in, out); // 0x01 "meta" ${instance_id} encode_bytes(META_KEY_INFIX_DELETE_BITMAP, out); // "delete_bitmap" @@ -312,6 +305,12 @@ void meta_pending_delete_bitmap_key(const MetaPendingDeleteBitmapInfo& in, std:: encode_int64(std::get<1>(in), out); // table_id } +void meta_schema_pb_dictionary_key(const MetaSchemaPBDictionaryInfo& in, std::string* out) { + encode_prefix(in, out); // 0x01 "meta" ${instance_id} + encode_bytes(META_KEY_SCHEMA_PB_DICTIONARY, out); // "tablet_schema_pb_dict" + encode_int64(std::get<1>(in), out); // index_id +} + //============================================================================== // Recycle keys //============================================================================== diff --git a/cloud/src/meta-service/keys.h b/cloud/src/meta-service/keys.h index 483332a133c0b1..21cf2208de1c6d 100644 --- a/cloud/src/meta-service/keys.h +++ b/cloud/src/meta-service/keys.h @@ -44,6 +44,7 @@ // 0x01 "meta" ${instance_id} "delete_bitmap_lock" ${table_id} ${partition_id} -> DeleteBitmapUpdateLockPB // 0x01 "meta" ${instance_id} "delete_bitmap_pending" ${table_id} -> PendingDeleteBitmapPB // 0x01 "meta" ${instance_id} "delete_bitmap" ${tablet_id} ${rowset_id} ${version} ${segment_id} -> roaringbitmap +// 0x01 "meta" ${instance_id} "tablet_schema_pb_dict" ${index_id} -> SchemaCloudDictionary // // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} -> TabletStatsPB // 0x01 "stats" ${instance_id} "tablet" ${table_id} ${index_id} ${partition_id} ${tablet_id} "data_size" -> int64 @@ -163,9 +164,6 @@ using JobRecycleKeyInfo = BasicKeyInfo<20 , std::tuple>; // 0:instance_id 1:index_id 2:schema_version using MetaSchemaKeyInfo = BasicKeyInfo<21, std::tuple>; -// 0:instance_id 1:tablet_id 2:rowset_id -using MetaRowsetSchemaKeyInfo = BasicKeyInfo<21, std::tuple>; - // 0:instance_id 1:tablet_id 2:rowest_id 3:version 4:seg_id using MetaDeleteBitmapInfo = BasicKeyInfo<22 , std::tuple>; @@ -184,6 +182,9 @@ using StorageVaultKeyInfo = BasicKeyInfo<26, std::tuple>; +// 0:instance_id 1:index_id +using MetaSchemaPBDictionaryInfo = BasicKeyInfo<28 , std::tuple>; + void instance_key(const InstanceKeyInfo& in, std::string* out); static inline std::string instance_key(const InstanceKeyInfo& in) { std::string s; instance_key(in, &s); return s; } @@ -213,19 +214,19 @@ void meta_rowset_tmp_key(const MetaRowsetTmpKeyInfo& in, std::string* out); void meta_tablet_idx_key(const MetaTabletIdxKeyInfo& in, std::string* out); void meta_tablet_key(const MetaTabletKeyInfo& in, std::string* out); void meta_schema_key(const MetaSchemaKeyInfo& in, std::string* out); -void meta_rowset_schema_key(const MetaRowsetSchemaKeyInfo& in, std::string* out); void meta_delete_bitmap_key(const MetaDeleteBitmapInfo& in, std::string* out); void meta_delete_bitmap_update_lock_key(const MetaDeleteBitmapUpdateLockInfo& in, std::string* out); void meta_pending_delete_bitmap_key(const MetaPendingDeleteBitmapInfo& in, std::string* out); +void meta_schema_pb_dictionary_key(const MetaSchemaPBDictionaryInfo& in, std::string* out); static inline std::string meta_rowset_key(const MetaRowsetKeyInfo& in) { std::string s; meta_rowset_key(in, &s); return s; } static inline std::string meta_rowset_tmp_key(const MetaRowsetTmpKeyInfo& in) { std::string s; meta_rowset_tmp_key(in, &s); return s; } static inline std::string meta_tablet_idx_key(const MetaTabletIdxKeyInfo& in) { std::string s; meta_tablet_idx_key(in, &s); return s; } static inline std::string meta_tablet_key(const MetaTabletKeyInfo& in) { std::string s; meta_tablet_key(in, &s); return s; } static inline std::string meta_schema_key(const MetaSchemaKeyInfo& in) { std::string s; meta_schema_key(in, &s); return s; } -static inline std::string meta_rowset_schema_key(const MetaRowsetSchemaKeyInfo& in) { std::string s; meta_rowset_schema_key(in, &s); return s; } static inline std::string meta_delete_bitmap_key(const MetaDeleteBitmapInfo& in) { std::string s; meta_delete_bitmap_key(in, &s); return s; } static inline std::string meta_delete_bitmap_update_lock_key(const MetaDeleteBitmapUpdateLockInfo& in) { std::string s; meta_delete_bitmap_update_lock_key(in, &s); return s; } static inline std::string meta_pending_delete_bitmap_key(const MetaPendingDeleteBitmapInfo& in) { std::string s; meta_pending_delete_bitmap_key(in, &s); return s; } +static inline std::string meta_schema_pb_dictionary_key(const MetaSchemaPBDictionaryInfo& in) { std::string s; meta_schema_pb_dictionary_key(in, &s); return s; } std::string recycle_key_prefix(std::string_view instance_id); void recycle_index_key(const RecycleIndexKeyInfo& in, std::string* out); diff --git a/cloud/src/meta-service/meta_service.cpp b/cloud/src/meta-service/meta_service.cpp index e27b6b5b944bfa..8bdcc0581b0afa 100644 --- a/cloud/src/meta-service/meta_service.cpp +++ b/cloud/src/meta-service/meta_service.cpp @@ -40,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -1087,14 +1088,11 @@ void MetaServiceImpl::commit_rowset(::google::protobuf::RpcController* controlle DCHECK(rowset_meta.tablet_schema().has_schema_version()); DCHECK_GE(rowset_meta.tablet_schema().schema_version(), 0); rowset_meta.set_schema_version(rowset_meta.tablet_schema().schema_version()); - std::string schema_key; + std::string schema_key = meta_schema_key( + {instance_id, rowset_meta.index_id(), rowset_meta.schema_version()}); if (rowset_meta.has_variant_type_in_schema()) { - // encodes schema in a seperate kv, since variant schema is volatile - schema_key = meta_rowset_schema_key( - {instance_id, rowset_meta.tablet_id(), rowset_meta.rowset_id_v2()}); - } else { - schema_key = meta_schema_key( - {instance_id, rowset_meta.index_id(), rowset_meta.schema_version()}); + write_schema_dict(code, msg, instance_id, txn.get(), &rowset_meta); + if (code != MetaServiceCode::OK) return; } put_schema_kv(code, msg, txn.get(), schema_key, rowset_meta.tablet_schema()); if (code != MetaServiceCode::OK) return; @@ -1439,8 +1437,12 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, } rowset_meta.set_index_id(idx.index_id()); } + bool need_read_schema_dict = false; auto arena = response->GetArena(); for (auto& rowset_meta : *response->mutable_rowset_meta()) { + if (rowset_meta.has_schema_dict_key_list()) { + need_read_schema_dict = true; + } if (rowset_meta.has_tablet_schema()) continue; if (!rowset_meta.has_schema_version()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -1450,15 +1452,6 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, rowset_meta.start_version(), rowset_meta.end_version()); return; } - if (rowset_meta.has_variant_type_in_schema()) { - // get rowset schema kv - auto key = meta_rowset_schema_key( - {instance_id, idx.tablet_id(), rowset_meta.rowset_id_v2()}); - if (!try_fetch_and_parse_schema(txn.get(), rowset_meta, key, code, msg)) { - return; - } - continue; - } if (auto it = version_to_schema.find(rowset_meta.schema_version()); it != version_to_schema.end()) { if (arena != nullptr) { @@ -1475,6 +1468,12 @@ void MetaServiceImpl::get_rowset(::google::protobuf::RpcController* controller, rowset_meta.mutable_tablet_schema()); } } + + if (need_read_schema_dict) { + read_schema_from_dict(code, msg, instance_id, idx.index_id(), txn.get(), + response->mutable_rowset_meta()); + if (code != MetaServiceCode::OK) return; + } } void MetaServiceImpl::get_tablet_stats(::google::protobuf::RpcController* controller, diff --git a/cloud/src/meta-service/meta_service_schema.cpp b/cloud/src/meta-service/meta_service_schema.cpp index 6f361fb9f76f42..ee69b1547cf0c6 100644 --- a/cloud/src/meta-service/meta_service_schema.cpp +++ b/cloud/src/meta-service/meta_service_schema.cpp @@ -18,7 +18,17 @@ #include "meta-service/meta_service_schema.h" #include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "common/config.h" #include "common/logging.h" #include "common/sync_point.h" #include "common/util.h" @@ -32,6 +42,8 @@ namespace config { extern int16_t meta_schema_value_version; } +constexpr static const char* VARIANT_TYPE_NAME = "VARIANT"; + void put_schema_kv(MetaServiceCode& code, std::string& msg, Transaction* txn, std::string_view schema_key, const doris::TabletSchemaCloudPB& schema) { TxnErrorCode err = cloud::key_exists(txn, schema_key); @@ -118,5 +130,247 @@ bool parse_schema_value(const ValueBuf& buf, doris::TabletSchemaCloudPB* schema) // TODO(plat1ko): Apply decompression based on value version return buf.to_pb(schema); } +/** + * Processes dictionary items, mapping them to a dictionary key and adding the key to rowset meta. + * If it's a new item, generates a new key and increments the item ID. This function is also responsible + * for removing dynamic parts from the original RowsetMeta's TabletSchema to ensure the stability of + * FDB schema key-value pairs. + * + * @param dict The schema cloud dictionary reference, used for storing and managing schema dictionary data. + * @param item_dict A mapping from item unique identifiers to their protobuf representations, used to find + * and process specific item data. + * @param result Pointer to the collection of result items. Stores filtered or transformed items. Can be nullptr + * if collecting results is not required. + * @param items The collection of items to be processed. These items are filtered and potentially added to the dictionary. + * @param filter A function to determine which items should be processed. If it returns true, the item is processed. + * @param add_dict_key_fn A function to handle the logic when a new item is added to the dictionary, such as updating metadata. + */ +template +void process_dictionary(SchemaCloudDictionary& dict, + const google::protobuf::Map& item_dict, + google::protobuf::RepeatedPtrField* result, + const google::protobuf::RepeatedPtrField& items, + const std::function& filter, + const std::function& add_dict_key_fn) { + if (items.empty()) { + return; + } + // Use deterministic method to do serialization since structure like + // `google::protobuf::Map`'s serialization is unstable + auto serialize_fn = [](const ItemPB& item) -> std::string { + std::string output; + google::protobuf::io::StringOutputStream string_output_stream(&output); + google::protobuf::io::CodedOutputStream output_stream(&string_output_stream); + output_stream.SetSerializationDeterministic(true); + item.SerializeToCodedStream(&output_stream); + return output; + }; + + google::protobuf::RepeatedPtrField none_ext_items; + std::unordered_map reversed_dict; + for (const auto& [key, val] : item_dict) { + reversed_dict[serialize_fn(val)] = key; + } + + for (const auto& item : items) { + if (filter(item)) { + // Filter none extended items, mainly extended columns and extended indexes + *none_ext_items.Add() = item; + continue; + } + const std::string serialized_key = serialize_fn(item); + auto it = reversed_dict.find(serialized_key); + if (it != reversed_dict.end()) { + // Add existed dict key to related dict + add_dict_key_fn(it->second); + } else { + // Add new dictionary key-value pair and update current_xxx_dict_id. + int64_t current_dict_id = 0; + if constexpr (std::is_same_v) { + current_dict_id = dict.current_column_dict_id() + 1; + dict.set_current_column_dict_id(current_dict_id); + dict.mutable_column_dict()->emplace(current_dict_id, item); + } + if constexpr (std::is_same_v) { + current_dict_id = dict.current_index_dict_id() + 1; + dict.set_current_index_dict_id(current_dict_id); + dict.mutable_index_dict()->emplace(current_dict_id, item); + } + add_dict_key_fn(current_dict_id); + reversed_dict[serialized_key] = current_dict_id; + // LOG(INFO) << "Add dict key = " << current_dict_id << " dict value = " << item.ShortDebugString(); + } + } + // clear extended items to prevent writing them to fdb + if (result != nullptr) { + result->Swap(&none_ext_items); + } +} + +// Writes schema dictionary metadata to RowsetMetaCloudPB. +// Schema was extended in BE side, we need to reset schema to original frontend schema and store +// such restored schema in fdb. And also add extra dict key info to RowsetMetaCloudPB. +void write_schema_dict(MetaServiceCode& code, std::string& msg, const std::string& instance_id, + Transaction* txn, RowsetMetaCloudPB* rowset_meta) { + std::stringstream ss; + // wrtie dict to rowset meta and update dict + SchemaCloudDictionary dict; + std::string dict_key = meta_schema_pb_dictionary_key({instance_id, rowset_meta->index_id()}); + ValueBuf dict_val; + auto err = cloud::get(txn, dict_key, &dict_val); + LOG(INFO) << "Retrieved column pb dictionary, index_id=" << rowset_meta->index_id() + << " key=" << hex(dict_key) << " error=" << err; + if (err != TxnErrorCode::TXN_KEY_NOT_FOUND && err != TxnErrorCode::TXN_OK) { + // Handle retrieval error. + ss << "Failed to retrieve column pb dictionary, instance_id=" << instance_id + << " table_id=" << rowset_meta->index_id() << " key=" << hex(dict_key) + << " error=" << err; + msg = ss.str(); + code = cast_as(err); + return; + } + if (err == TxnErrorCode::TXN_OK && !dict_val.to_pb(&dict)) { + // Handle parse error. + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = fmt::format("Malformed tablet dictionary value, key={}", hex(dict_key)); + return; + } + + // collect sparse columns and clear in parent column + google::protobuf::RepeatedPtrField sparse_columns; + for (auto& column_pb : *rowset_meta->mutable_tablet_schema()->mutable_column()) { + if (column_pb.type() == VARIANT_TYPE_NAME && !column_pb.sparse_columns().empty()) { + // set parent_id for restore info + for (auto& sparse_col : *column_pb.mutable_sparse_columns()) { + sparse_col.set_parent_unique_id(column_pb.unique_id()); + } + sparse_columns.Add(column_pb.sparse_columns().begin(), + column_pb.sparse_columns().end()); + } + // clear sparse columns to prevent writing them to fdb + column_pb.clear_sparse_columns(); + } + auto* dict_list = rowset_meta->mutable_schema_dict_key_list(); + // handle column dict + auto original_column_dict_id = dict.current_column_dict_id(); + auto column_filter = [&](const doris::ColumnPB& col) -> bool { return col.unique_id() >= 0; }; + auto column_dict_adder = [&](int32_t key) { dict_list->add_column_dict_key_list(key); }; + process_dictionary( + dict, dict.column_dict(), rowset_meta->mutable_tablet_schema()->mutable_column(), + rowset_meta->tablet_schema().column(), column_filter, column_dict_adder); + + // handle sparse column dict + auto sparse_column_dict_adder = [&](int32_t key) { + dict_list->add_sparse_column_dict_key_list(key); + }; + // not filter any + auto sparse_column_filter = [&](const doris::ColumnPB& col) -> bool { return false; }; + process_dictionary(dict, dict.column_dict(), nullptr, sparse_columns, + sparse_column_filter, sparse_column_dict_adder); + + // handle index info dict + auto original_index_dict_id = dict.current_index_dict_id(); + auto index_filter = [&](const doris::TabletIndexPB& index_pb) -> bool { + return index_pb.index_suffix_name().empty(); + }; + auto index_dict_adder = [&](int32_t key) { dict_list->add_index_info_dict_key_list(key); }; + process_dictionary( + dict, dict.index_dict(), rowset_meta->mutable_tablet_schema()->mutable_index(), + rowset_meta->tablet_schema().index(), index_filter, index_dict_adder); + + // Write back modified dictionaries. + if (original_index_dict_id != dict.current_index_dict_id() || + original_column_dict_id != dict.current_column_dict_id()) { + // If dictionary was modified, serialize and save it. + std::string dict_val; + if (!dict.SerializeToString(&dict_val)) { + // Handle serialization error. + code = MetaServiceCode::PROTOBUF_SERIALIZE_ERR; + ss << "Failed to serialize dictionary for saving, txn_id=" << rowset_meta->txn_id(); + msg = ss.str(); + return; + } + // Limit the size of dict value + if (dict_val.size() > config::schema_dict_kv_size_limit) { + code = MetaServiceCode::KV_TXN_COMMIT_ERR; + ss << "Failed to write dictionary for saving, txn_id=" << rowset_meta->txn_id() + << ", reached the limited size threshold of SchemaDictKeyList " + << config::schema_dict_kv_size_limit; + msg = ss.str(); + } + // splitting large values (>90*1000) into multiple KVs + cloud::put(txn, dict_key, dict_val, 0); + LOG(INFO) << "Dictionary saved, key=" << hex(dict_key) + << " txn_id=" << rowset_meta->txn_id() << " Dict size=" << dict.column_dict_size() + << ", Current column ID=" << dict.current_column_dict_id() + << ", Current index ID=" << dict.current_index_dict_id(); + } +} + +void read_schema_from_dict(MetaServiceCode& code, std::string& msg, const std::string& instance_id, + int64_t index_id, Transaction* txn, + google::protobuf::RepeatedPtrField* rowset_metas) { + std::stringstream ss; + + // read dict if any rowset has dict key list + SchemaCloudDictionary dict; + std::string column_dict_key = meta_schema_pb_dictionary_key({instance_id, index_id}); + ValueBuf dict_val; + auto err = cloud::get(txn, column_dict_key, &dict_val); + if (err != TxnErrorCode::TXN_OK && err != TxnErrorCode::TXN_KEY_NOT_FOUND) { + code = cast_as(err); + ss << "internal error, failed to get dict ret=" << err; + msg = ss.str(); + return; + } + if (err == TxnErrorCode::TXN_OK && !dict_val.to_pb(&dict)) [[unlikely]] { + code = MetaServiceCode::PROTOBUF_PARSE_ERR; + msg = "failed to parse SchemaCloudDictionary"; + return; + } + LOG(INFO) << "Get schema_dict, column size=" << dict.column_dict_size() + << ", index size=" << dict.index_dict_size(); + + auto fill_schema_with_dict = [&](RowsetMetaCloudPB* out) { + std::unordered_map unique_id_map; + //init map + for (ColumnPB& column : *out->mutable_tablet_schema()->mutable_column()) { + unique_id_map[column.unique_id()] = &column; + } + // column info + for (size_t i = 0; i < out->schema_dict_key_list().column_dict_key_list_size(); ++i) { + int dict_key = out->schema_dict_key_list().column_dict_key_list(i); + const ColumnPB& dict_val = dict.column_dict().at(dict_key); + ColumnPB& to_add = *out->mutable_tablet_schema()->add_column(); + to_add = dict_val; + VLOG_DEBUG << "fill dict column " << dict_val.ShortDebugString(); + } + + // index info + for (size_t i = 0; i < out->schema_dict_key_list().index_info_dict_key_list_size(); ++i) { + int dict_key = out->schema_dict_key_list().index_info_dict_key_list(i); + const doris::TabletIndexPB& dict_val = dict.index_dict().at(dict_key); + doris::TabletIndexPB& to_add = *out->mutable_tablet_schema()->add_index(); + to_add = dict_val; + VLOG_DEBUG << "fill dict index " << dict_val.ShortDebugString(); + } + + // sparse column info + for (size_t i = 0; i < out->schema_dict_key_list().sparse_column_dict_key_list_size(); + ++i) { + int dict_key = out->schema_dict_key_list().sparse_column_dict_key_list(i); + const ColumnPB& dict_val = dict.column_dict().at(dict_key); + *unique_id_map.at(dict_val.parent_unique_id())->add_sparse_columns() = dict_val; + VLOG_DEBUG << "fill dict sparse column" << dict_val.ShortDebugString(); + } + }; + + // fill rowsets's schema with dict info + for (auto& rowset_meta : *rowset_metas) { + if (rowset_meta.has_schema_dict_key_list()) { + fill_schema_with_dict(&rowset_meta); + } + } +} } // namespace doris::cloud diff --git a/cloud/src/meta-service/meta_service_schema.h b/cloud/src/meta-service/meta_service_schema.h index 44fabeafd73b28..d44f01f9747128 100644 --- a/cloud/src/meta-service/meta_service_schema.h +++ b/cloud/src/meta-service/meta_service_schema.h @@ -30,4 +30,13 @@ void put_schema_kv(MetaServiceCode& code, std::string& msg, Transaction* txn, // Return true if parse success [[nodiscard]] bool parse_schema_value(const ValueBuf& buf, doris::TabletSchemaCloudPB* schema); +// Writes schema dictionary metadata to RowsetMetaCloudPB +void write_schema_dict(MetaServiceCode& code, std::string& msg, const std::string& instance_id, + Transaction* txn, RowsetMetaCloudPB* rowset_meta); + +// Read schema from dictionary metadata, modified to rowset_metas +void read_schema_from_dict(MetaServiceCode& code, std::string& msg, const std::string& instance_id, + int64_t index_id, Transaction* txn, + google::protobuf::RepeatedPtrField* rowset_metas); + } // namespace doris::cloud diff --git a/cloud/src/recycler/recycler.cpp b/cloud/src/recycler/recycler.cpp index ed39dd8df66221..3089aeb66d6bd8 100644 --- a/cloud/src/recycler/recycler.cpp +++ b/cloud/src/recycler/recycler.cpp @@ -1153,6 +1153,7 @@ int InstanceRecycler::recycle_tablets(int64_t table_id, int64_t index_id, int64_ txn->remove(job_key_begin, job_key_end); LOG(WARNING) << "remove job kv, begin=" << hex(job_key_begin) << " end=" << hex(job_key_end); std::string schema_key_begin, schema_key_end; + std::string schema_dict_key; if (partition_id <= 0) { // Delete schema kv of this index meta_schema_key({instance_id_, index_id, 0}, &schema_key_begin); @@ -1160,6 +1161,9 @@ int InstanceRecycler::recycle_tablets(int64_t table_id, int64_t index_id, int64_ txn->remove(schema_key_begin, schema_key_end); LOG(WARNING) << "remove schema kv, begin=" << hex(schema_key_begin) << " end=" << hex(schema_key_end); + meta_schema_pb_dictionary_key({instance_id_, index_id}, &schema_dict_key); + txn->remove(schema_dict_key); + LOG(WARNING) << "remove schema dict kv, key=" << hex(schema_dict_key); } TxnErrorCode err = txn->commit(); @@ -1336,11 +1340,6 @@ int InstanceRecycler::recycle_tablet(int64_t tablet_id) { std::string delete_bitmap_end = meta_delete_bitmap_key({instance_id_, tablet_id + 1, "", 0, 0}); txn->remove(delete_bitmap_start, delete_bitmap_end); - // remove rowset schema - std::string rowset_schema_start = meta_rowset_schema_key({instance_id_, tablet_id, ""}); - std::string rowset_schema_end = meta_rowset_schema_key({instance_id_, tablet_id + 1, ""}); - txn->remove(rowset_schema_start, rowset_schema_end); - TxnErrorCode err = txn->commit(); if (err != TxnErrorCode::TXN_OK) { LOG(WARNING) << "failed to delete rowset kv of tablet " << tablet_id << ", err=" << err; @@ -1401,7 +1400,6 @@ int InstanceRecycler::recycle_rowsets() { }); std::vector rowset_keys; - std::vector rowset_schema_keys; std::vector rowsets; // Store keys of rowset recycled by background workers @@ -1529,9 +1527,6 @@ int InstanceRecycler::recycle_rowsets() { return -1; } } else { - auto schema_key = meta_rowset_schema_key( - {instance_id_, rowset_meta->tablet_id(), rowset_meta->rowset_id_v2()}); - rowset_schema_keys.push_back(std::move(schema_key)); rowset_keys.push_back(std::string(k)); if (rowset_meta->num_segments() > 0) { // Skip empty rowset rowsets.push_back(std::move(*rowset_meta)); @@ -1542,23 +1537,15 @@ int InstanceRecycler::recycle_rowsets() { auto loop_done = [&]() -> int { std::vector rowset_keys_to_delete; - std::vector rowset_schema_keys_to_delete; std::vector rowsets_to_delete; rowset_keys_to_delete.swap(rowset_keys); - rowset_schema_keys_to_delete.swap(rowset_schema_keys); rowsets_to_delete.swap(rowsets); worker_pool->submit([&, rowset_keys_to_delete = std::move(rowset_keys_to_delete), - rowsets_to_delete = std::move(rowsets_to_delete), - rowset_schema_keys_to_delete = - std::move(rowset_schema_keys_to_delete)]() { + rowsets_to_delete = std::move(rowsets_to_delete)]() { if (delete_rowset_data(rowsets_to_delete) != 0) { LOG(WARNING) << "failed to delete rowset data, instance_id=" << instance_id_; return; } - if (txn_remove(txn_kv_.get(), rowset_schema_keys_to_delete) != 0) { - LOG(WARNING) << "failed to delete recycle rowset kv, instance_id=" << instance_id_; - return; - } if (txn_remove(txn_kv_.get(), rowset_keys_to_delete) != 0) { LOG(WARNING) << "failed to delete recycle rowset kv, instance_id=" << instance_id_; return; @@ -1618,11 +1605,10 @@ int InstanceRecycler::recycle_tmp_rowsets() { // Elements in `tmp_rowset_keys` has the same lifetime as `it` std::vector tmp_rowset_keys; - std::vector tmp_rowset_schema_keys; std::vector tmp_rowsets; auto handle_rowset_kv = [&num_scanned, &num_expired, &tmp_rowset_keys, &tmp_rowsets, - &expired_rowset_size, &total_rowset_size, &tmp_rowset_schema_keys, + &expired_rowset_size, &total_rowset_size, this](std::string_view k, std::string_view v) -> int { ++num_scanned; total_rowset_size += v.size(); @@ -1652,11 +1638,6 @@ int InstanceRecycler::recycle_tmp_rowsets() { tmp_rowset_keys.push_back(k); return 0; } - if (rowset.has_variant_type_in_schema()) { - auto schema_key = meta_rowset_schema_key( - {instance_id_, rowset.tablet_id(), rowset.rowset_id_v2()}); - tmp_rowset_schema_keys.push_back(std::move(schema_key)); - } // TODO(plat1ko): check rowset not referenced LOG(INFO) << "delete rowset data, instance_id=" << instance_id_ << " tablet_id=" << rowset.tablet_id() << " rowset_id=" << rowset.rowset_id_v2() @@ -1670,8 +1651,7 @@ int InstanceRecycler::recycle_tmp_rowsets() { return 0; }; - auto loop_done = [&tmp_rowset_keys, &tmp_rowsets, &num_recycled, &tmp_rowset_schema_keys, - this]() -> int { + auto loop_done = [&tmp_rowset_keys, &tmp_rowsets, &num_recycled, this]() -> int { std::unique_ptr> defer((int*)0x01, [&](int*) { tmp_rowset_keys.clear(); tmp_rowsets.clear(); @@ -1680,10 +1660,6 @@ int InstanceRecycler::recycle_tmp_rowsets() { LOG(WARNING) << "failed to delete tmp rowset data, instance_id=" << instance_id_; return -1; } - if (txn_remove(txn_kv_.get(), tmp_rowset_schema_keys) != 0) { - LOG(WARNING) << "failed to delete tmp rowset schema kv, instance_id=" << instance_id_; - return -1; - } if (txn_remove(txn_kv_.get(), tmp_rowset_keys) != 0) { LOG(WARNING) << "failed to delete tmp rowset kv, instance_id=" << instance_id_; return -1; diff --git a/cloud/test/keys_test.cpp b/cloud/test/keys_test.cpp index ce7cbb0551e1bd..a92a685cda45f8 100644 --- a/cloud/test/keys_test.cpp +++ b/cloud/test/keys_test.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include #include @@ -1024,3 +1025,32 @@ TEST(KeysTest, DecodeKeysTest) { ASSERT_TRUE(!pretty_key.empty()) << key; std::cout << "\n" << pretty_key << std::endl; } + +TEST(KeysTest, MetaSchemaPBDictionaryTest) { + using namespace doris::cloud; + std::string instance_id = "instance_id_meta_dict"; + int64_t index_id = 123456; + + // 0:instance_id 1:index_id + MetaSchemaPBDictionaryInfo dict_key {instance_id, index_id}; + std::string encoded_dict_key; + meta_schema_pb_dictionary_key(dict_key, &encoded_dict_key); + std::cout << hex(encoded_dict_key) << std::endl; + + std::string decoded_instance_id; + std::string decoded_prefix; + std::string decoded_meta_prefix; + int64_t decoded_index_id; + std::string_view key_sv(encoded_dict_key); + remove_user_space_prefix(&key_sv); + ASSERT_EQ(decode_bytes(&key_sv, &decoded_prefix), 0); + ASSERT_EQ(decode_bytes(&key_sv, &decoded_instance_id), 0); + ASSERT_EQ(decode_bytes(&key_sv, &decoded_meta_prefix), 0); + ASSERT_EQ(decode_int64(&key_sv, &decoded_index_id), 0); + ASSERT_TRUE(key_sv.empty()); + + EXPECT_EQ("meta", decoded_prefix); + EXPECT_EQ("tablet_schema_pb_dict", decoded_meta_prefix); + EXPECT_EQ(instance_id, decoded_instance_id); + EXPECT_EQ(index_id, decoded_index_id); +} diff --git a/cloud/test/schema_kv_test.cpp b/cloud/test/schema_kv_test.cpp index 8fc03b951ce1af..4d52533a710d57 100644 --- a/cloud/test/schema_kv_test.cpp +++ b/cloud/test/schema_kv_test.cpp @@ -280,7 +280,8 @@ static void commit_txn(MetaServiceProxy* meta_service, int64_t db_id, int64_t tx static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, const std::string& rowset_id, int32_t schema_version, - int64_t version = -1) { + int64_t version = -1, + const TabletSchemaCloudPB* schema = nullptr) { doris::RowsetMetaCloudPB rowset; rowset.set_rowset_id(0); // required rowset.set_rowset_id_v2(rowset_id); @@ -295,6 +296,10 @@ static doris::RowsetMetaCloudPB create_rowset(int64_t txn_id, int64_t tablet_id, } rowset.mutable_tablet_schema()->set_schema_version(schema_version); rowset.set_txn_expiration(::time(nullptr)); // Required by DCHECK + if (schema != nullptr) { + rowset.mutable_tablet_schema()->CopyFrom(*schema); + rowset.mutable_tablet_schema()->set_schema_version(schema_version); + } return rowset; } @@ -319,19 +324,54 @@ static void commit_rowset(MetaServiceProxy* meta_service, const doris::RowsetMet } static void insert_rowset(MetaServiceProxy* meta_service, int64_t db_id, const std::string& label, - int64_t table_id, int64_t tablet_id, int32_t schema_version) { + int64_t table_id, int64_t tablet_id, int32_t schema_version, + const TabletSchemaCloudPB* schema = nullptr) { int64_t txn_id = 0; ASSERT_NO_FATAL_FAILURE(begin_txn(meta_service, db_id, label, table_id, txn_id)); CreateRowsetResponse res; - auto rowset = create_rowset(txn_id, tablet_id, next_rowset_id(), schema_version); + auto rowset = create_rowset(txn_id, tablet_id, next_rowset_id(), schema_version, -1, schema); + rowset.set_has_variant_type_in_schema(schema != nullptr); prepare_rowset(meta_service, rowset, res); ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; res.Clear(); ASSERT_NO_FATAL_FAILURE(commit_rowset(meta_service, rowset, res)); - ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label; + ASSERT_EQ(res.status().code(), MetaServiceCode::OK) << label << ", msg=" << res.status().msg(); commit_txn(meta_service, db_id, txn_id, label); } +static TabletSchemaCloudPB getVariantSchema() { + TabletSchemaCloudPB schema; + schema.set_schema_version(3); + // columns + ColumnPB var; + var.set_type("VARIANT"); + var.set_unique_id(10); + ColumnPB var_sub1; + var_sub1.set_type("INT"); + var_sub1.set_unique_id(-1); + schema.add_column()->CopyFrom(var_sub1); + ColumnPB var_sub2; + var_sub2.set_type("DOUBLE"); + var_sub2.set_unique_id(-1); + schema.add_column()->CopyFrom(var_sub2); + ColumnPB var_sparse_sub1; + var_sparse_sub1.set_type("DOUBLE"); + var_sparse_sub1.set_unique_id(-1); + var.add_sparse_columns()->CopyFrom(var_sparse_sub1); + schema.add_column()->CopyFrom(var); + + // indexes + doris::TabletIndexPB index1; + index1.set_index_id(111); + index1.set_index_suffix_name("aaabbbccc"); + schema.add_index()->CopyFrom(index1); + + doris::TabletIndexPB index2; + index2.set_index_id(222); + schema.add_index()->CopyFrom(index2); + return schema; +} + TEST(DetachSchemaKVTest, RowsetTest) { auto meta_service = get_meta_service(); // meta_service->resource_mgr().reset(); // Do not use resource manager @@ -439,7 +479,8 @@ TEST(DetachSchemaKVTest, RowsetTest) { auto insert_and_get_rowset = [&meta_service](int64_t table_id, int64_t index_id, int64_t partition_id, int64_t tablet_id, int label_base, - google::protobuf::Arena* arena = nullptr) { + google::protobuf::Arena* arena = nullptr, + const TabletSchemaCloudPB* schema = nullptr) { config::write_schema_kv = false; std::mt19937 rng(std::chrono::system_clock::now().time_since_epoch().count()); std::uniform_int_distribution dist1(1, 4); @@ -451,14 +492,14 @@ TEST(DetachSchemaKVTest, RowsetTest) { schema_versions.push_back(dist1(rng)); ASSERT_NO_FATAL_FAILURE(insert_rowset(meta_service.get(), db_id, std::to_string(++label_base), table_id, tablet_id, - schema_versions.back())); + schema_versions.back(), schema)); } config::write_schema_kv = true; for (int i = 0; i < 15; ++i) { schema_versions.push_back(dist2(rng)); ASSERT_NO_FATAL_FAILURE(insert_rowset(meta_service.get(), db_id, std::to_string(++label_base), table_id, tablet_id, - schema_versions.back())); + schema_versions.back(), schema)); } // check get rowset response auto get_rowset_res = google::protobuf::Arena::CreateMessage(arena); @@ -481,11 +522,21 @@ TEST(DetachSchemaKVTest, RowsetTest) { 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); + 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); + EXPECT_EQ(get_rowset_res->rowset_meta(10).tablet_schema().SerializeAsString(), + schema->SerializeAsString()); + get_rowset_res->mutable_rowset_meta(10)->mutable_tablet_schema()->set_schema_version( + schema_version); + } }; insert_and_get_rowset(10031, 10032, 10033, 10034, 300); // use arena google::protobuf::Arena arena; insert_and_get_rowset(10041, 10042, 10043, 10044, 400, &arena); + TabletSchemaCloudPB schema = getVariantSchema(); + insert_and_get_rowset(10051, 10052, 10053, 10054, 500, &arena, &schema); } TEST(DetachSchemaKVTest, InsertExistedRowsetTest) { diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index 3ab783c6f8a425..82a9011dc1c78e 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -133,6 +133,15 @@ message RowsetMetaPB { optional bool has_variant_type_in_schema = 1005; } +message SchemaDictKeyList { + // save the dict keys for column pb info + repeated int32 column_dict_key_list = 1; + // save the dict keys for tablet index pb info + repeated int32 index_info_dict_key_list = 2; + // save the dict keys for sparse column pb info + repeated int32 sparse_column_dict_key_list = 3; +}; + message RowsetMetaCloudPB { required int64 rowset_id = 1; // Deprecated. Use rowset_id_v2 instead. optional int64 partition_id = 2; @@ -200,7 +209,11 @@ message RowsetMetaCloudPB { // If enable_segments_file_size is false, // the segments_file_size maybe is empty or error optional bool enable_segments_file_size = 103; + + // extra info for variants optional bool has_variant_type_in_schema = 104; + // dict key lists for compress schema info + optional SchemaDictKeyList schema_dict_key_list = 105; } message SegmentStatisticsPB { @@ -299,6 +312,18 @@ message ColumnPB { // sparse column within a variant column repeated ColumnPB sparse_columns = 21; optional bool is_auto_increment = 22; + // only reference by variant sparse columns + optional int32 parent_unique_id = 23; +} + +// Dictionary of Schema info, to reduce TabletSchemaCloudPB fdb kv size +message SchemaCloudDictionary { + map column_dict= 1; + // monotonic increasing + optional int64 current_column_dict_id = 2; + map index_dict = 3; + // monotonic increasing + optional int64 current_index_dict_id = 4; } enum IndexType { diff --git a/regression-test/data/variant_p0/concurrent_insert.out b/regression-test/data/variant_p0/concurrent_insert.out new file mode 100644 index 00000000000000..610f877220fca4 --- /dev/null +++ b/regression-test/data/variant_p0/concurrent_insert.out @@ -0,0 +1,103 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_1 -- +0 {"k0":0,"x10":123} {"k0":0,"x10":123} +0 {"k0":0,"x100":123} {"k0":0,"x100":123} +0 {"k0":0,"x110":123} {"k0":0,"x110":123} +0 {"k0":0,"x120":123} {"k0":0,"x120":123} +0 {"k0":0,"x130":123} {"k0":0,"x130":123} +0 {"k0":0,"x140":123} {"k0":0,"x140":123} +0 {"k0":0,"x150":123} {"k0":0,"x150":123} +0 {"k0":0,"x160":123} {"k0":0,"x160":123} +0 {"k0":0,"x170":123} {"k0":0,"x170":123} +0 {"k0":0,"x180":123} {"k0":0,"x180":123} +0 {"k0":0,"x20":123} {"k0":0,"x20":123} +0 {"k0":0,"x30":123} {"k0":0,"x30":123} +0 {"k0":0,"x40":123} {"k0":0,"x40":123} +0 {"k0":0,"x50":123} {"k0":0,"x50":123} +0 {"k0":0,"x60":123} {"k0":0,"x60":123} +0 {"k0":0,"x70":123} {"k0":0,"x70":123} +0 {"k0":0,"x80":123} {"k0":0,"x80":123} +0 {"k0":0,"x90":123} {"k0":0,"x90":123} +1 {"k1":1,"x1":123} {"k1":1,"x1":123} +1 {"k1":1,"x101":123} {"k1":1,"x101":123} +1 {"k1":1,"x11":123} {"k1":1,"x11":123} +1 {"k1":1,"x111":123} {"k1":1,"x111":123} +1 {"k1":1,"x121":123} {"k1":1,"x121":123} +1 {"k1":1,"x131":123} {"k1":1,"x131":123} +1 {"k1":1,"x141":123} {"k1":1,"x141":123} +1 {"k1":1,"x151":123} {"k1":1,"x151":123} +1 {"k1":1,"x161":123} {"k1":1,"x161":123} +1 {"k1":1,"x171":123} {"k1":1,"x171":123} +1 {"k1":1,"x21":123} {"k1":1,"x21":123} +1 {"k1":1,"x31":123} {"k1":1,"x31":123} +1 {"k1":1,"x41":123} {"k1":1,"x41":123} +1 {"k1":1,"x51":123} {"k1":1,"x51":123} +1 {"k1":1,"x61":123} {"k1":1,"x61":123} +1 {"k1":1,"x71":123} {"k1":1,"x71":123} +1 {"k1":1,"x81":123} {"k1":1,"x81":123} +1 {"k1":1,"x91":123} {"k1":1,"x91":123} +2 {"k2":2,"x102":123} {"k2":2,"x102":123} +2 {"k2":2,"x112":123} {"k2":2,"x112":123} +2 {"k2":2,"x12":123} {"k2":2,"x12":123} +2 {"k2":2,"x122":123} {"k2":2,"x122":123} +2 {"k2":2,"x132":123} {"k2":2,"x132":123} +2 {"k2":2,"x142":123} {"k2":2,"x142":123} +2 {"k2":2,"x152":123} {"k2":2,"x152":123} +2 {"k2":2,"x162":123} {"k2":2,"x162":123} +2 {"k2":2,"x172":123} {"k2":2,"x172":123} +2 {"k2":2,"x2":123} {"k2":2,"x2":123} +2 {"k2":2,"x22":123} {"k2":2,"x22":123} +2 {"k2":2,"x32":123} {"k2":2,"x32":123} +2 {"k2":2,"x42":123} {"k2":2,"x42":123} +2 {"k2":2,"x52":123} {"k2":2,"x52":123} +2 {"k2":2,"x62":123} {"k2":2,"x62":123} +2 {"k2":2,"x72":123} {"k2":2,"x72":123} +2 {"k2":2,"x82":123} {"k2":2,"x82":123} +2 {"k2":2,"x92":123} {"k2":2,"x92":123} +3 {"k3":3,"x103":123} {"k3":3,"x103":123} +3 {"k3":3,"x113":123} {"k3":3,"x113":123} +3 {"k3":3,"x123":123} {"k3":3,"x123":123} +3 {"k3":3,"x13":123} {"k3":3,"x13":123} +3 {"k3":3,"x133":123} {"k3":3,"x133":123} +3 {"k3":3,"x143":123} {"k3":3,"x143":123} +3 {"k3":3,"x153":123} {"k3":3,"x153":123} +3 {"k3":3,"x163":123} {"k3":3,"x163":123} +3 {"k3":3,"x173":123} {"k3":3,"x173":123} +3 {"k3":3,"x23":123} {"k3":3,"x23":123} +3 {"k3":3,"x3":123} {"k3":3,"x3":123} +3 {"k3":3,"x33":123} {"k3":3,"x33":123} +3 {"k3":3,"x43":123} {"k3":3,"x43":123} +3 {"k3":3,"x53":123} {"k3":3,"x53":123} +3 {"k3":3,"x63":123} {"k3":3,"x63":123} +3 {"k3":3,"x73":123} {"k3":3,"x73":123} +3 {"k3":3,"x83":123} {"k3":3,"x83":123} +3 {"k3":3,"x93":123} {"k3":3,"x93":123} +4 {"k4":4,"x104":123} {"k4":4,"x104":123} +4 {"k4":4,"x114":123} {"k4":4,"x114":123} +4 {"k4":4,"x124":123} {"k4":4,"x124":123} +4 {"k4":4,"x134":123} {"k4":4,"x134":123} +4 {"k4":4,"x14":123} {"k4":4,"x14":123} +4 {"k4":4,"x144":123} {"k4":4,"x144":123} +4 {"k4":4,"x154":123} {"k4":4,"x154":123} +4 {"k4":4,"x164":123} {"k4":4,"x164":123} +4 {"k4":4,"x174":123} {"k4":4,"x174":123} +4 {"k4":4,"x24":123} {"k4":4,"x24":123} +4 {"k4":4,"x34":123} {"k4":4,"x34":123} +4 {"k4":4,"x4":123} {"k4":4,"x4":123} +4 {"k4":4,"x44":123} {"k4":4,"x44":123} +4 {"k4":4,"x54":123} {"k4":4,"x54":123} +4 {"k4":4,"x64":123} {"k4":4,"x64":123} +4 {"k4":4,"x74":123} {"k4":4,"x74":123} +4 {"k4":4,"x84":123} {"k4":4,"x84":123} +4 {"k4":4,"x94":123} {"k4":4,"x94":123} +5 {"k5":5,"x105":123} {"k5":5,"x105":123} +5 {"k5":5,"x115":123} {"k5":5,"x115":123} +5 {"k5":5,"x125":123} {"k5":5,"x125":123} +5 {"k5":5,"x135":123} {"k5":5,"x135":123} +5 {"k5":5,"x145":123} {"k5":5,"x145":123} +5 {"k5":5,"x15":123} {"k5":5,"x15":123} +5 {"k5":5,"x155":123} {"k5":5,"x155":123} +5 {"k5":5,"x165":123} {"k5":5,"x165":123} +5 {"k5":5,"x175":123} {"k5":5,"x175":123} +5 {"k5":5,"x25":123} {"k5":5,"x25":123} + diff --git a/regression-test/suites/variant_github_events_p0/load.groovy b/regression-test/suites/variant_github_events_p0/load.groovy index 1f1b510e7a6c1e..e01d64ce19ab37 100644 --- a/regression-test/suites/variant_github_events_p0/load.groovy +++ b/regression-test/suites/variant_github_events_p0/load.groovy @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +import org.codehaus.groovy.runtime.IOGroovyMethods + suite("regression_test_variant_github_events_p0", "nonConcurrent"){ def backendId_to_backendIP = [:] def backendId_to_backendHttpPort = [:] @@ -77,6 +79,35 @@ suite("regression_test_variant_github_events_p0", "nonConcurrent"){ load_json_data.call(table_name, """${getS3Url() + '/regression/gharchive.m/2022-11-07-10.json'}""") load_json_data.call(table_name, """${getS3Url() + '/regression/gharchive.m/2022-11-07-22.json'}""") load_json_data.call(table_name, """${getS3Url() + '/regression/gharchive.m/2022-11-07-23.json'}""") + + def tablets = sql_return_maparray """ show tablets from github_events; """ + // trigger compactions for all tablets in github_events + for (def tablet in tablets) { + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_run_cumulative_compaction(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactJson = parseJson(out.trim()) + } + + // wait for all compactions done + for (def tablet in tablets) { + boolean running = true + do { + Thread.sleep(1000) + String tablet_id = tablet.TabletId + backend_id = tablet.BackendId + (code, out, err) = be_get_compaction_status(backendId_to_backendIP.get(backend_id), backendId_to_backendHttpPort.get(backend_id), tablet_id) + logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err) + assertEquals(code, 0) + def compactionStatus = parseJson(out.trim()) + assertEquals("success", compactionStatus.status.toLowerCase()) + running = compactionStatus.run_status + } while (running) + } + + // TODO fix compaction issue, this case could be stable qt_sql """select cast(v["payload"]["pull_request"]["additions"] as int) from github_events where cast(v["repo"]["name"] as string) = 'xpressengine/xe-core' order by 1;""" qt_sql """select * from github_events where cast(v["repo"]["name"] as string) = 'xpressengine/xe-core' order by 1 limit 10""" diff --git a/regression-test/suites/variant_p0/complexjson.groovy b/regression-test/suites/variant_p0/complexjson.groovy index 0ebb038bb34f53..012089a3f847c8 100644 --- a/regression-test/suites/variant_p0/complexjson.groovy +++ b/regression-test/suites/variant_p0/complexjson.groovy @@ -28,7 +28,7 @@ suite("regression_test_variant_complexjson", "variant_type_complex_json") { properties("replication_num" = "1", "disable_auto_compaction" = "true"); """ } - table_name = "complexjson" + def table_name = "complexjson" create_table table_name sql """insert into ${table_name} values (1, '{ "id": 1, diff --git a/regression-test/suites/variant_p0/concurrent_insert.groovy b/regression-test/suites/variant_p0/concurrent_insert.groovy new file mode 100644 index 00000000000000..d8a96c92417731 --- /dev/null +++ b/regression-test/suites/variant_p0/concurrent_insert.groovy @@ -0,0 +1,54 @@ +// 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("regression_test_variant_concurrent_schema_update", ""){ + def table_name = "var_concurrent" + sql "DROP TABLE IF EXISTS ${table_name}" + sql """ + CREATE TABLE IF NOT EXISTS ${table_name} ( + k bigint, + v variant, + v1 variant + ) + DUPLICATE KEY(`k`) + DISTRIBUTED BY HASH(k) BUCKETS 3 + properties("replication_num" = "1"); + """ + t1 = Thread.startDaemon { + for (int k = 1; k <= 60; k++) { + int x = k % 10; + sql """insert into ${table_name} values(${x}, '{"k${x}" : ${x}, "x${k}" : 123}', '{"k${x}" : ${x}, "x${k}" : 123}')""" + } + } + t2 = Thread.startDaemon { + for (int k = 61; k <= 120; k++) { + int x = k % 10; + sql """insert into ${table_name} values(${x}, '{"k${x}" : ${x}, "x${k}" : 123}', '{"k${x}" : ${x}, "x${k}" : 123}')""" + } + } + t3 = Thread.startDaemon { + for (int k = 121; k <= 180; k++) { + int x = k % 10; + sql """insert into ${table_name} values(${x}, '{"k${x}" : ${x}, "x${k}" : 123}', '{"k${x}" : ${x}, "x${k}" : 123}')""" + } + } + t1.join() + t2.join() + t3.join() + qt_sql_1 "select * from ${table_name} order by k, cast(v as string), cast(v1 as string) limit 100" + // qt_sql_3 """desc ${table_name}""" +} \ No newline at end of file From baf6d8ee78c769d75ff9fc6715fc0b429586a4ee Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 12 Apr 2024 10:33:13 +0800 Subject: [PATCH 17/71] [pipelineX](fix) Fix data pooling judgement for bucket join (#33533) --- be/src/runtime/runtime_state.cpp | 7 +++++-- .../main/java/org/apache/doris/planner/OlapScanNode.java | 5 +++++ .../src/main/java/org/apache/doris/planner/ScanNode.java | 7 ++++++- .../src/main/java/org/apache/doris/qe/Coordinator.java | 6 ++---- 4 files changed, 18 insertions(+), 7 deletions(-) diff --git a/be/src/runtime/runtime_state.cpp b/be/src/runtime/runtime_state.cpp index 2713ee441dd0df..2d9d939186de99 100644 --- a/be/src/runtime/runtime_state.cpp +++ b/be/src/runtime/runtime_state.cpp @@ -531,12 +531,15 @@ Status RuntimeState::register_producer_runtime_filter(const doris::TRuntimeFilte bool need_local_merge, doris::IRuntimeFilter** producer_filter, bool build_bf_exactly) { + // If runtime filter need to be local merged, `build_bf_exactly` will lead to bloom filters with + // different size need to be merged which is not allowed. + // So if `need_local_merge` is true, we will disable `build_bf_exactly`. if (desc.has_remote_targets || need_local_merge) { return global_runtime_filter_mgr()->register_local_merge_producer_filter( - desc, query_options(), producer_filter, build_bf_exactly); + desc, query_options(), producer_filter, build_bf_exactly && !need_local_merge); } else { return local_runtime_filter_mgr()->register_producer_filter( - desc, query_options(), producer_filter, build_bf_exactly); + desc, query_options(), producer_filter, build_bf_exactly && !need_local_merge); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java index e96a734151d96c..61ffc770ae3905 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -1813,4 +1813,9 @@ public void addTopnFilterSortNode(SortNode sortNode) { public List getTopnFilterSortNodes() { return topnFilterSortNodes; } + + @Override + public int numScanBackends() { + return scanBackendIds.size(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java index 490a72f895b12e..a0b5d5911cfae2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java @@ -734,7 +734,12 @@ public boolean ignoreStorageDataDistribution(ConnectContext context, int numBack && context.getSessionVariable().getEnablePipelineXEngine() && !fragment.hasNullAwareLeftAntiJoin() && getScanRangeNum() - < ConnectContext.get().getSessionVariable().getParallelExecInstanceNum() * numBackends; + < ConnectContext.get().getSessionVariable().getParallelExecInstanceNum() + * (numScanBackends() > 0 ? numScanBackends() : numBackends); + } + + public int numScanBackends() { + return 0; } public int getScanRangeNum() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 82b7cef3607353..c389bd36240145 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -2072,8 +2072,7 @@ private void computeFragmentHosts() throws Exception { && context.getSessionVariable().isForceToLocalShuffle(); boolean ignoreStorageDataDistribution = forceToLocalShuffle || (scanNodes.stream() .allMatch(scanNode -> scanNode.ignoreStorageDataDistribution(context, - fragmentExecParamsMap.get(scanNode.getFragment().getFragmentId()) - .scanRangeAssignment.size())) && useNereids); + addressToBackendID.size())) && useNereids); if (node.isPresent() && (!node.get().shouldDisableSharedScan(context) || ignoreStorageDataDistribution)) { expectedInstanceNum = Math.max(expectedInstanceNum, 1); @@ -2965,8 +2964,7 @@ private void assignScanRanges(PlanFragmentId fragmentId, int parallelExecInstanc && context.getSessionVariable().isForceToLocalShuffle(); boolean ignoreStorageDataDistribution = forceToLocalShuffle || (scanNodes.stream() .allMatch(node -> node.ignoreStorageDataDistribution(context, - fragmentExecParamsMap.get(node.getFragment().getFragmentId()) - .scanRangeAssignment.size())) + addressToBackendID.size())) && addressToScanRanges.entrySet().stream().allMatch(addressScanRange -> { return addressScanRange.getValue().size() < parallelExecInstanceNum; }) && useNereids); From ad24d4ac3a1bff25437362b9a3584b608ee3a430 Mon Sep 17 00:00:00 2001 From: lihangyu <15605149486@163.com> Date: Fri, 12 Apr 2024 10:33:47 +0800 Subject: [PATCH 18/71] [Feature](Variant) support aggregation model for Variant type (#33493) refactor use `insert_from` to replace `replace_column_data` for variable lengths columns --- be/src/olap/tablet_schema.h | 21 +- be/src/vec/columns/column_array.h | 16 +- be/src/vec/columns/column_map.h | 18 +- be/src/vec/columns/column_object.cpp | 21 +- be/src/vec/columns/column_object.h | 10 +- be/src/vec/columns/column_string.h | 29 +-- be/src/vec/columns/column_struct.h | 14 +- be/src/vec/common/schema_util.cpp | 7 +- be/src/vec/common/schema_util.h | 4 +- be/src/vec/exec/scan/new_olap_scanner.cpp | 2 +- be/src/vec/olap/block_reader.cpp | 4 +- be/src/vec/olap/vertical_block_reader.cpp | 4 +- regression-test/data/variant_p0/agg.out | 78 ++++++ .../variant_p0/compaction/test_compaction.out | 104 ++++++++ regression-test/data/variant_p0/desc.out | 230 +++++++++--------- regression-test/suites/variant_p0/agg.groovy | 63 +++++ .../compaction/test_compaction.groovy | 11 +- 17 files changed, 422 insertions(+), 214 deletions(-) create mode 100644 regression-test/data/variant_p0/agg.out create mode 100644 regression-test/suites/variant_p0/agg.groovy diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h index b5b8df730b3350..61d972dadc6938 100644 --- a/be/src/olap/tablet_schema.h +++ b/be/src/olap/tablet_schema.h @@ -127,7 +127,10 @@ class TabletColumn { int frac() const { return _frac; } inline bool visible() const { return _visible; } - void set_aggregation_method(FieldAggregationMethod agg) { _aggregation = agg; } + void set_aggregation_method(FieldAggregationMethod agg) { + _aggregation = agg; + _aggregation_name = get_string_by_aggregation_type(agg); + } /** * Add a sub column. @@ -426,6 +429,22 @@ class TabletSchema { return str; } + string dump_full_schema() const { + string str = "["; + for (auto p : _cols) { + if (str.size() > 1) { + str += ", "; + } + ColumnPB col_pb; + p->to_schema_pb(&col_pb); + str += "("; + str += col_pb.ShortDebugString(); + str += ")"; + } + str += "]"; + return str; + } + vectorized::Block create_block_by_cids(const std::vector& cids); std::shared_ptr copy_without_extracted_columns(); diff --git a/be/src/vec/columns/column_array.h b/be/src/vec/columns/column_array.h index 3176f7a45c6e15..118e7ab05c6428 100644 --- a/be/src/vec/columns/column_array.h +++ b/be/src/vec/columns/column_array.h @@ -217,23 +217,11 @@ class ColumnArray final : public COWHelper { const uint32_t* indices_end) override; void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { - DCHECK(size() > self_row); - const auto& r = assert_cast(rhs); - const size_t nested_row_size = r.size_at(row); - const size_t r_nested_start_off = r.offset_at(row); - - // we should clear data because we call resize() before replace_column_data() - if (self_row == 0) { - data->clear(); - } - get_offsets()[self_row] = get_offsets()[self_row - 1] + nested_row_size; - // we make sure call replace_column_data() by order so, here we just insert data for nested - data->insert_range_from(r.get_data(), r_nested_start_off, nested_row_size); + LOG(FATAL) << "Method replace_column_data is not supported for " << get_name(); } void replace_column_data_default(size_t self_row = 0) override { - DCHECK(size() > self_row); - get_offsets()[self_row] = get_offsets()[self_row - 1]; + LOG(FATAL) << "Method replace_column_data_default is not supported for " << get_name(); } void clear() override { diff --git a/be/src/vec/columns/column_map.h b/be/src/vec/columns/column_map.h index ed5a692defd182..2cdfcae8c73f70 100644 --- a/be/src/vec/columns/column_map.h +++ b/be/src/vec/columns/column_map.h @@ -135,25 +135,11 @@ class ColumnMap final : public COWHelper { } void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { - DCHECK(size() > self_row); - const auto& r = assert_cast(rhs); - const size_t nested_row_size = r.size_at(row); - const size_t r_key_nested_start_off = r.offset_at(row); - const size_t r_val_nested_start_off = r.offset_at(row); - - if (self_row == 0) { - keys_column->clear(); - values_column->clear(); - } - get_offsets()[self_row] = get_offsets()[self_row - 1] + nested_row_size; - // here we use batch size to avoid many virtual call in nested column - keys_column->insert_range_from(r.get_keys(), r_key_nested_start_off, nested_row_size); - values_column->insert_range_from(r.get_values(), r_val_nested_start_off, nested_row_size); + LOG(FATAL) << "Method replace_column_data is not supported for " << get_name(); } void replace_column_data_default(size_t self_row = 0) override { - DCHECK(size() > self_row); - get_offsets()[self_row] = get_offsets()[self_row - 1]; + LOG(FATAL) << "Method replace_column_data_default is not supported for " << get_name(); } ColumnArray::Offsets64& ALWAYS_INLINE get_offsets() { diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp index b88887b580f7ec..c596717194fc23 100644 --- a/be/src/vec/columns/column_object.cpp +++ b/be/src/vec/columns/column_object.cpp @@ -732,18 +732,7 @@ Field ColumnObject::operator[](size_t n) const { } void ColumnObject::get(size_t n, Field& res) const { - if (!is_finalized()) { - const_cast(this)->finalize(); - } - auto& map = res.get(); - for (const auto& entry : subcolumns) { - auto it = map.try_emplace(entry->path.get_path()).first; - if (WhichDataType(remove_nullable(entry->data.data_types.back())).is_json()) { - // JsonbFiled is special case - it->second = JsonbField(); - } - entry->data.data.back()->get(n, it->second); - } + res = (*this)[n]; } Status ColumnObject::try_insert_indices_from(const IColumn& src, const int* indices_begin, @@ -1513,4 +1502,12 @@ Status ColumnObject::sanitize() const { return Status::OK(); } +void ColumnObject::replace_column_data(const IColumn& col, size_t row, size_t self_row) { + LOG(FATAL) << "Method replace_column_data is not supported for " << get_name(); +} + +void ColumnObject::replace_column_data_default(size_t self_row) { + LOG(FATAL) << "Method replace_column_data_default is not supported for " << get_name(); +} + } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h index a940b4a8811fc7..26b2c66a7551b7 100644 --- a/be/src/vec/columns/column_object.h +++ b/be/src/vec/columns/column_object.h @@ -456,13 +456,11 @@ class ColumnObject final : public COWHelper { LOG(FATAL) << "should not call the method in column object"; } - void replace_column_data(const IColumn&, size_t row, size_t self_row) override { - LOG(FATAL) << "should not call the method in column object"; - } + bool is_variable_length() const override { return true; } - void replace_column_data_default(size_t self_row) override { - LOG(FATAL) << "should not call the method in column object"; - } + void replace_column_data(const IColumn&, size_t row, size_t self_row) override; + + void replace_column_data_default(size_t self_row) override; void get_indices_of_non_default_rows(Offsets64&, size_t, size_t) const override { LOG(FATAL) << "should not call the method in column object"; diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index b858ab86bf3503..405ada3b48d361 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -545,37 +545,12 @@ class ColumnString final : public COWHelper { } void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { - // we check this column size and self_row because we need to make sure when we call - // replace_column_data() with a batch column data. - // and this column data is cleared at the every beginning. - // next we replace column one by one. - DCHECK(size() > self_row); - const auto& r = assert_cast(rhs); - auto data = r.get_data_at(row); - - if (!self_row) { - // self_row == 0 means we first call replace_column_data() with batch column data. so we - // should clean last batch column data. - chars.clear(); - offsets[self_row] = data.size; - } else { - offsets[self_row] = offsets[self_row - 1] + data.size; - check_chars_length(offsets[self_row], self_row); - } - - chars.insert(data.data, data.data + data.size); + LOG(FATAL) << "Method replace_column_data is not supported for " << get_name(); } // should replace according to 0,1,2... ,size,0,1,2... void replace_column_data_default(size_t self_row = 0) override { - DCHECK(size() > self_row); - - if (!self_row) { - chars.clear(); - offsets[self_row] = 0; - } else { - offsets[self_row] = offsets[self_row - 1]; - } + LOG(FATAL) << "Method replace_column_data_default is not supported for " << get_name(); } void compare_internal(size_t rhs_row_id, const IColumn& rhs, int nan_direction_hint, diff --git a/be/src/vec/columns/column_struct.h b/be/src/vec/columns/column_struct.h index 34d1e3ecf0e3a4..2ca4fdec0153b4 100644 --- a/be/src/vec/columns/column_struct.h +++ b/be/src/vec/columns/column_struct.h @@ -88,6 +88,8 @@ class ColumnStruct final : public COWHelper { MutableColumnPtr clone_resized(size_t size) const override; size_t size() const override { return columns.at(0)->size(); } + bool is_variable_length() const override { return true; } + Field operator[](size_t n) const override; void get(size_t n, Field& res) const override; @@ -131,19 +133,11 @@ class ColumnStruct final : public COWHelper { return append_data_by_selector_impl(res, selector); } void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { - DCHECK(size() > self_row); - const auto& r = assert_cast(rhs); - - for (size_t idx = 0; idx < columns.size(); ++idx) { - columns[idx]->replace_column_data(r.get_column(idx), row, self_row); - } + LOG(FATAL) << "Method replace_column_data is not supported for " << get_name(); } void replace_column_data_default(size_t self_row = 0) override { - DCHECK(size() > self_row); - for (size_t idx = 0; idx < columns.size(); ++idx) { - columns[idx]->replace_column_data_default(self_row); - } + LOG(FATAL) << "Method replace_column_data_default is not supported for " << get_name(); } void insert_range_from(const IColumn& src, size_t start, size_t length) override; diff --git a/be/src/vec/common/schema_util.cpp b/be/src/vec/common/schema_util.cpp index 71cdece83335ab..5c7a2f8482a738 100644 --- a/be/src/vec/common/schema_util.cpp +++ b/be/src/vec/common/schema_util.cpp @@ -369,7 +369,7 @@ void update_least_sparse_column(const std::vector& schemas, update_least_schema_internal(subcolumns_types, common_schema, true, variant_col_unique_id); } -void inherit_tablet_index(TabletSchemaSPtr& schema) { +void inherit_root_attributes(TabletSchemaSPtr& schema) { std::unordered_map variants_index_meta; // Get all variants tablet index metas if exist for (const auto& col : schema->columns()) { @@ -392,6 +392,7 @@ void inherit_tablet_index(TabletSchemaSPtr& schema) { // above types are not supported in bf col.set_is_bf_column(schema->column(col.parent_unique_id()).is_bf_column()); } + col.set_aggregation_method(schema->column(col.parent_unique_id()).aggregation()); auto it = variants_index_meta.find(col.parent_unique_id()); // variant has no index meta, ignore if (it == variants_index_meta.end()) { @@ -467,7 +468,7 @@ Status get_least_common_schema(const std::vector& schemas, update_least_sparse_column(schemas, output_schema, unique_id, path_set); } - inherit_tablet_index(output_schema); + inherit_root_attributes(output_schema); if (check_schema_size && output_schema->columns().size() > config::variant_max_merged_tablet_schema_size) { return Status::DataQualityError("Reached max column size limit {}", @@ -710,7 +711,7 @@ void rebuild_schema_and_block(const TabletSchemaSPtr& original, VLOG_DEBUG << "set root_path : " << full_root_path.get_path(); } - vectorized::schema_util::inherit_tablet_index(flush_schema); + vectorized::schema_util::inherit_root_attributes(flush_schema); } // --------------------------- diff --git a/be/src/vec/common/schema_util.h b/be/src/vec/common/schema_util.h index e1bb037f70e8c7..e6ed60480f5ac1 100644 --- a/be/src/vec/common/schema_util.h +++ b/be/src/vec/common/schema_util.h @@ -116,8 +116,8 @@ void update_least_sparse_column(const std::vector& schemas, TabletSchemaSPtr& common_schema, int32_t variant_col_unique_id, const std::unordered_set& path_set); -// inherit index info from it's parent column -void inherit_tablet_index(TabletSchemaSPtr& schema); +// inherit attributes like index/agg info from it's parent column +void inherit_root_attributes(TabletSchemaSPtr& schema); // Rebuild schema from original schema by extend dynamic columns generated from ColumnObject. // Block consists of two parts, dynamic part of columns and static part of columns. diff --git a/be/src/vec/exec/scan/new_olap_scanner.cpp b/be/src/vec/exec/scan/new_olap_scanner.cpp index c4b7f68160985d..98894e225eac3b 100644 --- a/be/src/vec/exec/scan/new_olap_scanner.cpp +++ b/be/src/vec/exec/scan/new_olap_scanner.cpp @@ -444,7 +444,7 @@ Status NewOlapScanner::_init_variant_columns() { } } } - schema_util::inherit_tablet_index(tablet_schema); + schema_util::inherit_root_attributes(tablet_schema); return Status::OK(); } diff --git a/be/src/vec/olap/block_reader.cpp b/be/src/vec/olap/block_reader.cpp index e2f37fee0101ac..7a23c9c445692f 100644 --- a/be/src/vec/olap/block_reader.cpp +++ b/be/src/vec/olap/block_reader.cpp @@ -476,10 +476,10 @@ size_t BlockReader::_copy_agg_data() { auto& dst_column = _stored_data_columns[idx]; if (_stored_has_variable_length_tag[idx]) { //variable length type should replace ordered + dst_column->clear(); for (size_t i = 0; i < copy_size; i++) { auto& ref = _stored_row_ref[i]; - dst_column->replace_column_data(*ref.block->get_by_position(idx).column, - ref.row_pos, i); + dst_column->insert_from(*ref.block->get_by_position(idx).column, ref.row_pos); } } else { for (auto& it : _temp_ref_map) { diff --git a/be/src/vec/olap/vertical_block_reader.cpp b/be/src/vec/olap/vertical_block_reader.cpp index 4fa518d58ac677..0092a020a2e76b 100644 --- a/be/src/vec/olap/vertical_block_reader.cpp +++ b/be/src/vec/olap/vertical_block_reader.cpp @@ -330,10 +330,10 @@ size_t VerticalBlockReader::_copy_agg_data() { auto& dst_column = _stored_data_columns[idx]; if (_stored_has_variable_length_tag[idx]) { //variable length type should replace ordered + dst_column->clear(); for (size_t i = 0; i < copy_size; i++) { auto& ref = _stored_row_ref[i]; - dst_column->replace_column_data(*ref.block->get_by_position(idx).column, - ref.row_pos, i); + dst_column->insert_from(*ref.block->get_by_position(idx).column, ref.row_pos); } } else { for (auto& it : _temp_ref_map) { diff --git a/regression-test/data/variant_p0/agg.out b/regression-test/data/variant_p0/agg.out new file mode 100644 index 00000000000000..958e3d41a7bf85 --- /dev/null +++ b/regression-test/data/variant_p0/agg.out @@ -0,0 +1,78 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- + +-- !sql2 -- +1 \N {"c":[{"a":1}]} +1022 \N {"f":17034,"g":1.111} +1029 \N {"c":1} +1999 \N {"c":1} + +-- !sql3 -- +1 {"a":1,"b":{"c":[{"a":1}]}} +2 [2] +3 3 +4 "4" +5 5 + +-- !sql4 -- +{"c":[{"a":1}]} [{"a":1}] \N +{"f":17034,"g":1.111} \N \N +{"c":1} 1 \N +{"c":1} 1 \N + +-- !sql5 -- + +-- !sql6 -- +{"c":[{"a":1}]} +{"f":17034,"g":1.111} +{"c":1} +{"c":1} + +-- !sql7 -- +1 {"a":1,"b":{"c":[{"a":1}]}} 59 +1022 {"a":1,"b":{"f":17034,"g":1.111}} 12 +1029 \N 12 +1999 {"a":1,"b":{"c":1}} 11 + +-- !sql8 -- +1 {"a":1,"b":{"c":[{"a":1}]}} 59 +2 [2] 2 +3 3 3 +4 "4" 4 +5 5 5 +6 "[6]" 6 +7 7 7 +8 8.11111 8 +9 "9999" 9 +10 1000000 10 +11 [123] 11 +12 [123.2] 12 +1022 {"a":1,"b":{"f":17034,"g":1.111}} 12 +1029 \N 12 +1999 {"a":1,"b":{"c":1}} 11 +19921 {"a":1,"d":10} 11 + +-- !sql9 -- +0 {"a":11245,"f":["123456"]} +1 {"a":11245,"f":["123456"]} +2 {"a":11245,"f":["123456"]} +3 {"a":11245,"f":["123456"]} +4 {"a":11245,"f":["123456"]} +5 {"a":11245,"f":["123456"]} +6 {"a":11245,"f":["123456"]} +7 {"a":11245,"f":["123456"]} +8 {"a":11245,"f":["123456"]} +9 {"a":11245,"f":["123456"]} + +-- !sql9 -- +1025 {"a":11245,"y":11111111} +1026 {"a":11245,"y":11111111} +1027 {"a":11245,"y":11111111} +1028 {"a":11245,"y":11111111} +1029 {"a":11245,"y":11111111} +1030 {"a":11245,"y":11111111} +1031 {"a":11245,"y":11111111} +1032 {"a":11245,"y":11111111} +1033 {"a":11245,"y":11111111} +1034 {"a":11245,"y":11111111} + diff --git a/regression-test/data/variant_p0/compaction/test_compaction.out b/regression-test/data/variant_p0/compaction/test_compaction.out index 50943aef7b658a..57a5c142fbb1f6 100644 --- a/regression-test/data/variant_p0/compaction/test_compaction.out +++ b/regression-test/data/variant_p0/compaction/test_compaction.out @@ -279,3 +279,107 @@ {"c":[{"a":1}]} [{"a":1}] {"c":1} 1 +-- !sql_1 -- +1 {"x":[1]} +2 {"a":"1"} +3 {"x":[3]} +4 {"y":1} +5 {"z":2.0} +6 {"x":111} +7 {"m":1} +8 {"l":2} +9 {"g":1.11} +10 {"z":1.1111} +11 {"sala":0} +12 {"dddd":0.1} +13 {"a":1} +14 {"a":[[[1]]]} +15 {"a":1} +16 {"a":"1223"} +17 {"a":[1]} +18 {"a":["1",2,1.1]} +19 {"a":1,"b":{"c":1}} +20 {"a":1,"b":{"c":[{"a":1}]}} +21 {"a":1,"b":{"c":[{"a":1}]}} +22 {"a":1,"b":{"c":[{"a":1}]}} +1022 {"a":1,"b":10} +1029 {"a":1,"b":{"c":1}} +1999 {"a":1,"b":{"c":1}} +19921 {"a":1,"b":10} + +-- !sql_2 -- +14 [null] +17 [1] +18 [1, 2, null] + +-- !sql_3 -- +19 1 {"c":1} +20 1 {"c":[{"a":1}]} +21 1 {"c":[{"a":1}]} +22 1 {"c":[{"a":1}]} +1029 1 {"c":1} +1999 1 {"c":1} + +-- !sql_5 -- +10 \N +{"c":1} 1 +{"c":1} 1 +10 \N +{"c":[{"a":1}]} [{"a":1}] +{"c":[{"a":1}]} [{"a":1}] +{"c":[{"a":1}]} [{"a":1}] +{"c":1} 1 +{} \N +{} \N + +-- !sql_11 -- +1 {"x":[1]} +2 {"a":"1"} +3 {"x":[3]} +4 {"y":1} +5 {"z":2.0} +6 {"x":111} +7 {"m":1} +8 {"l":2} +9 {"g":1.11} +10 {"z":1.1111} +11 {"sala":0} +12 {"dddd":0.1} +13 {"a":1} +14 {"a":[[[1]]]} +15 {"a":1} +16 {"a":"1223"} +17 {"a":[1]} +18 {"a":["1",2,1.1]} +19 {"a":1,"b":{"c":1}} +20 {"a":1,"b":{"c":[{"a":1}]}} +21 {"a":1,"b":{"c":[{"a":1}]}} +22 {"a":1,"b":{"c":[{"a":1}]}} +1022 {"a":1,"b":10} +1029 {"a":1,"b":{"c":1}} +1999 {"a":1,"b":{"c":1}} +19921 {"a":1,"b":10} + +-- !sql_22 -- +14 [null] +17 [1] +18 [1, 2, null] + +-- !sql_33 -- +19 1 {"c":1} +20 1 {"c":[{"a":1}]} +21 1 {"c":[{"a":1}]} +22 1 {"c":[{"a":1}]} +1029 1 {"c":1} +1999 1 {"c":1} + +-- !sql_55 -- +10 \N +{"c":1} 1 +{"c":1} 1 +10 \N +{"c":[{"a":1}]} [{"a":1}] +{"c":[{"a":1}]} [{"a":1}] +{"c":[{"a":1}]} [{"a":1}] +{"c":1} 1 + diff --git a/regression-test/data/variant_p0/desc.out b/regression-test/data/variant_p0/desc.out index ecf0925a9c9cd0..b46b5f9b4b08d8 100644 --- a/regression-test/data/variant_p0/desc.out +++ b/regression-test/data/variant_p0/desc.out @@ -2,107 +2,107 @@ -- !sql_1 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_2 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.ddd.aaa TINYINT Yes false \N -v.ddd.mxmxm JSON Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.ddd.aaa TINYINT Yes false \N NONE +v.ddd.mxmxm JSON Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_3 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_6_1 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.ddd.aaa TINYINT Yes false \N -v.ddd.mxmxm JSON Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.ddd.aaa TINYINT Yes false \N NONE +v.ddd.mxmxm JSON Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_6_2 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_6_3 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE -- !sql_6 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N -v.ddd.aaa TINYINT Yes false \N -v.ddd.mxmxm JSON Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE +v.ddd.aaa TINYINT Yes false \N NONE +v.ddd.mxmxm JSON Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_7 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_7_1 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_7_2 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE -- !sql_7_3 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N -v.xxxx TEXT Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE +v.xxxx TEXT Yes false \N NONE -- !sql_8 -- k BIGINT Yes true \N v1 VARIANT Yes false \N NONE v2 VARIANT Yes false \N NONE v3 VARIANT Yes false \N NONE -v1.a SMALLINT Yes false \N -v1.b JSON Yes false \N -v1.c.c SMALLINT Yes false \N -v1.c.e DOUBLE Yes false \N -v1.oooo.xxxx.xxx TINYINT Yes false \N -v2.a SMALLINT Yes false \N -v2.xxxx TEXT Yes false \N -v3.a SMALLINT Yes false \N -v3.b JSON Yes false \N -v3.c.c SMALLINT Yes false \N -v3.c.e DOUBLE Yes false \N +v1.a SMALLINT Yes false \N NONE +v1.b JSON Yes false \N NONE +v1.c.c SMALLINT Yes false \N NONE +v1.c.e DOUBLE Yes false \N NONE +v1.oooo.xxxx.xxx TINYINT Yes false \N NONE +v2.a SMALLINT Yes false \N NONE +v2.xxxx TEXT Yes false \N NONE +v3.a SMALLINT Yes false \N NONE +v3.b JSON Yes false \N NONE +v3.c.c SMALLINT Yes false \N NONE +v3.c.e DOUBLE Yes false \N NONE -- !sql_9 -- k BIGINT Yes true \N @@ -111,88 +111,88 @@ v VARIANT Yes false \N NONE -- !sql_9_1 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N -v.oooo.xxxx.xxx TINYINT Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE +v.oooo.xxxx.xxx TINYINT Yes false \N NONE -- !sql_10 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.k1 TINYINT Yes false \N -v.k2 TEXT Yes false \N -v.k3 ARRAY Yes false [] -v.k4 DOUBLE Yes false \N -v.k5 JSON Yes false \N +v.k1 TINYINT Yes false \N NONE +v.k2 TEXT Yes false \N NONE +v.k3 ARRAY Yes false [] NONE +v.k4 DOUBLE Yes false \N NONE +v.k5 JSON Yes false \N NONE -- !sql_10_1 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE v2 VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N -v.k1 TINYINT Yes false \N -v.k2 TEXT Yes false \N -v.k3 ARRAY Yes false [] -v.k4 DOUBLE Yes false \N -v.k5 JSON Yes false \N -v.oooo.xxxx.xxx TINYINT Yes false \N -v2.a SMALLINT Yes false \N -v2.b JSON Yes false \N -v2.c.c SMALLINT Yes false \N -v2.c.e DOUBLE Yes false \N -v2.oooo.xxxx.xxx TINYINT Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE +v.k1 TINYINT Yes false \N NONE +v.k2 TEXT Yes false \N NONE +v.k3 ARRAY Yes false [] NONE +v.k4 DOUBLE Yes false \N NONE +v.k5 JSON Yes false \N NONE +v.oooo.xxxx.xxx TINYINT Yes false \N NONE +v2.a SMALLINT Yes false \N NONE +v2.b JSON Yes false \N NONE +v2.c.c SMALLINT Yes false \N NONE +v2.c.e DOUBLE Yes false \N NONE +v2.oooo.xxxx.xxx TINYINT Yes false \N NONE -- !sql_10_2 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N -v.k1 TINYINT Yes false \N -v.k2 TEXT Yes false \N -v.k3 ARRAY Yes false [] -v.k4 DOUBLE Yes false \N -v.k5 JSON Yes false \N -v.oooo.xxxx.xxx TINYINT Yes false \N -v2.a SMALLINT Yes false \N -v2.b JSON Yes false \N -v2.c.c SMALLINT Yes false \N -v2.c.e DOUBLE Yes false \N -v2.oooo.xxxx.xxx TINYINT Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE +v.k1 TINYINT Yes false \N NONE +v.k2 TEXT Yes false \N NONE +v.k3 ARRAY Yes false [] NONE +v.k4 DOUBLE Yes false \N NONE +v.k5 JSON Yes false \N NONE +v.oooo.xxxx.xxx TINYINT Yes false \N NONE +v2.a SMALLINT Yes false \N NONE +v2.b JSON Yes false \N NONE +v2.c.c SMALLINT Yes false \N NONE +v2.c.e DOUBLE Yes false \N NONE +v2.oooo.xxxx.xxx TINYINT Yes false \N NONE -- !sql_10_3 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE v3 VARIANT Yes false \N NONE -v.a SMALLINT Yes false \N -v.b JSON Yes false \N -v.c.c SMALLINT Yes false \N -v.c.e DOUBLE Yes false \N -v.k1 TINYINT Yes false \N -v.k2 TEXT Yes false \N -v.k3 ARRAY Yes false [] -v.k4 DOUBLE Yes false \N -v.k5 JSON Yes false \N -v.oooo.xxxx.xxx TINYINT Yes false \N -v3.a SMALLINT Yes false \N -v3.b JSON Yes false \N -v3.c.c SMALLINT Yes false \N -v3.c.e DOUBLE Yes false \N -v3.oooo.xxxx.xxx TINYINT Yes false \N +v.a SMALLINT Yes false \N NONE +v.b JSON Yes false \N NONE +v.c.c SMALLINT Yes false \N NONE +v.c.e DOUBLE Yes false \N NONE +v.k1 TINYINT Yes false \N NONE +v.k2 TEXT Yes false \N NONE +v.k3 ARRAY Yes false [] NONE +v.k4 DOUBLE Yes false \N NONE +v.k5 JSON Yes false \N NONE +v.oooo.xxxx.xxx TINYINT Yes false \N NONE +v3.a SMALLINT Yes false \N NONE +v3.b JSON Yes false \N NONE +v3.c.c SMALLINT Yes false \N NONE +v3.c.e DOUBLE Yes false \N NONE +v3.oooo.xxxx.xxx TINYINT Yes false \N NONE -- !sql_11 -- k BIGINT Yes true \N v VARIANT Yes false \N NONE -v.!@#^&*() TEXT Yes false \N -v.名字 TEXT Yes false \N -v.画像.丬文 TEXT Yes false \N -v.画像.地址 TEXT Yes false \N -v.金额 SMALLINT Yes false \N +v.!@#^&*() TEXT Yes false \N NONE +v.名字 TEXT Yes false \N NONE +v.画像.丬文 TEXT Yes false \N NONE +v.画像.地址 TEXT Yes false \N NONE +v.金额 SMALLINT Yes false \N NONE -- !sql_12 -- k BIGINT Yes true \N diff --git a/regression-test/suites/variant_p0/agg.groovy b/regression-test/suites/variant_p0/agg.groovy new file mode 100644 index 00000000000000..b5010ee7f723c8 --- /dev/null +++ b/regression-test/suites/variant_p0/agg.groovy @@ -0,0 +1,63 @@ +// 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("regression_test_variant_agg"){ + sql """DROP TABLE IF EXISTS var_agg""" + sql """ + CREATE TABLE IF NOT EXISTS var_agg ( + k bigint, + v variant replace, + s bigint sum + ) + AGGREGATE KEY(`k`) + DISTRIBUTED BY HASH(k) BUCKETS 4 + properties("replication_num" = "1", "disable_auto_compaction" = "false"); + """ + sql """insert into var_agg values (1, '[1]', 1),(1, '{"a" : 1}', 1);""" + sql """insert into var_agg values (2, '[2]', 2),(1, '{"a" : [[[1]]]}', 2);""" + sql """insert into var_agg values (3, '3', 3),(1, '{"a" : 1}', 3), (1, '{"a" : [1]}', 3);""" + sql """insert into var_agg values (4, '"4"', 4),(1, '{"a" : "1223"}', 4);""" + sql """insert into var_agg values (5, '5', 5),(1, '{"a" : [1]}', 5);""" + sql """insert into var_agg values (6, '"[6]"', 6),(1, '{"a" : ["1", 2, 1.1]}', 6);""" + sql """insert into var_agg values (7, '7', 7),(1, '{"a" : 1, "b" : {"c" : 1}}', 7);""" + sql """insert into var_agg values (8, '8.11111', 8),(1, '{"a" : 1, "b" : {"c" : [{"a" : 1}]}}', 8);""" + sql """insert into var_agg values (9, '"9999"', 9),(1, '{"a" : 1, "b" : {"c" : [{"a" : 1}]}}', 9);""" + sql """insert into var_agg values (10, '1000000', 10),(1, '{"a" : 1, "b" : {"c" : [{"a" : 1}]}}', 10);""" + sql """insert into var_agg values (11, '[123.0]', 11),(1999, '{"a" : 1, "b" : {"c" : 1}}', 11),(19921, '{"a" : 1, "d" : 10}', 11);""" + sql """insert into var_agg values (12, '[123.2]', 12),(1022, '{"a" : 1, "b" : {"f" : 17034, "g" :1.111 }}', 12),(1029, '{"a" : 1, "b" : {"c" : 1}}', 12);""" + qt_sql1 "select k, cast(v['a'] as array) from var_agg where size(cast(v['a'] as array)) > 0 order by k, cast(v['a'] as string) asc" + qt_sql2 "select k, cast(v as int), cast(v['b'] as string) from var_agg where length(cast(v['b'] as string)) > 4 order by k, cast(v as string), cast(v['b'] as string) " + qt_sql3 "select k, v from var_agg order by k, cast(v as string) limit 5" + qt_sql4 "select v['b'], v['b']['c'], cast(v as int) from var_agg where cast(v['b'] as string) is not null and cast(v['b'] as string) != '{}' order by k,cast(v as string) desc limit 10000;" + qt_sql5 "select v['b'] from var_agg where cast(v['b'] as int) > 0;" + qt_sql6 "select cast(v['b'] as string) from var_agg where cast(v['b'] as string) is not null and cast(v['b'] as string) != '{}' order by k, cast(v['b'] as string) " + qt_sql7 "select * from var_agg where cast(v['b'] as string) is not null and cast(v['b'] as string) != '{}' order by k, cast(v['b'] as string) " + qt_sql8 "select * from var_agg order by 1, cast(2 as string), 3" + sql "alter table var_agg drop column s" + sql """insert into var_agg select 5, '{"a" : 1234, "xxxx" : "fffff", "point" : 42000}' as json_str + union all select 5, '{"a": 1123}' as json_str union all select *, '{"a": 11245, "x" : 42005}' as json_str from numbers("number" = "1024") limit 1024;""" + sql """insert into var_agg select 5, '{"a" : 1234, "xxxx" : "fffff", "point" : 42000}' as json_str + union all select 5, '{"a": 1123}' as json_str union all select *, '{"a": 11245, "y" : 11111111}' as json_str from numbers("number" = "2048") where number > 1024 limit 1024;""" + sql """insert into var_agg select 5, '{"a" : 1234, "xxxx" : "fffff", "point" : 42000}' as json_str + union all select 5, '{"a": 1123}' as json_str union all select *, '{"a": 11245, "c" : 1.11}' as json_str from numbers("number" = "1024") limit 1024;""" + sql """insert into var_agg select 5, '{"a" : 1234, "xxxx" : "fffff", "point" : 42000}' as json_str + union all select 5, '{"a": 1123}' as json_str union all select *, '{"a": 11245, "e" : [123456]}' as json_str from numbers("number" = "1024") limit 1024;""" + sql """insert into var_agg select 5, '{"a" : 1234, "xxxx" : "fffff", "point" : 42000}' as json_str + union all select 5, '{"a": 1123}' as json_str union all select *, '{"a": 11245, "f" : ["123456"]}' as json_str from numbers("number" = "1024") limit 1024;""" + qt_sql9 "select * from var_agg order by cast(2 as string), 3, 1 limit 10" + qt_sql9 "select * from var_agg where k > 1024 order by cast(2 as string), 3, 1 limit 10" +} \ No newline at end of file diff --git a/regression-test/suites/variant_p0/compaction/test_compaction.groovy b/regression-test/suites/variant_p0/compaction/test_compaction.groovy index e5359f305fca26..48d916e38e3909 100644 --- a/regression-test/suites/variant_p0/compaction/test_compaction.groovy +++ b/regression-test/suites/variant_p0/compaction/test_compaction.groovy @@ -41,10 +41,14 @@ suite("test_compaction_variant") { } def create_table = { tableName, buckets="auto", key_type="DUPLICATE" -> sql "DROP TABLE IF EXISTS ${tableName}" + def var_def = "variant" + if (key_type == "AGGREGATE") { + var_def = "variant replace" + } sql """ CREATE TABLE IF NOT EXISTS ${tableName} ( k bigint, - v variant + v ${var_def} ) ${key_type} KEY(`k`) DISTRIBUTED BY HASH(k) BUCKETS ${buckets} @@ -52,7 +56,8 @@ suite("test_compaction_variant") { """ } - def key_types = ["DUPLICATE", "UNIQUE"] + def key_types = ["DUPLICATE", "UNIQUE", "AGGREGATE"] + // def key_types = ["AGGREGATE"] for (int i = 0; i < key_types.size(); i++) { def tableName = "simple_variant_${key_types[i]}" // 1. simple cases @@ -62,7 +67,7 @@ suite("test_compaction_variant") { sql """insert into ${tableName} values (2, '{"a" : "1"}'),(14, '{"a" : [[[1]]]}');""" sql """insert into ${tableName} values (3, '{"x" : [3]}'),(15, '{"a" : 1}')""" sql """insert into ${tableName} values (4, '{"y": 1}'),(16, '{"a" : "1223"}');""" - sql """insert into ${tableName} values (5, '{"z" : 2}'),(17, '{"a" : [1]}');""" + sql """insert into ${tableName} values (5, '{"z" : 2.0}'),(17, '{"a" : [1]}');""" sql """insert into ${tableName} values (6, '{"x" : 111}'),(18, '{"a" : ["1", 2, 1.1]}');""" sql """insert into ${tableName} values (7, '{"m" : 1}'),(19, '{"a" : 1, "b" : {"c" : 1}}');""" sql """insert into ${tableName} values (8, '{"l" : 2}'),(20, '{"a" : 1, "b" : {"c" : [{"a" : 1}]}}');""" From ab015d6379cff2e0251ce077e3c2bca238cbb9b2 Mon Sep 17 00:00:00 2001 From: walter Date: Fri, 12 Apr 2024 11:10:15 +0800 Subject: [PATCH 19/71] [fix](fs) Close local file writer when downloading finished (#33556) --- be/src/io/fs/hdfs_file_system.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/be/src/io/fs/hdfs_file_system.cpp b/be/src/io/fs/hdfs_file_system.cpp index bd5350661068a2..5ea742c20d5253 100644 --- a/be/src/io/fs/hdfs_file_system.cpp +++ b/be/src/io/fs/hdfs_file_system.cpp @@ -320,8 +320,7 @@ Status HdfsFileSystem::download_impl(const Path& remote_file, const Path& local_ RETURN_IF_ERROR(local_writer->append({read_buf.get(), read_len})); } - - return Status::OK(); + return local_writer->close(); } } // namespace doris::io From f5e2cd9ac5b5f8f049f6c66a57ea0d4791ea81ce Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Fri, 12 Apr 2024 11:37:50 +0800 Subject: [PATCH 20/71] [chore](Nereids) better way to return error (#33507) --- fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 15d8c22af5ed7d..3aab767d284912 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 @@ -601,7 +601,8 @@ public void execute(TUniqueId queryId) throws Exception { if (e instanceof NereidsException && !context.getSessionVariable().enableFallbackToOriginalPlanner && !forceFallback) { LOG.warn("Analyze failed. {}", context.getQueryIdentifier(), e); - throw ((NereidsException) e).getException(); + context.getState().setError(e.getMessage()); + return; } if (LOG.isDebugEnabled()) { LOG.debug("fall back to legacy planner on statement:\n{}", originStmt.originStmt); From d10585d10cace0dac21bcb50cfbf04519958de6b Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Fri, 12 Apr 2024 12:03:47 +0800 Subject: [PATCH 21/71] [feature](profile) add transaction statistics for profile (#33488) 1. commit total time 2. fs operator total time rename file count rename dir count delete dir count 3. add partition total time add partition count 4. update partition total time update partition count like: ``` - Transaction Commit Time: 906ms - FileSystem Operator Time: 833ms - Rename File Count: 4 - Rename Dir Count: 0 - Delete Dir Count: 0 - HMS Add Partition Time: 0ms - HMS Add Partition Count: 0 - HMS Update Partition Time: 68ms - HMS Update Partition Count: 4 ``` --- .../apache/doris/common/profile/Profile.java | 2 +- .../doris/common/profile/SummaryProfile.java | 120 +++++++++++++++++- .../doris/datasource/hive/HMSTransaction.java | 92 ++++++++++++-- .../commands/insert/HiveInsertExecutor.java | 11 +- .../org/apache/doris/qe/StmtExecutor.java | 4 +- .../doris/transaction/TransactionType.java | 24 ++++ .../doris/datasource/hive/HmsCommitTest.java | 5 + 7 files changed, 241 insertions(+), 17 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionType.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/Profile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/Profile.java index 12ba687bfd1bcc..b9cefdd0c4aa04 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/Profile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/Profile.java @@ -89,7 +89,7 @@ public List getExecutionProfiles() { } // This API will also add the profile to ProfileManager, so that we could get the profile from ProfileManager. - // isFinished ONLY means the cooridnator or stmtexecutor is finished. + // isFinished ONLY means the coordinator or stmtexecutor is finished. public synchronized void updateSummary(long startTime, Map summaryInfo, boolean isFinished, Planner planner) { try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java index b7bc5b4728e2d6..b3e8898ed40a05 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java @@ -21,6 +21,7 @@ import org.apache.doris.common.util.RuntimeProfile; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.thrift.TUnit; +import org.apache.doris.transaction.TransactionType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -89,10 +90,19 @@ public class SummaryProfile { public static final String FRAGMENT_COMPRESSED_SIZE = "Fragment Compressed Size"; public static final String FRAGMENT_RPC_COUNT = "Fragment RPC Count"; + public static final String TRANSACTION_COMMIT_TIME = "Transaction Commit Time"; + public static final String FILESYSTEM_OPT_TIME = "FileSystem Operator Time"; + public static final String FILESYSTEM_OPT_RENAME_FILE_CNT = "Rename File Count"; + public static final String FILESYSTEM_OPT_RENAME_DIR_CNT = "Rename Dir Count"; + public static final String FILESYSTEM_OPT_DELETE_DIR_CNT = "Delete Dir Count"; + public static final String HMS_ADD_PARTITION_TIME = "HMS Add Partition Time"; + public static final String HMS_ADD_PARTITION_CNT = "HMS Add Partition Count"; + public static final String HMS_UPDATE_PARTITION_TIME = "HMS Update Partition Time"; + public static final String HMS_UPDATE_PARTITION_CNT = "HMS Update Partition Count"; // These info will display on FE's web ui table, every one will be displayed as // a column, so that should not - // add many columns here. Add to ExcecutionSummary list. + // add many columns here. Add to ExecutionSummary list. public static final ImmutableList SUMMARY_KEYS = ImmutableList.of(PROFILE_ID, TASK_TYPE, START_TIME, END_TIME, TOTAL_TIME, TASK_STATE, USER, DEFAULT_DB, SQL_STATEMENT); @@ -111,7 +121,8 @@ public class SummaryProfile { QUERY_DISTRIBUTED_TIME, INIT_SCAN_NODE_TIME, FINALIZE_SCAN_NODE_TIME, - GET_SPLITS_TIME, GET_PARTITIONS_TIME, + GET_SPLITS_TIME, + GET_PARTITIONS_TIME, GET_PARTITION_FILES_TIME, CREATE_SCAN_RANGE_TIME, GET_PARTITION_VERSION_TIME, @@ -136,7 +147,9 @@ public class SummaryProfile { TOTAL_INSTANCES_NUM, INSTANCES_NUM_PER_BE, PARALLEL_FRAGMENT_EXEC_INSTANCE, - TRACE_ID); + TRACE_ID, + TRANSACTION_COMMIT_TIME + ); // Ident of each item. Default is 0, which doesn't need to present in this Map. // Please set this map for new profile items if they need ident. @@ -162,6 +175,14 @@ public class SummaryProfile { .put(SEND_FRAGMENT_PHASE2_TIME, 1) .put(FRAGMENT_COMPRESSED_SIZE, 1) .put(FRAGMENT_RPC_COUNT, 1) + .put(FILESYSTEM_OPT_TIME, 1) + .put(FILESYSTEM_OPT_RENAME_FILE_CNT, 2) + .put(FILESYSTEM_OPT_RENAME_DIR_CNT, 2) + .put(FILESYSTEM_OPT_DELETE_DIR_CNT, 2) + .put(HMS_ADD_PARTITION_TIME, 1) + .put(HMS_ADD_PARTITION_CNT, 2) + .put(HMS_UPDATE_PARTITION_TIME, 1) + .put(HMS_UPDATE_PARTITION_CNT, 2) .build(); private RuntimeProfile summaryProfile; @@ -212,6 +233,17 @@ public class SummaryProfile { private long getPartitionVersionByHasDataCount = 0; private long getTableVersionTime = 0; private long getTableVersionCount = 0; + private long transactionCommitBeginTime = -1; + private long transactionCommitEndTime = -1; + private long filesystemOptTime = -1; + private long hmsAddPartitionTime = -1; + private long hmsAddPartitionCnt = 0; + private long hmsUpdatePartitionTime = -1; + private long hmsUpdatePartitionCnt = 0; + private long filesystemRenameFileCnt = 0; + private long filesystemRenameDirCnt = 0; + private long filesystemDeleteDirCnt = 0; + private TransactionType transactionType = TransactionType.UNKNOWN; public SummaryProfile() { summaryProfile = new RuntimeProfile(SUMMARY_PROFILE_NAME); @@ -317,6 +349,7 @@ private void updateExecutionSummaryProfile() { RuntimeProfile.printCounter(queryFetchResultConsumeTime, TUnit.TIME_MS)); executionSummaryProfile.addInfoString(WRITE_RESULT_TIME, RuntimeProfile.printCounter(queryWriteResultConsumeTime, TUnit.TIME_MS)); + setTransactionSummary(); if (Config.isCloudMode()) { executionSummaryProfile.addInfoString(GET_PARTITION_VERSION_TIME, getPrettyGetPartitionVersionTime()); @@ -328,6 +361,31 @@ private void updateExecutionSummaryProfile() { } } + public void setTransactionSummary() { + executionSummaryProfile.addInfoString(TRANSACTION_COMMIT_TIME, + getPrettyTime(transactionCommitEndTime, transactionCommitBeginTime, TUnit.TIME_MS)); + + if (transactionType.equals(TransactionType.HMS)) { + executionSummaryProfile.addInfoString(FILESYSTEM_OPT_TIME, + getPrettyTime(filesystemOptTime, 0, TUnit.TIME_MS)); + executionSummaryProfile.addInfoString(FILESYSTEM_OPT_RENAME_FILE_CNT, + getPrettyCount(filesystemRenameFileCnt)); + executionSummaryProfile.addInfoString(FILESYSTEM_OPT_RENAME_DIR_CNT, + getPrettyCount(filesystemRenameDirCnt)); + executionSummaryProfile.addInfoString(FILESYSTEM_OPT_DELETE_DIR_CNT, + getPrettyCount(filesystemDeleteDirCnt)); + + executionSummaryProfile.addInfoString(HMS_ADD_PARTITION_TIME, + getPrettyTime(hmsAddPartitionTime, 0, TUnit.TIME_MS)); + executionSummaryProfile.addInfoString(HMS_ADD_PARTITION_CNT, + getPrettyCount(hmsAddPartitionCnt)); + executionSummaryProfile.addInfoString(HMS_UPDATE_PARTITION_TIME, + getPrettyTime(hmsUpdatePartitionTime, 0, TUnit.TIME_MS)); + executionSummaryProfile.addInfoString(HMS_UPDATE_PARTITION_CNT, + getPrettyCount(hmsUpdatePartitionCnt)); + } + } + public void setParseSqlStartTime(long parseSqlStartTime) { this.parseSqlStartTime = parseSqlStartTime; } @@ -607,6 +665,10 @@ private String getPrettyGetPartitionVersionCount() { return RuntimeProfile.printCounter(getPartitionVersionCount, TUnit.UNIT); } + private String getPrettyCount(long cnt) { + return RuntimeProfile.printCounter(cnt, TUnit.UNIT); + } + private String getPrettyGetTableVersionTime() { if (getTableVersionTime == 0) { return "N/A"; @@ -624,4 +686,56 @@ private String getPrettyTime(long end, long start, TUnit unit) { } return RuntimeProfile.printCounter(end - start, unit); } + + public void setTransactionBeginTime(TransactionType type) { + this.transactionCommitBeginTime = TimeUtils.getStartTimeMs(); + this.transactionType = type; + } + + public void setTransactionEndTime() { + this.transactionCommitEndTime = TimeUtils.getStartTimeMs(); + } + + public void freshFilesystemOptTime() { + if (this.filesystemOptTime == -1) { + // Because this value needs to be summed up. + // If it is not set zero here: + // 1. If the detection time is longer than 1ms, + // the final cumulative value will be 1 ms less due to -1 initialization. + // 2. if the detection time is no longer than 1ms, + // the final cumulative value will be -1 always. + // This is considered to be the indicator's not being detected, + // Apparently not, it's just that the value detected is 0. + this.filesystemOptTime = 0; + } + this.filesystemOptTime += System.currentTimeMillis() - tempStarTime; + } + + public void setHmsAddPartitionTime() { + this.hmsAddPartitionTime = TimeUtils.getStartTimeMs() - tempStarTime; + } + + public void addHmsAddPartitionCnt(long c) { + this.hmsAddPartitionCnt = c; + } + + public void setHmsUpdatePartitionTime() { + this.hmsUpdatePartitionTime = TimeUtils.getStartTimeMs() - tempStarTime; + } + + public void addHmsUpdatePartitionCnt(long c) { + this.hmsUpdatePartitionCnt = c; + } + + public void addRenameFileCnt(long c) { + this.filesystemRenameFileCnt += c; + } + + public void incRenameDirCnt() { + this.filesystemRenameDirCnt += 1; + } + + public void incDeleteDirRecursiveCnt() { + this.filesystemDeleteDirCnt += 1; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java index 84221b74e7f1e5..0e668e0eda5195 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java @@ -23,8 +23,10 @@ import org.apache.doris.backup.Status; import org.apache.doris.common.Pair; +import org.apache.doris.common.profile.SummaryProfile; import org.apache.doris.fs.FileSystem; import org.apache.doris.fs.remote.RemoteFile; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.THivePartitionUpdate; import org.apache.doris.thrift.TUpdateMode; import org.apache.doris.transaction.Transaction; @@ -68,6 +70,7 @@ public class HMSTransaction implements Transaction { private final FileSystem fs; private String dbName; private String tbName; + private Optional summaryProfile = Optional.empty(); private final Map> tableActions = new HashMap<>(); private final Map, Action>> @@ -79,6 +82,10 @@ public class HMSTransaction implements Transaction { public HMSTransaction(HiveMetadataOps hiveOps) { this.hiveOps = hiveOps; this.fs = hiveOps.getFs(); + + if (ConnectContext.get().getExecutor() != null) { + summaryProfile = Optional.of(ConnectContext.get().getExecutor().getSummaryProfile()); + } } @Override @@ -597,7 +604,7 @@ private DeleteRecursivelyResult doRecursiveDeleteFiles(Path directory, boolean d } public boolean deleteIfExists(Path path) { - Status status = fs.delete(path.toString()); + Status status = wrapperDeleteWithProfileSummary(path.toString()); if (status.ok()) { return true; } @@ -1057,7 +1064,7 @@ public void prepareInsertExistingTable(TableAndMore tableAndMore) { String targetPath = table.getSd().getLocation(); String writePath = tableAndMore.getCurrentLocation(); if (!targetPath.equals(writePath)) { - fs.asyncRename( + wrapperAsyncRenameWithProfileSummary( fileSystemExecutor, asyncFileSystemTaskFutures, fileSystemTaskCancelled, @@ -1083,7 +1090,7 @@ public void prepareAlterTable(TableAndMore tableAndMore) { if (!targetPath.equals(writePath)) { Path path = new Path(targetPath); String oldTablePath = new Path(path.getParent(), "_temp_" + path.getName()).toString(); - Status status = fs.renameDir( + Status status = wrapperRenameDirWithProfileSummary( targetPath, oldTablePath, () -> renameDirectoryTasksForAbort.add(new RenameDirectoryTask(oldTablePath, targetPath))); @@ -1093,7 +1100,7 @@ public void prepareAlterTable(TableAndMore tableAndMore) { } clearDirsForFinish.add(oldTablePath); - status = fs.renameDir( + status = wrapperRenameDirWithProfileSummary( writePath, targetPath, () -> directoryCleanUpTasksForAbort.add( @@ -1120,7 +1127,7 @@ public void prepareAddPartition(PartitionAndMore partitionAndMore) { String writePath = partitionAndMore.getCurrentLocation(); if (!targetPath.equals(writePath)) { - fs.asyncRenameDir( + wrapperAsyncRenameDirWithProfileSummary( fileSystemExecutor, asyncFileSystemTaskFutures, fileSystemTaskCancelled, @@ -1160,7 +1167,7 @@ public void prepareInsertExistPartition(PartitionAndMore partitionAndMore) { directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false)); if (!targetPath.equals(writePath)) { - fs.asyncRename( + wrapperAsyncRenameWithProfileSummary( fileSystemExecutor, asyncFileSystemTaskFutures, fileSystemTaskCancelled, @@ -1189,7 +1196,7 @@ private void runRenameDirTasksForAbort() { for (RenameDirectoryTask task : renameDirectoryTasksForAbort) { status = fs.exists(task.getRenameFrom()); if (status.ok()) { - status = fs.renameDir(task.getRenameFrom(), task.getRenameTo(), () -> {}); + status = wrapperRenameDirWithProfileSummary(task.getRenameFrom(), task.getRenameTo(), () -> {}); if (!status.ok()) { LOG.warn("Failed to abort rename dir from {} to {}:{}", task.getRenameFrom(), task.getRenameTo(), status.getErrMsg()); @@ -1201,7 +1208,7 @@ private void runRenameDirTasksForAbort() { private void runClearPathsForFinish() { Status status; for (String path : clearDirsForFinish) { - status = fs.delete(path); + status = wrapperDeleteWithProfileSummary(path); if (!status.ok()) { LOG.warn("Failed to recursively delete path {}:{}", path, status.getErrCode()); } @@ -1216,7 +1223,7 @@ public void prepareAlterPartition(PartitionAndMore partitionAndMore) { if (!targetPath.equals(writePath)) { Path path = new Path(targetPath); String oldPartitionPath = new Path(path.getParent(), "_temp_" + path.getName()).toString(); - Status status = fs.renameDir( + Status status = wrapperRenameDirWithProfileSummary( targetPath, oldPartitionPath, () -> renameDirectoryTasksForAbort.add(new RenameDirectoryTask(oldPartitionPath, targetPath))); @@ -1228,7 +1235,7 @@ public void prepareAlterPartition(PartitionAndMore partitionAndMore) { } clearDirsForFinish.add(oldPartitionPath); - status = fs.renameDir( + status = wrapperRenameDirWithProfileSummary( writePath, targetPath, () -> directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true))); @@ -1250,18 +1257,35 @@ public void prepareAlterPartition(PartitionAndMore partitionAndMore) { private void waitForAsyncFileSystemTasks() { + summaryProfile.ifPresent(SummaryProfile::setTempStartTime); + for (CompletableFuture future : asyncFileSystemTaskFutures) { MoreFutures.getFutureValue(future, RuntimeException.class); } + + summaryProfile.ifPresent(SummaryProfile::freshFilesystemOptTime); } private void doAddPartitionsTask() { + + summaryProfile.ifPresent(profile -> { + profile.setTempStartTime(); + profile.addHmsAddPartitionCnt(addPartitionsTask.getPartitions().size()); + }); + if (!addPartitionsTask.isEmpty()) { addPartitionsTask.run(hiveOps); } + + summaryProfile.ifPresent(SummaryProfile::setHmsAddPartitionTime); } private void doUpdateStatisticsTasks() { + summaryProfile.ifPresent(profile -> { + profile.setTempStartTime(); + profile.addHmsUpdatePartitionCnt(updateStatisticsTasks.size()); + }); + ImmutableList.Builder> updateStatsFutures = ImmutableList.builder(); List failedTaskDescriptions = new ArrayList<>(); List suppressedExceptions = new ArrayList<>(); @@ -1289,6 +1313,8 @@ private void doUpdateStatisticsTasks() { suppressedExceptions.forEach(exception::addSuppressed); throw exception; } + + summaryProfile.ifPresent(SummaryProfile::setHmsUpdatePartitionTime); } public void doNothing() { @@ -1312,4 +1338,50 @@ public void rollback() { runRenameDirTasksForAbort(); } } + + public Status wrapperRenameDirWithProfileSummary(String origFilePath, + String destFilePath, + Runnable runWhenPathNotExist) { + summaryProfile.ifPresent(profile -> { + profile.setTempStartTime(); + profile.incRenameDirCnt(); + }); + + Status status = fs.renameDir(origFilePath, destFilePath, runWhenPathNotExist); + + summaryProfile.ifPresent(SummaryProfile::freshFilesystemOptTime); + return status; + } + + public Status wrapperDeleteWithProfileSummary(String remotePath) { + summaryProfile.ifPresent(profile -> { + profile.setTempStartTime(); + profile.incDeleteDirRecursiveCnt(); + }); + + Status status = fs.delete(remotePath); + + summaryProfile.ifPresent(SummaryProfile::freshFilesystemOptTime); + return status; + } + + public void wrapperAsyncRenameWithProfileSummary(Executor executor, + List> renameFileFutures, + AtomicBoolean cancelled, + String origFilePath, + String destFilePath, + List fileNames) { + fs.asyncRename(executor, renameFileFutures, cancelled, origFilePath, destFilePath, fileNames); + summaryProfile.ifPresent(profile -> profile.addRenameFileCnt(fileNames.size())); + } + + public void wrapperAsyncRenameDirWithProfileSummary(Executor executor, + List> renameFileFutures, + AtomicBoolean cancelled, + String origFilePath, + String destFilePath, + Runnable runWhenPathNotExist) { + fs.asyncRenameDir(executor, renameFileFutures, cancelled, origFilePath, destFilePath, runWhenPathNotExist); + summaryProfile.ifPresent(SummaryProfile::incRenameDirCnt); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java index 66dfe763e469be..116a04215d8eb3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.UserException; +import org.apache.doris.common.profile.SummaryProfile; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSTransaction; @@ -36,6 +37,7 @@ import org.apache.doris.qe.StmtExecutor; import org.apache.doris.transaction.TransactionManager; import org.apache.doris.transaction.TransactionStatus; +import org.apache.doris.transaction.TransactionType; import com.google.common.base.Strings; import org.apache.logging.log4j.LogManager; @@ -53,6 +55,7 @@ public class HiveInsertExecutor extends AbstractInsertExecutor { private TransactionStatus txnStatus = TransactionStatus.ABORTED; private final TransactionManager transactionManager; private final String catalogName; + private Optional summaryProfile = Optional.empty(); /** * constructor @@ -63,6 +66,10 @@ public HiveInsertExecutor(ConnectContext ctx, HMSExternalTable table, super(ctx, table, labelName, planner, insertCtx); catalogName = table.getCatalog().getName(); transactionManager = table.getCatalog().getTransactionManager(); + + if (ConnectContext.get().getExecutor() != null) { + summaryProfile = Optional.of(ConnectContext.get().getExecutor().getSummaryProfile()); + } } public long getTxnId() { @@ -102,7 +109,9 @@ protected void onComplete() throws UserException { String dbName = ((HMSExternalTable) table).getDbName(); String tbName = table.getName(); transaction.finishInsertTable(dbName, tbName); + summaryProfile.ifPresent(profile -> profile.setTransactionBeginTime(TransactionType.HMS)); transactionManager.commit(txnId); + summaryProfile.ifPresent(SummaryProfile::setTransactionEndTime); txnStatus = TransactionStatus.COMMITTED; Env.getCurrentEnv().getCatalogMgr().refreshExternalTable( dbName, @@ -135,7 +144,7 @@ protected void afterExec(StmtExecutor executor) { sb.append("{"); sb.append("'status':'") .append(ctx.isTxnModel() ? TransactionStatus.PREPARE.name() : txnStatus.name()); - // sb.append("', 'txnId':'").append(txnId).append("'"); + sb.append("', 'txnId':'").append(txnId).append("'"); if (!Strings.isNullOrEmpty(errMsg)) { sb.append(", 'err':'").append(errMsg).append("'"); } 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 3aab767d284912..d28d73c58e6e5e 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 @@ -1142,13 +1142,13 @@ public void updateProfile(boolean isFinished) { if (!context.getSessionVariable().enableProfile()) { return; } - // If any error happends in update profile, we should ignore this error + // If any error happened in update profile, we should ignore this error // and ensure the sql is finished normally. For example, if update profile // failed, the insert stmt should be success try { profile.updateSummary(context.startTime, getSummaryInfo(isFinished), isFinished, this.planner); } catch (Throwable t) { - LOG.warn("failed to update profile, ingore this error", t); + LOG.warn("failed to update profile, ignore this error", t); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionType.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionType.java new file mode 100644 index 00000000000000..2372c199738116 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/TransactionType.java @@ -0,0 +1,24 @@ +// 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.transaction; + +public enum TransactionType { + UNKNOWN, + HMS, + ICEBERG +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java index fba91cb0b55642..54bbf5eca3f12b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.datasource.HMSCachedClientTest; import org.apache.doris.fs.LocalDfsFileSystem; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.THiveLocationParams; import org.apache.doris.thrift.THivePartitionUpdate; import org.apache.doris.thrift.TUpdateMode; @@ -69,6 +70,10 @@ public static void beforeClass() throws Throwable { writeLocation = "file://" + writePath.toAbsolutePath() + "/"; createTestHiveCatalog(); createTestHiveDatabase(); + + // context + ConnectContext connectContext = new ConnectContext(); + connectContext.setThreadLocalInfo(); } @AfterClass From ff137833461eef7cb74c9a1585e8c4976aef9c55 Mon Sep 17 00:00:00 2001 From: feiniaofeiafei <53502832+feiniaofeiafei@users.noreply.github.com> Date: Fri, 12 Apr 2024 12:25:13 +0800 Subject: [PATCH 22/71] [feat](nereids) support create view in nereids (#32743) --- .../org/apache/doris/nereids/DorisParser.g4 | 3 + .../apache/doris/analysis/BaseViewStmt.java | 2 - .../apache/doris/analysis/CreateViewStmt.java | 9 + .../doris/nereids/StatementContext.java | 14 + .../nereids/analyzer/UnboundRelation.java | 31 +- .../doris/nereids/analyzer/UnboundStar.java | 16 +- .../nereids/parser/LogicalPlanBuilder.java | 51 ++- .../doris/nereids/parser/NereidsParser.java | 15 +- .../rules/analysis/BindExpression.java | 14 + .../nereids/rules/analysis/BindRelation.java | 8 + .../trees/expressions/SlotReference.java | 5 + .../trees/expressions/WindowFrame.java | 28 +- .../expressions/functions/scalar/Lambda.java | 2 +- .../doris/nereids/trees/plans/PlanType.java | 3 +- .../plans/commands/CreateViewCommand.java | 49 +++ .../plans/commands/info/CreateViewInfo.java | 329 ++++++++++++++++++ .../commands/info/SimpleColumnDefinition.java | 6 + .../trees/plans/visitor/CommandVisitor.java | 5 + .../org/apache/doris/nereids/util/Utils.java | 17 + .../data/ddl_p0/test_create_view_nereids.out | 228 ++++++++++++ ...st_show_create_table_and_views_nereids.out | 46 +++ .../suites/ddl_p0/test_create_view.groovy | 5 +- .../ddl_p0/test_create_view_nereids.groovy | 280 +++++++++++++++ .../test_show_create_table_and_views.groovy | 2 + ...show_create_table_and_views_nereids.groovy | 128 +++++++ 25 files changed, 1272 insertions(+), 24 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateViewCommand.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java create mode 100644 regression-test/data/ddl_p0/test_create_view_nereids.out create mode 100644 regression-test/data/show_p0/test_show_create_table_and_views_nereids.out create mode 100644 regression-test/suites/ddl_p0/test_create_view_nereids.groovy create mode 100644 regression-test/suites/show_p0/test_show_create_table_and_views_nereids.groovy 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 22fe3355fffc7f..754c68ee4aee47 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 @@ -60,6 +60,9 @@ statementBase properties=propertyClause? (BROKER extProperties=propertyClause)? (AS query)? #createTable + | CREATE VIEW (IF NOT EXISTS)? name=multipartIdentifier + (LEFT_PAREN cols=simpleColumnDefs RIGHT_PAREN)? + (COMMENT STRING_LITERAL)? AS query #createView | explain? INSERT (INTO | OVERWRITE TABLE) (tableName=multipartIdentifier | DORIS_INTERNAL_TABLE_ID LEFT_PAREN tableId=INTEGER_VALUE RIGHT_PAREN) partitionSpec? // partition define diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java index 8114448f0d4e15..d8740f03f523b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java @@ -25,7 +25,6 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.ToSqlContext; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; @@ -50,7 +49,6 @@ public class BaseViewStmt extends DdlStmt { protected QueryStmt cloneStmt; public BaseViewStmt(TableName tableName, List cols, QueryStmt queryStmt) { - Preconditions.checkNotNull(queryStmt); this.tableName = tableName; this.cols = cols; this.viewDefStmt = queryStmt; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateViewStmt.java index 5791c3c430c04a..8b53d18fd9b602 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateViewStmt.java @@ -17,6 +17,7 @@ package org.apache.doris.analysis; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; @@ -93,4 +94,12 @@ public void analyze(Analyzer analyzer) throws UserException { } } } + + public void setInlineViewDef(String querySql) { + inlineViewDef = querySql; + } + + public void setFinalColumns(List columns) { + finalCols.addAll(columns); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 7b444995120cab..403c605ba75992 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -54,6 +54,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeMap; import javax.annotation.concurrent.GuardedBy; /** @@ -121,6 +122,11 @@ public class StatementContext { private BitSet disableRules; + // for create view support in nereids + // key is the start and end position of the sql substring that needs to be replaced, + // and value is the new string used for replacement. + private TreeMap, String> indexInSqlToString = new TreeMap<>(new Pair.PairComparator<>()); + public StatementContext() { this.connectContext = ConnectContext.get(); } @@ -354,4 +360,12 @@ public boolean isHasUnknownColStats() { public void setHasUnknownColStats(boolean hasUnknownColStats) { this.hasUnknownColStats = hasUnknownColStats; } + + public TreeMap, String> getIndexInSqlToString() { + return indexInSqlToString; + } + + public void addIndexInSqlToString(Pair pair, String replacement) { + indexInSqlToString.put(pair, replacement); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java index 4514ea05bfb9c9..34217b249fc510 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.analyzer; import org.apache.doris.analysis.TableScanParams; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; @@ -55,35 +56,44 @@ public class UnboundRelation extends LogicalRelation implements Unbound, BlockFu private final Optional tableSample; private final Optional indexName; private TableScanParams scanParams; + // the start and end position of the sql substring(e.g. "t1", "db1.t1", "ctl1.db1.t1") + private final Optional> indexInSqlString; public UnboundRelation(RelationId id, List nameParts) { this(id, nameParts, Optional.empty(), Optional.empty(), ImmutableList.of(), false, ImmutableList.of(), - ImmutableList.of(), Optional.empty(), Optional.empty(), null); + ImmutableList.of(), Optional.empty(), Optional.empty(), null, Optional.empty()); } public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart) { this(id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, ImmutableList.of(), - ImmutableList.of(), Optional.empty(), Optional.empty(), null); + ImmutableList.of(), Optional.empty(), Optional.empty(), null, Optional.empty()); } public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart, List tabletIds, List hints, Optional tableSample, Optional indexName) { this(id, nameParts, Optional.empty(), Optional.empty(), - partNames, isTempPart, tabletIds, hints, tableSample, indexName, null); + partNames, isTempPart, tabletIds, hints, tableSample, indexName, null, Optional.empty()); } public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart, List tabletIds, List hints, Optional tableSample, Optional indexName, TableScanParams scanParams) { this(id, nameParts, Optional.empty(), Optional.empty(), - partNames, isTempPart, tabletIds, hints, tableSample, indexName, scanParams); + partNames, isTempPart, tabletIds, hints, tableSample, indexName, scanParams, Optional.empty()); } public UnboundRelation(RelationId id, List nameParts, Optional groupExpression, Optional logicalProperties, List partNames, boolean isTempPart, List tabletIds, List hints, Optional tableSample, Optional indexName) { this(id, nameParts, groupExpression, logicalProperties, partNames, - isTempPart, tabletIds, hints, tableSample, indexName, null); + isTempPart, tabletIds, hints, tableSample, indexName, null, Optional.empty()); + } + + public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart, + List tabletIds, List hints, Optional tableSample, Optional indexName, + TableScanParams scanParams, Optional> indexInSqlString) { + this(id, nameParts, Optional.empty(), Optional.empty(), + partNames, isTempPart, tabletIds, hints, tableSample, indexName, scanParams, indexInSqlString); } /** @@ -92,7 +102,7 @@ public UnboundRelation(RelationId id, List nameParts, Optional nameParts, Optional groupExpression, Optional logicalProperties, List partNames, boolean isTempPart, List tabletIds, List hints, Optional tableSample, Optional indexName, - TableScanParams scanParams) { + TableScanParams scanParams, Optional> indexInSqlString) { super(id, PlanType.LOGICAL_UNBOUND_RELATION, groupExpression, logicalProperties); this.nameParts = ImmutableList.copyOf(Objects.requireNonNull(nameParts, "nameParts should not null")); this.partNames = ImmutableList.copyOf(Objects.requireNonNull(partNames, "partNames should not null")); @@ -102,6 +112,7 @@ public UnboundRelation(RelationId id, List nameParts, Optional getNameParts() { @@ -122,14 +133,14 @@ public LogicalProperties computeLogicalProperties() { public Plan withGroupExpression(Optional groupExpression) { return new UnboundRelation(relationId, nameParts, groupExpression, Optional.of(getLogicalProperties()), - partNames, isTempPart, tabletIds, hints, tableSample, indexName, null); + partNames, isTempPart, tabletIds, hints, tableSample, indexName, null, indexInSqlString); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { return new UnboundRelation(relationId, nameParts, groupExpression, logicalProperties, partNames, - isTempPart, tabletIds, hints, tableSample, indexName, null); + isTempPart, tabletIds, hints, tableSample, indexName, null, indexInSqlString); } @Override @@ -187,4 +198,8 @@ public Optional getTableSample() { public TableScanParams getScanParams() { return scanParams; } + + public Optional> getIndexInSqlString() { + return indexInSqlString; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundStar.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundStar.java index 43705611562b71..cf73f877d604eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundStar.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundStar.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.analyzer; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; @@ -28,17 +29,26 @@ import java.util.List; import java.util.Objects; +import java.util.Optional; /** * Star expression. */ public class UnboundStar extends NamedExpression implements LeafExpression, Unbound, PropagateNullable { - private final List qualifier; + // the start and end position of the sql substring(e.g. "*", "table.*") + private final Optional> indexInSqlString; public UnboundStar(List qualifier) { super(ImmutableList.of()); this.qualifier = Objects.requireNonNull(ImmutableList.copyOf(qualifier), "qualifier can not be null"); + this.indexInSqlString = Optional.empty(); + } + + public UnboundStar(List qualifier, Optional> indexInSqlString) { + super(ImmutableList.of()); + this.qualifier = Objects.requireNonNull(ImmutableList.copyOf(qualifier), "qualifier can not be null"); + this.indexInSqlString = indexInSqlString; } @Override @@ -71,6 +81,10 @@ public boolean equals(Object o) { return qualifier.equals(that.qualifier); } + public Optional> getIndexInSqlString() { + return indexInSqlString; + } + @Override public int hashCode() { return Objects.hash(super.hashCode(), qualifier); 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 5f0268317602c8..8dc75102923957 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 @@ -77,6 +77,7 @@ import org.apache.doris.nereids.DorisParser.CreateProcedureContext; import org.apache.doris.nereids.DorisParser.CreateRowPolicyContext; import org.apache.doris.nereids.DorisParser.CreateTableContext; +import org.apache.doris.nereids.DorisParser.CreateViewContext; import org.apache.doris.nereids.DorisParser.CteContext; import org.apache.doris.nereids.DorisParser.DataTypeWithNullableContext; import org.apache.doris.nereids.DorisParser.DateCeilContext; @@ -360,6 +361,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateViewCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; @@ -386,6 +388,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; import org.apache.doris.nereids.trees.plans.commands.info.CreateMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; +import org.apache.doris.nereids.trees.plans.commands.info.CreateViewInfo; import org.apache.doris.nereids.trees.plans.commands.info.DMLCommandType; import org.apache.doris.nereids.trees.plans.commands.info.DefaultValue; import org.apache.doris.nereids.trees.plans.commands.info.DistributionDescriptor; @@ -472,6 +475,15 @@ */ @SuppressWarnings({"OptionalUsedAsFieldOrParameterType", "OptionalGetWithoutIsPresent"}) public class LogicalPlanBuilder extends DorisParserBaseVisitor { + private final boolean forCreateView; + + public LogicalPlanBuilder() { + forCreateView = false; + } + + public LogicalPlanBuilder(boolean forCreateView) { + this.forCreateView = forCreateView; + } @SuppressWarnings("unchecked") protected T typedVisit(ParseTree ctx) { @@ -1329,11 +1341,16 @@ public LogicalPlan visitTableName(TableNameContext ctx) { scanParams = new TableScanParams(ctx.optScanParams().funcName.getText(), map); } + MultipartIdentifierContext identifier = ctx.multipartIdentifier(); TableSample tableSample = ctx.sample() == null ? null : (TableSample) visit(ctx.sample()); - LogicalPlan checkedRelation = LogicalPlanBuilderAssistant.withCheckPolicy( + UnboundRelation relation = forCreateView ? new UnboundRelation(StatementScopeIdGenerator.newRelationId(), + tableId, partitionNames, isTempPart, tabletIdLists, relationHints, + Optional.ofNullable(tableSample), indexName, scanParams, + Optional.of(Pair.of(identifier.start.getStartIndex(), identifier.stop.getStopIndex()))) : new UnboundRelation(StatementScopeIdGenerator.newRelationId(), tableId, partitionNames, isTempPart, tabletIdLists, relationHints, - Optional.ofNullable(tableSample), indexName, scanParams)); + Optional.ofNullable(tableSample), indexName, scanParams); + LogicalPlan checkedRelation = LogicalPlanBuilderAssistant.withCheckPolicy(relation); LogicalPlan plan = withTableAlias(checkedRelation, ctx.tableAlias()); for (LateralViewContext lateralViewContext : ctx.lateralView()) { plan = withGenerate(plan, lateralViewContext); @@ -1382,7 +1399,9 @@ public Expression visitStar(StarContext ctx) { } else { target = ImmutableList.of(); } - return new UnboundStar(target); + return forCreateView + ? new UnboundStar(target, Optional.of(Pair.of(ctx.start.getStartIndex(), ctx.stop.getStopIndex()))) + : new UnboundStar(target); }); } @@ -2413,6 +2432,19 @@ private LogicalPlan plan(ParserRuleContext tree) { * create table parsing * ******************************************************************************************** */ + @Override + public LogicalPlan visitCreateView(CreateViewContext ctx) { + List nameParts = visitMultipartIdentifier(ctx.name); + String comment = ctx.STRING_LITERAL() == null ? "" : LogicalPlanBuilderAssistant.escapeBackSlash( + ctx.STRING_LITERAL().getText().substring(1, ctx.STRING_LITERAL().getText().length() - 1)); + LogicalPlan logicalPlan = visitQuery(ctx.query()); + String querySql = getOriginSql(ctx.query()); + CreateViewInfo info = new CreateViewInfo(ctx.EXISTS() != null, new TableNameInfo(nameParts), + comment, logicalPlan, querySql, + ctx.cols == null ? Lists.newArrayList() : visitSimpleColumnDefs(ctx.cols)); + return new CreateViewCommand(info); + } + @Override public LogicalPlan visitCreateTable(CreateTableContext ctx) { String ctlName = null; @@ -2860,8 +2892,11 @@ private LogicalPlan withSelectQuerySpecification( if (!expressions.stream().allMatch(UnboundSlot.class::isInstance)) { throw new ParseException("only column name is supported in except clause", selectColumnCtx); } - project = new LogicalProject<>(ImmutableList.of(new UnboundStar(ImmutableList.of())), - expressions, isDistinct, aggregate); + UnboundStar star = forCreateView ? new UnboundStar(ImmutableList.of(), + Optional.of(Pair.of(selectColumnCtx.start.getStartIndex(), + selectColumnCtx.stop.getStopIndex()))) + : new UnboundStar(ImmutableList.of()); + project = new LogicalProject<>(ImmutableList.of(star), expressions, isDistinct, aggregate); } else { List projects = getNamedExpressions(selectColumnCtx.namedExpressionSeq()); project = new LogicalProject<>(projects, ImmutableList.of(), isDistinct, aggregate); @@ -3040,8 +3075,10 @@ private LogicalPlan withProjection(LogicalPlan input, SelectColumnClauseContext if (!expressions.stream().allMatch(UnboundSlot.class::isInstance)) { throw new ParseException("only column name is supported in except clause", selectCtx); } - return new LogicalProject<>(ImmutableList.of(new UnboundStar(ImmutableList.of())), - expressions, isDistinct, input); + UnboundStar star = forCreateView ? new UnboundStar(ImmutableList.of(), + Optional.of(Pair.of(selectCtx.start.getStartIndex(), selectCtx.stop.getStopIndex()))) : + new UnboundStar(ImmutableList.of()); + return new LogicalProject<>(ImmutableList.of(star), expressions, isDistinct, input); } else { List projects = getNamedExpressions(selectCtx.namedExpressionSeq()); return new LogicalProject<>(projects, Collections.emptyList(), isDistinct, input); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java index 0ee3f5d068f9ce..da6881a4905468 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java @@ -164,7 +164,20 @@ private T parse(String sql, @Nullable LogicalPlanBuilder logicalPlanBuilder, return (T) realLogicalPlanBuilder.visit(tree); } - private ParserRuleContext toAst(String sql, Function parseFunction) { + public LogicalPlan parseForCreateView(String sql) { + return parseForCreateViewInternal(sql, null, DorisParser::singleStatement); + } + + private T parseForCreateViewInternal(String sql, @Nullable LogicalPlanBuilder logicalPlanBuilder, + Function parseFunction) { + ParserRuleContext tree = toAst(sql, parseFunction); + LogicalPlanBuilder realLogicalPlanBuilder = logicalPlanBuilder == null + ? new LogicalPlanBuilder(true) : logicalPlanBuilder; + return (T) realLogicalPlanBuilder.visit(tree); + } + + /** toAst */ + public static ParserRuleContext toAst(String sql, Function parseFunction) { DorisLexer lexer = new DorisLexer(new CaseInsensitiveStream(CharStreams.fromString(sql))); CommonTokenStream tokenStream = new CommonTokenStream(lexer); DorisParser parser = new DorisParser(tokenStream); 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 1d0fd3ed632da0..e20d3e8d551852 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 @@ -28,6 +28,7 @@ import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.analyzer.UnboundStar; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.parser.LogicalPlanBuilder; @@ -571,6 +572,7 @@ private Plan bindProject(MatchingContext> ctx) { () -> analyzer.analyzeToSet(project.getExcepts())); Builder boundProjections = ImmutableList.builderWithExpectedSize(project.arity()); + StatementContext statementContext = ctx.statementContext; for (Expression expression : project.getProjects()) { Expression expr = analyzer.analyze(expression); if (!(expr instanceof BoundStar)) { @@ -582,6 +584,13 @@ private Plan bindProject(MatchingContext> ctx) { slots = Utils.filterImmutableList(slots, slot -> !boundExcepts.get().contains(slot)); } boundProjections.addAll(slots); + + // for create view stmt expand star + List slotsForLambda = slots; + UnboundStar unboundStar = (UnboundStar) expression; + unboundStar.getIndexInSqlString().ifPresent(pair -> + statementContext.addIndexInSqlToString(pair, toSqlWithBackquote(slotsForLambda)) + ); } } return project.withProjects(boundProjections.build()); @@ -975,6 +984,11 @@ private interface CustomSlotBinderAnalyzer { List bindSlot(ExpressionAnalyzer analyzer, UnboundSlot unboundSlot); } + public String toSqlWithBackquote(List slots) { + return slots.stream().map(slot -> ((SlotReference) slot).getQualifiedNameWithBackquote()) + .collect(Collectors.joining(", ")); + } + private boolean hasAggregateFunction(Expression expression, FunctionRegistry functionRegistry) { return expression.anyMatch(expr -> { if (expr instanceof AggregateFunction) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 2e47cc05fbea18..fcbdcbd64e60b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -31,6 +31,7 @@ import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.nereids.CTEContext; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.analyzer.UnboundResultSink; @@ -62,6 +63,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalTestScan; import org.apache.doris.nereids.trees.plans.logical.LogicalView; import org.apache.doris.nereids.util.RelationUtil; +import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.ConnectContext; import com.google.common.base.Preconditions; @@ -239,6 +241,12 @@ private LogicalPlan makeOlapScan(TableIf table, UnboundRelation unboundRelation, private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelation, List tableQualifier, CascadesContext cascadesContext) { + // for create view stmt replace tablename to ctl.db.tablename + unboundRelation.getIndexInSqlString().ifPresent(pair -> { + StatementContext statementContext = cascadesContext.getStatementContext(); + statementContext.addIndexInSqlToString(pair, + Utils.qualifiedNameWithBackquote(tableQualifier)); + }); List qualifierWithoutTableName = Lists.newArrayList(); qualifierWithoutTableName.addAll(tableQualifier.subList(0, tableQualifier.size() - 1)); switch (table.getType()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java index 760836455127ea..3d11a7d011da15 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.TableIf; +import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.trees.plans.algebra.Relation; import org.apache.doris.nereids.types.DataType; @@ -296,4 +297,8 @@ private static Supplier> buildInternalName( internalName.isPresent() ? internalName : Optional.of(name.get())); } } + + public String getQualifiedNameWithBackquote() throws UnboundException { + return Utils.qualifiedNameWithBackquote(getQualifier(), getName()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowFrame.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowFrame.java index 90449ed0f86bad..5cbb93ce3748ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowFrame.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowFrame.java @@ -99,7 +99,7 @@ public String toSql() { StringBuilder sb = new StringBuilder(); sb.append(frameUnits + " "); if (rightBoundary != null) { - sb.append("BETWEEN " + leftBoundary + " AND " + rightBoundary); + sb.append("BETWEEN " + leftBoundary.toSql() + " AND " + rightBoundary.toSql()); } else { sb.append(leftBoundary); } @@ -215,6 +215,32 @@ public String toString() { return sb.toString(); } + /** toSql*/ + public String toSql() { + StringBuilder sb = new StringBuilder(); + boundOffset.ifPresent(value -> sb.append(value + " ")); + switch (frameBoundType) { + case UNBOUNDED_PRECEDING: + sb.append("UNBOUNDED PRECEDING"); + break; + case UNBOUNDED_FOLLOWING: + sb.append("UNBOUNDED FOLLOWING"); + break; + case CURRENT_ROW: + sb.append("CURRENT ROW"); + break; + case PRECEDING: + sb.append("PRECEDING"); + break; + case FOLLOWING: + sb.append("FOLLOWING"); + break; + default: + break; + } + return sb.toString(); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Lambda.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Lambda.java index 1d6c725db09d90..e8261f6391dda9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Lambda.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Lambda.java @@ -132,7 +132,7 @@ public String toSql() { if (argumentNames.size() > 1) { argStr = argumentNames.stream().collect(Collectors.joining(", ", "(", ")")); } - builder.append(String.format("%s -> %s", argStr, getLambdaFunction().toString())); + builder.append(String.format("%s -> %s", argStr, getLambdaFunction().toSql())); for (int i = 1; i < getArguments().size(); i++) { builder.append(", ").append(getArgument(i).toSql()); } 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 1c493deae033cb..daab18438b4774 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 @@ -148,5 +148,6 @@ public enum PlanType { CREATE_PROCEDURE_COMMAND, DROP_PROCEDURE_COMMAND, SHOW_PROCEDURE_COMMAND, - SHOW_CREATE_PROCEDURE_COMMAND + SHOW_CREATE_PROCEDURE_COMMAND, + CREATE_VIEW_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateViewCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateViewCommand.java new file mode 100644 index 00000000000000..d78308664b5e38 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateViewCommand.java @@ -0,0 +1,49 @@ +// 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.commands; + +import org.apache.doris.analysis.CreateViewStmt; +import org.apache.doris.catalog.Env; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.info.CreateViewInfo; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; + +/** CreateViewCommand */ +public class CreateViewCommand extends Command { + private final CreateViewInfo createViewInfo; + + public CreateViewCommand(CreateViewInfo createViewInfo) { + super(PlanType.CREATE_VIEW_COMMAND); + this.createViewInfo = createViewInfo; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + createViewInfo.init(ctx); + createViewInfo.validate(ctx); + CreateViewStmt createViewStmt = createViewInfo.translateToLegacyStmt(ctx); + Env.getCurrentEnv().createView(createViewStmt); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitCreateViewCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java new file mode 100644 index 00000000000000..632e884685652f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java @@ -0,0 +1,329 @@ +// 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.commands.info; + +import org.apache.doris.analysis.ColWithComment; +import org.apache.doris.analysis.CreateViewStmt; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeNameFormat; +import org.apache.doris.common.Pair; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.Util; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.DorisParser; +import org.apache.doris.nereids.DorisParser.NamedExpressionContext; +import org.apache.doris.nereids.DorisParser.NamedExpressionSeqContext; +import org.apache.doris.nereids.DorisParserBaseVisitor; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.analyzer.UnboundResultSink; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.jobs.executor.AbstractBatchJobExecutor; +import org.apache.doris.nereids.jobs.rewrite.RewriteJob; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.analysis.AnalyzeCTE; +import org.apache.doris.nereids.rules.analysis.BindExpression; +import org.apache.doris.nereids.rules.analysis.BindRelation; +import org.apache.doris.nereids.rules.analysis.CheckPolicy; +import org.apache.doris.nereids.rules.analysis.EliminateLogicalSelectHint; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.tree.ParseTree; +import org.antlr.v4.runtime.tree.RuleNode; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * CreateViewInfo + */ +public class CreateViewInfo { + private final boolean ifNotExists; + private final TableNameInfo viewName; + private final String comment; + private final LogicalPlan logicalQuery; + private final String querySql; + private final List simpleColumnDefinitions; + private final List finalCols = Lists.newArrayList(); + private Plan analyzedPlan; + + /** constructor*/ + public CreateViewInfo(boolean ifNotExists, TableNameInfo viewName, String comment, LogicalPlan logicalQuery, + String querySql, List simpleColumnDefinitions) { + this.ifNotExists = ifNotExists; + this.viewName = viewName; + this.comment = comment; + if (logicalQuery instanceof LogicalFileSink) { + throw new AnalysisException("Not support OUTFILE clause in CREATE VIEW statement"); + } + this.logicalQuery = logicalQuery; + this.querySql = querySql; + this.simpleColumnDefinitions = simpleColumnDefinitions; + } + + /** init */ + public void init(ConnectContext ctx) throws UserException { + analyzeAndFillRewriteSqlMap(querySql, ctx); + OutermostPlanFinderContext outermostPlanFinderContext = new OutermostPlanFinderContext(); + analyzedPlan.accept(OutermostPlanFinder.INSTANCE, outermostPlanFinderContext); + List outputs = outermostPlanFinderContext.outermostPlan.getOutput(); + createFinalCols(outputs); + } + + /**validate*/ + public void validate(ConnectContext ctx) throws UserException { + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + planner.plan(new UnboundResultSink<>(logicalQuery), PhysicalProperties.ANY, ExplainLevel.NONE); + viewName.analyze(ctx); + FeNameFormat.checkTableName(viewName.getTbl()); + // disallow external catalog + Util.prohibitExternalCatalog(viewName.getCtl(), "CreateViewStmt"); + // check privilege + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), viewName.getDb(), + viewName.getTbl(), PrivPredicate.CREATE)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "CREATE"); + } + Set colSets = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + for (Column col : finalCols) { + if (!colSets.add(col.getName())) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DUP_FIELDNAME, col.getName()); + } + } + } + + /**translateToLegacyStmt*/ + public CreateViewStmt translateToLegacyStmt(ConnectContext ctx) { + List cols = Lists.newArrayList(); + for (SimpleColumnDefinition def : simpleColumnDefinitions) { + cols.add(def.translateToColWithComment()); + } + CreateViewStmt createViewStmt = new CreateViewStmt(ifNotExists, viewName.transferToTableName(), cols, comment, + null); + // expand star(*) in project list and replace table name with qualifier + String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString()); + + // rewrite project alias + rewrittenSql = rewriteProjectsToUserDefineAlias(rewrittenSql); + + createViewStmt.setInlineViewDef(rewrittenSql); + createViewStmt.setFinalColumns(finalCols); + return createViewStmt; + } + + private void analyzeAndFillRewriteSqlMap(String sql, ConnectContext ctx) { + StatementContext stmtCtx = ctx.getStatementContext(); + LogicalPlan parsedViewPlan = new NereidsParser().parseForCreateView(sql); + if (parsedViewPlan instanceof UnboundResultSink) { + parsedViewPlan = (LogicalPlan) ((UnboundResultSink) parsedViewPlan).child(); + } + CascadesContext viewContextForStar = CascadesContext.initContext( + stmtCtx, parsedViewPlan, PhysicalProperties.ANY); + AnalyzerForCreateView analyzerForStar = new AnalyzerForCreateView(viewContextForStar); + analyzerForStar.analyze(); + analyzedPlan = viewContextForStar.getRewritePlan(); + } + + private String rewriteSql(Map, String> indexStringSqlMap) { + StringBuilder builder = new StringBuilder(); + int beg = 0; + for (Map.Entry, String> entry : indexStringSqlMap.entrySet()) { + Pair index = entry.getKey(); + builder.append(querySql, beg, index.first); + builder.append(entry.getValue()); + beg = index.second + 1; + } + builder.append(querySql, beg, querySql.length()); + return builder.toString(); + } + + private String rewriteProjectsToUserDefineAlias(String resSql) { + IndexFinder finder = new IndexFinder(); + ParserRuleContext tree = NereidsParser.toAst(resSql, DorisParser::singleStatement); + finder.visit(tree); + if (simpleColumnDefinitions.isEmpty()) { + return resSql; + } + List namedExpressionContexts = finder.getNamedExpressionContexts(); + StringBuilder replaceWithColsBuilder = new StringBuilder(); + for (int i = 0; i < namedExpressionContexts.size(); ++i) { + NamedExpressionContext namedExpressionContext = namedExpressionContexts.get(i); + int start = namedExpressionContext.expression().start.getStartIndex(); + int stop = namedExpressionContext.expression().stop.getStopIndex(); + replaceWithColsBuilder.append(resSql, start, stop + 1); + replaceWithColsBuilder.append(" AS `"); + String escapeBacktick = finalCols.get(i).getName().replace("`", "``"); + replaceWithColsBuilder.append(escapeBacktick); + replaceWithColsBuilder.append('`'); + if (i != namedExpressionContexts.size() - 1) { + replaceWithColsBuilder.append(", "); + } + } + String replaceWithCols = replaceWithColsBuilder.toString(); + return StringUtils.overlay(resSql, replaceWithCols, finder.getIndex().first, + finder.getIndex().second + 1); + } + + private void createFinalCols(List outputs) throws org.apache.doris.common.AnalysisException { + if (simpleColumnDefinitions.isEmpty()) { + for (Slot output : outputs) { + Column column = new Column(output.getName(), output.getDataType().toCatalogDataType()); + finalCols.add(column); + } + } else { + if (outputs.size() != simpleColumnDefinitions.size()) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_VIEW_WRONG_LIST); + } + for (int i = 0; i < simpleColumnDefinitions.size(); ++i) { + Column column = new Column(simpleColumnDefinitions.get(i).getName(), + outputs.get(i).getDataType().toCatalogDataType()); + column.setComment(simpleColumnDefinitions.get(i).getComment()); + finalCols.add(column); + } + } + } + + private static class OutermostPlanFinderContext { + public Plan outermostPlan = null; + public boolean found = false; + } + + private static class OutermostPlanFinder extends DefaultPlanVisitor { + public static final OutermostPlanFinder INSTANCE = new OutermostPlanFinder(); + + @Override + public Void visit(Plan plan, OutermostPlanFinderContext ctx) { + if (ctx.found) { + return null; + } + ctx.outermostPlan = plan; + ctx.found = true; + return null; + } + + @Override + public Void visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, + OutermostPlanFinderContext ctx) { + if (ctx.found) { + return null; + } + return super.visit(cteAnchor, ctx); + } + + @Override + public Void visitLogicalCTEProducer(LogicalCTEProducer cteProducer, + OutermostPlanFinderContext ctx) { + return null; + } + } + + /** traverse ast to find the outermost project list location information in sql*/ + private static class IndexFinder extends DorisParserBaseVisitor { + private boolean found = false; + private int startIndex; + private int stopIndex; + private List namedExpressionContexts = Lists.newArrayList(); + + @Override + public Void visitChildren(RuleNode node) { + if (found) { + return null; + } + int n = node.getChildCount(); + for (int i = 0; i < n; ++i) { + ParseTree c = node.getChild(i); + c.accept(this); + } + return null; + } + + @Override + public Void visitCte(DorisParser.CteContext ctx) { + return null; + } + + @Override + public Void visitSelectColumnClause(DorisParser.SelectColumnClauseContext ctx) { + if (found) { + return null; + } + startIndex = ctx.getStart().getStartIndex(); + stopIndex = ctx.getStop().getStopIndex(); + found = true; + + NamedExpressionSeqContext namedExpressionSeqContext = ctx.namedExpressionSeq(); + namedExpressionContexts = namedExpressionSeqContext.namedExpression(); + return null; + } + + public Pair getIndex() { + return Pair.of(startIndex, stopIndex); + } + + public List getNamedExpressionContexts() { + return namedExpressionContexts; + } + } + + private static class AnalyzerForCreateView extends AbstractBatchJobExecutor { + private final List jobs; + + public AnalyzerForCreateView(CascadesContext cascadesContext) { + super(cascadesContext); + jobs = buildAnalyzeViewJobsForStar(); + } + + public void analyze() { + execute(); + } + + @Override + public List getJobs() { + return jobs; + } + + private static List buildAnalyzeViewJobsForStar() { + return jobs( + topDown(new EliminateLogicalSelectHint()), + topDown(new AnalyzeCTE()), + bottomUp( + new BindRelation(), + new CheckPolicy(), + new BindExpression() + ) + ); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/SimpleColumnDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/SimpleColumnDefinition.java index 51080bf45d2372..e4e7f2013cb341 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/SimpleColumnDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/SimpleColumnDefinition.java @@ -17,6 +17,8 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.analysis.ColWithComment; + /** * column def for mv */ @@ -39,4 +41,8 @@ public String getName() { public String getComment() { return comment; } + + public ColWithComment translateToColWithComment() { + return new ColWithComment(name, comment); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 6bcc41e39e45f9..527ccb1ffe080b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand; import org.apache.doris.nereids.trees.plans.commands.CreateProcedureCommand; import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateViewCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromCommand; import org.apache.doris.nereids.trees.plans.commands.DeleteFromUsingCommand; import org.apache.doris.nereids.trees.plans.commands.DropConstraintCommand; @@ -156,4 +157,8 @@ default R visitShowProcedureStatusCommand(ShowProcedureStatusCommand showProcedu default R visitShowCreateProcedureCommand(ShowCreateProcedureCommand showCreateProcedureCommand, C context) { return visitCommand(showCreateProcedureCommand, context); } + + default R visitCreateViewCommand(CreateViewCommand createViewCommand, C context) { + return visitCommand(createViewCommand, context); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java index c28b18e697d34c..852e148ef1d9cb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java @@ -128,6 +128,23 @@ public static String qualifiedName(List qualifier, String name) { return StringUtils.join(qualifiedNameParts(qualifier, name), "."); } + /** get qualified name with Backtick */ + public static String qualifiedNameWithBackquote(List qualifiers, String name) { + List fullName = new ArrayList<>(qualifiers); + fullName.add(name); + return qualifiedNameWithBackquote(fullName); + } + + /** get qualified name with Backtick */ + public static String qualifiedNameWithBackquote(List qualifiers) { + List qualifierWithBackquote = Lists.newArrayListWithCapacity(qualifiers.size()); + for (String qualifier : qualifiers) { + String escapeQualifier = qualifier.replace("`", "``"); + qualifierWithBackquote.add('`' + escapeQualifier + '`'); + } + return StringUtils.join(qualifierWithBackquote, "."); + } + /** * Get sql string for plan. * diff --git a/regression-test/data/ddl_p0/test_create_view_nereids.out b/regression-test/data/ddl_p0/test_create_view_nereids.out new file mode 100644 index 00000000000000..1fda4f63f57f79 --- /dev/null +++ b/regression-test/data/ddl_p0/test_create_view_nereids.out @@ -0,0 +1,228 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !test_view_1 -- +1 [1, 2, 3] +2 [10, -2, 8] +3 [-1, 20, 0] + +-- !test_view_2 -- +1 [1, 2, 3] [1, 1, 1] +2 [10, -2, 8] [1, 0, 1] +3 [-1, 20, 0] [0, 1, 0] + +-- !test_view_3 -- +1 [1, 2, 3] [1, 2, 3] [1, 2, 3] +2 [10, -2, 8] [10, 8] [10, 8] +3 [-1, 20, 0] [20] [20] + +-- !test_view_4 -- +1 [1, 2, 3] [1, 2, 3] [1, 2, 3] +2 [10, -2, 8] [10, 8] [10, 8] +3 [-1, 20, 0] [20] [20] + +-- !test_view_5 -- +1 [1, 2, 3] [1, 1, 1] +2 [10, -2, 8] [1, 0, 1] +3 [-1, 20, 0] [0, 1, 0] + +-- !test_view_6 -- +v1 CREATE VIEW `v1` COMMENT 'VIEW' AS SELECT\n error_code, \n 1, \n 'string', \n now(), \n dayofyear(op_time), \n cast (source AS BIGINT), \n min(`timestamp`) OVER (\n ORDER BY \n op_time DESC ROWS BETWEEN UNBOUNDED PRECEDING\n AND 1 FOLLOWING\n ), \n 1 > 2,\n 2 + 3,\n 1 IN (1, 2, 3, 4), \n remark LIKE '%like', \n CASE WHEN remark = 's' THEN 1 ELSE 2 END,\n TRUE | FALSE \n FROM \n `internal`.`regression_test_ddl_p0`.`view_column_name_test_nereids`; + +-- !test_with_as -- +1 1 2 +1 1 4 +1 3 6 +2 1 3 +2 1 4 +2 1 7 +2 3 5 +2 3 9 +2 4 2 +3 2 8 +3 5 \N +3 5 6 +3 5 6 +3 5 8 +4 5 6 +6 \N 6 +6 7 1 + +-- !test_with_as_sql -- +test_view_with_as CREATE VIEW `test_view_with_as` COMMENT 'VIEW' AS (\n with t1 as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), t2 as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`), \n t3 as (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`) SELECT `t1`.`pk`, `t1`.`a`, `t1`.`b` FROM t1); utf8mb4 utf8mb4_0900_bin + +-- !test_union -- +1 1 2 +1 1 2 +1 1 4 +1 1 4 +1 3 6 +1 3 6 +2 1 3 +2 1 3 +2 1 4 +2 1 4 +2 1 7 +2 1 7 +2 3 5 +2 3 5 +2 3 9 +2 3 9 +2 4 2 +2 4 2 +3 2 8 +3 2 8 +3 5 \N +3 5 \N +3 5 6 +3 5 6 +3 5 6 +3 5 6 +3 5 8 +3 5 8 +4 5 6 +4 5 6 +6 \N 6 +6 \N 6 +6 7 1 +6 7 1 + +-- !test_union_sql -- +test_view_union CREATE VIEW `test_view_union` COMMENT 'VIEW' AS (select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a` AS `c2`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` AS `c3` from `internal`.`regression_test_ddl_p0`.`mal_test_view` Union all SELECT `internal`.`regression_test_ddl_p0`.`mal_test_view`.`pk`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` FROM `internal`.`regression_test_ddl_p0`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin + +-- !test_count_star -- +17 + +-- !test_count_star_sql -- +test_view_count_star CREATE VIEW `test_view_count_star` COMMENT 'VIEW' AS (select count(*) AS `c1` from `internal`.`regression_test_ddl_p0`.`mal_test_view` having count(*) > 0); utf8mb4 utf8mb4_0900_bin + +-- !test_expression -- +\N \N 6 +2 4 2 +2 4 3 +2 4 4 +2 4 4 +2 4 7 +3 5 8 +4 6 5 +4 6 6 +4 6 9 +5 7 2 +6 8 \N +6 8 6 +6 8 6 +6 8 6 +6 8 8 +8 10 1 + +-- !test_expression_sql -- +test_view_expression CREATE VIEW `test_view_expression` COMMENT 'VIEW' AS (select a+1 AS `c1`, abs(a)+2+1 AS `c2`, cast(b as varchar(10)) AS `c3` from `internal`.`regression_test_ddl_p0`.`mal_test_view`); utf8mb4 utf8mb4_0900_bin + +-- !test_alias -- +\N \N 6 +2 4 2 +2 4 3 +2 4 4 +2 4 4 +2 4 7 +3 5 8 +4 6 5 +4 6 6 +4 6 9 +5 7 2 +6 8 \N +6 8 6 +6 8 6 +6 8 6 +6 8 8 +8 10 1 + +-- !test_alias_sql -- +test_view_alias CREATE VIEW `test_view_alias` COMMENT 'VIEW' AS (\n select c8 AS `c1`, c2 AS `c2`, c1 AS `c3` from (select a+1 c8,abs(a)+2+1 as c2, cast(b as varchar(10)) as c1 from `internal`.`regression_test_ddl_p0`.`mal_test_view`) t); utf8mb4 utf8mb4_0900_bin + +-- !test_star_except -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_star_except_sql -- +test_view_star_except CREATE VIEW `test_view_star_except` COMMENT 'VIEW' AS select `internal`.`regression_test_ddl_p0`.`mal_test_view`.`a`, `internal`.`regression_test_ddl_p0`.`mal_test_view`.`b` from `internal`.`regression_test_ddl_p0`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin + +-- !test_create_view_from_view -- +1 1 2 +1 1 4 +1 3 6 +2 1 3 +2 1 4 +2 1 7 +2 3 5 +2 3 9 +2 4 2 +3 2 8 +3 5 \N +3 5 6 +3 5 6 +3 5 8 +4 5 6 +6 \N 6 +6 7 1 + +-- !test_create_view_from_view_sql -- +test_view_from_view CREATE VIEW `test_view_from_view` COMMENT 'VIEW' AS select `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`pk` AS `c1`, `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`a` AS `c2`, `internal`.`regression_test_ddl_p0`.`test_view_with_as`.`b` AS `c3` from `internal`.`regression_test_ddl_p0`.`test_view_with_as`; utf8mb4 utf8mb4_0900_bin + +-- !test_backquote_in_view_define -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_backquote_in_view_define_sql -- +test_backquote_in_view_define CREATE VIEW `test_backquote_in_view_define` COMMENT 'VIEW' AS select a AS `ab``c`, b AS `c2` from `internal`.`regression_test_ddl_p0`.`mal_test_view`; utf8mb4 utf8mb4_0900_bin + +-- !test_backquote_in_table_alias -- +\N 6 +1 2 +1 3 +1 4 +1 4 +1 7 +2 8 +3 5 +3 6 +3 9 +4 2 +5 \N +5 6 +5 6 +5 6 +5 8 +7 1 + +-- !test_backquote_in_table_alias_sql -- +test_backquote_in_table_alias CREATE VIEW `test_backquote_in_table_alias` COMMENT 'VIEW' AS select `ab``c`.`a` AS `c1`, `ab``c`.`b` AS `c2` from (select a,b from `internal`.`regression_test_ddl_p0`.`mal_test_view`) `ab``c`; utf8mb4 utf8mb4_0900_bin + diff --git a/regression-test/data/show_p0/test_show_create_table_and_views_nereids.out b/regression-test/data/show_p0/test_show_create_table_and_views_nereids.out new file mode 100644 index 00000000000000..3a5728d3d9e50a --- /dev/null +++ b/regression-test/data/show_p0/test_show_create_table_and_views_nereids.out @@ -0,0 +1,46 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !show -- +show_create_table_and_views_nereids_table CREATE TABLE `show_create_table_and_views_nereids_table` (\n `user_id` LARGEINT NOT NULL,\n `good_id` LARGEINT NOT NULL,\n `cost` BIGINT SUM NULL DEFAULT "0"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nCOMMENT 'OLAP'\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V1",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); + +-- !select -- +1 1 30 +1 2 5 +1 3 10 +1 300 2 +2 1 100 +2 2 10 +2 3 44 +2 200 1111 +3 1 10 +3 2 1 +23 900 1 +100 100 1 +200 20 1 +300 20 1 + +-- !select -- +1 5 +2 10 +3 1 + +-- !show -- +show_create_table_and_views_nereids_view CREATE VIEW `show_create_table_and_views_nereids_view` COMMENT 'VIEW' AS SELECT user_id AS `user_id`, cost AS `cost` FROM `internal`.`show_create_table_and_views_nereids_db`.`show_create_table_and_views_nereids_table`\n WHERE good_id = 2; utf8mb4 utf8mb4_0900_bin + +-- !select -- +1 47 +2 1265 +3 11 +23 1 +100 1 +200 1 +300 1 + +-- !show -- +show_create_table_and_views_nereids_table CREATE TABLE `show_create_table_and_views_nereids_table` (\n `user_id` LARGEINT NOT NULL,\n `good_id` LARGEINT NOT NULL,\n `cost` BIGINT SUM NULL DEFAULT "0"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nCOMMENT 'OLAP'\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V1",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); + +-- !show -- +show_create_table_and_views_nereids_like CREATE TABLE `show_create_table_and_views_nereids_like` (\n `user_id` LARGEINT NOT NULL,\n `good_id` LARGEINT NOT NULL,\n `cost` BIGINT SUM NULL DEFAULT "0"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nCOMMENT 'OLAP'\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V1",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); + +-- !show -- +show_create_table_and_views_nereids_like_with_rollup CREATE TABLE `show_create_table_and_views_nereids_like_with_rollup` (\n `user_id` LARGEINT NOT NULL,\n `good_id` LARGEINT NOT NULL,\n `cost` BIGINT SUM NULL DEFAULT "0"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nCOMMENT 'OLAP'\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V1",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); + diff --git a/regression-test/suites/ddl_p0/test_create_view.groovy b/regression-test/suites/ddl_p0/test_create_view.groovy index 6020aa310509dd..e66798e50bbb19 100644 --- a/regression-test/suites/ddl_p0/test_create_view.groovy +++ b/regression-test/suites/ddl_p0/test_create_view.groovy @@ -16,7 +16,7 @@ // under the License. suite("test_create_view") { - + sql "SET enable_nereids_planner=false;" sql """DROP TABLE IF EXISTS count_distinct""" sql """ CREATE TABLE IF NOT EXISTS count_distinct @@ -42,6 +42,7 @@ suite("test_create_view") { "dynamic_partition.buckets" = "3" ); """ + sql "drop view if exists test_count_distinct" sql """ CREATE VIEW IF NOT EXISTS test_count_distinct ( @@ -203,4 +204,4 @@ suite("test_create_view") { view_column_name_test """ qt_test_view_6 """ SHOW VIEW FROM view_column_name_test;""" -} +} \ No newline at end of file diff --git a/regression-test/suites/ddl_p0/test_create_view_nereids.groovy b/regression-test/suites/ddl_p0/test_create_view_nereids.groovy new file mode 100644 index 00000000000000..48885a47fd157a --- /dev/null +++ b/regression-test/suites/ddl_p0/test_create_view_nereids.groovy @@ -0,0 +1,280 @@ +// 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_create_view_nereids") { + sql "SET enable_nereids_planner=true;" + sql "SET enable_fallback_to_original_planner=false;" + sql """DROP TABLE IF EXISTS count_distinct_nereids""" + sql """ + CREATE TABLE IF NOT EXISTS count_distinct_nereids + ( + RQ DATE NOT NULL COMMENT "日期", + v1 VARCHAR(100) NOT NULL COMMENT "字段1", + v2 VARCHAR(100) NOT NULL COMMENT "字段2", + v3 VARCHAR(100) REPLACE_IF_NOT_NULL COMMENT "字段3" + ) + AGGREGATE KEY(RQ,v1,v2) + PARTITION BY RANGE(RQ) + ( + PARTITION p20220908 VALUES LESS THAN ('2022-09-09') + ) + DISTRIBUTED BY HASH(v1,v2) BUCKETS 3 + PROPERTIES( + "replication_num" = "1", + "dynamic_partition.enable" = "true", + "dynamic_partition.time_unit" = "DAY", + "dynamic_partition.start" = "-3", + "dynamic_partition.end" = "3", + "dynamic_partition.prefix" = "p", + "dynamic_partition.buckets" = "3" + ); + """ + sql """ + CREATE VIEW IF NOT EXISTS test_count_distinct_nereids + ( + RQ comment "日期", + v1 comment "v1", + v2 comment "v2", + v3 comment "v3" + ) + AS + select aa.RQ as RQ, aa.v1 as v1,aa.v2 as v2 , bb.v3 as v3 from + ( + select RQ, count(distinct v1) as v1 , count(distinct v2 ) as v2 + from count_distinct_nereids + group by RQ + ) aa + LEFT JOIN + ( + select RQ, max(v3) as v3 + from count_distinct_nereids + group by RQ + ) bb + on aa.RQ = bb.RQ; + """ + + sql """select * from test_count_distinct_nereids""" + sql """DROP VIEW IF EXISTS test_count_distinct_nereids""" + sql """DROP TABLE IF EXISTS count_distinct_nereids""" + + sql """DROP TABLE IF EXISTS test_view_t1""" + sql """ + CREATE TABLE `test_view_t1` ( + k1 int, + k2 date, + v1 int + ) ENGINE=OLAP + UNIQUE KEY(`k1`,`k2`) + COMMENT '测试' + PARTITION BY RANGE(k2) ( + PARTITION p1 VALUES [('2023-07-01'), ('2023-07-10')), + PARTITION p2 VALUES [('2023-07-11'), ('2023-07-20')) + ) + DISTRIBUTED BY HASH(`k1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + );""" + sql """DROP TABLE IF EXISTS test_view_t2""" + sql """ + CREATE TABLE `test_view_t2` ( + k1 int, + k2 date, + v1 int + ) ENGINE=OLAP + UNIQUE KEY(`k1`,`k2`) + COMMENT '测试' + PARTITION BY RANGE(k2) ( + PARTITION p1 VALUES [('2023-07-01'), ('2023-07-05')), + PARTITION p2 VALUES [('2023-07-05'), ('2023-07-15')) + ) + DISTRIBUTED BY HASH(`k1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); """ + sql """ + CREATE VIEW IF NOT EXISTS my_view_nereids AS + SELECT test_view_t1.* FROM test_view_t1 PARTITION(p1) JOIN test_view_t2 PARTITION(p2) ON test_view_t1.k1 = test_view_t2.k1; """ + sql """SELECT * FROM my_view_nereids""" + sql """DROP VIEW IF EXISTS my_view_nereids""" + sql """DROP TABLE IF EXISTS test_view_t1""" + sql """DROP TABLE IF EXISTS test_view_t2""" + + + sql """DROP TABLE IF EXISTS view_baseall_nereids""" + sql """DROP VIEW IF EXISTS test_view7_nereids""" + sql """DROP VIEW IF EXISTS test_view8""" + sql """ + CREATE TABLE `view_baseall_nereids` ( + `k1` int(11) NULL, + `k3` array NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 5 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "is_being_synced" = "false", + "storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false" + ); + """ + sql """insert into view_baseall_nereids values(1,[1,2,3]);""" + sql """insert into view_baseall_nereids values(2,[10,-2,8]);""" + sql """insert into view_baseall_nereids values(3,[-1,20,0]);""" + + qt_test_view_1 """ select * from view_baseall_nereids order by k1; """ + qt_test_view_2 """ select *, array_map(x->x>0,k3) from view_baseall_nereids order by k1; """ + qt_test_view_3 """ select *, array_filter(x->x>0,k3),array_filter(`k3`, array_map(x -> x > 0, `k3`)) from view_baseall_nereids order by k1; """ + + + sql """ + create view IF NOT EXISTS test_view7_nereids (k1,k2,k3,k4) as + select *, array_filter(x->x>0,k3),array_filter(`k3`, array_map(x -> x > 0, `k3`)) from view_baseall_nereids order by k1; + """ + qt_test_view_4 """ select * from test_view7_nereids order by k1; """ + + sql """ + create view IF NOT EXISTS test_view8_nereids (k1,k2,k3) as + select *, array_map(x->x>0,k3) from view_baseall_nereids order by k1; + """ + qt_test_view_5 """ select * from test_view8_nereids order by k1; """ + + sql """DROP TABLE IF EXISTS view_column_name_test_nereids""" + sql """ + CREATE TABLE IF NOT EXISTS view_column_name_test_nereids + ( + `timestamp` DATE NOT NULL COMMENT "['0000-01-01', '9999-12-31']", + `type` TINYINT NOT NULL COMMENT "[-128, 127]", + `error_code` INT COMMENT "[-2147483648, 2147483647]", + `error_msg` VARCHAR(300) COMMENT "[1-65533]", + `op_id` BIGINT COMMENT "[-9223372036854775808, 9223372036854775807]", + `op_time` DATETIME COMMENT "['0000-01-01 00:00:00', '9999-12-31 23:59:59']", + `target` float COMMENT "4 字节", + `source` double COMMENT "8 字节", + `lost_cost` decimal(12,2) COMMENT "", + `remark` string COMMENT "1m size", + `op_userid` LARGEINT COMMENT "[-2^127 + 1 ~ 2^127 - 1]", + `plate` SMALLINT COMMENT "[-32768, 32767]", + `iscompleted` boolean COMMENT "true 或者 false" + ) + DISTRIBUTED BY HASH(`type`) BUCKETS 1 + PROPERTIES ('replication_num' = '1'); + """ + + sql """ + DROP VIEW IF EXISTS v1 + """ + sql """ + CREATE VIEW v1 AS + SELECT + error_code, + 1, + 'string', + now(), + dayofyear(op_time), + cast (source AS BIGINT), + min(`timestamp`) OVER ( + ORDER BY + op_time DESC ROWS BETWEEN UNBOUNDED PRECEDING + AND 1 FOLLOWING + ), + 1 > 2, + 2 + 3, + 1 IN (1, 2, 3, 4), + remark LIKE '%like', + CASE WHEN remark = 's' THEN 1 ELSE 2 END, + TRUE | FALSE + FROM + view_column_name_test_nereids + """ + qt_test_view_6 """ SHOW VIEW FROM view_column_name_test_nereids;""" + + // test with as + sql """ + DROP TABLE IF EXISTS mal_test_view + """ + + sql """ + create table mal_test_view(pk int, a int, b int) distributed by hash(pk) buckets 10 + properties('replication_num' = '1'); + """ + + sql """ + insert into mal_test_view values(2,1,3),(1,1,2),(3,5,6),(6,null,6),(4,5,6),(2,1,4),(2,3,5),(1,1,4) + ,(3,5,6),(3,5,null),(6,7,1),(2,1,7),(2,4,2),(2,3,9),(1,3,6),(3,5,8),(3,2,8); + """ + sql "DROP VIEW if exists test_view_with_as" + sql """CREATE VIEW if not exists test_view_with_as AS ( + with t1 as (select * from mal_test_view), t2 as (select * from mal_test_view), + t3 as (select * from mal_test_view) SELECT * FROM t1);""" + qt_test_with_as "select * from test_view_with_as order by pk, a, b" + qt_test_with_as_sql "show create view test_view_with_as" + + // test union + sql "DROP VIEW if exists test_view_union" + sql """CREATE VIEW test_view_union(c1,c2,c3) AS + (select * from mal_test_view Union all SELECT * FROM mal_test_view);""" + qt_test_union "select c1,c2,c3 from test_view_union order by c1,c2,c3" + qt_test_union_sql "show create view test_view_union" + + // test count(*) + sql "drop view if exists test_view_count_star;" + sql "CREATE VIEW test_view_count_star(c1) AS (select count(*) from mal_test_view having count(*) > 0);" + qt_test_count_star "select c1 from test_view_count_star order by c1" + qt_test_count_star_sql "show create view test_view_count_star" + + // test expression + sql "drop view if exists test_view_expression;" + sql """CREATE VIEW test_view_expression(c1,c2,c3) AS (select a+1,abs(a)+2+1 as c2, cast(b as varchar(10)) as c1 from mal_test_view);""" + qt_test_expression "select * from test_view_expression order by c1,c2,c3" + qt_test_expression_sql "show create view test_view_expression;" + + // test alias + sql "drop view if exists test_view_alias;" + sql """CREATE VIEW test_view_alias(c1,c2,c3) AS ( + select c8 as c9, c2 as c3, c1 as c4 from (select a+1 c8,abs(a)+2+1 as c2, cast(b as varchar(10)) as c1 from mal_test_view) t);""" + qt_test_alias "select * from test_view_alias order by c1,c2,c3;" + qt_test_alias_sql "show create view test_view_alias;" + + // test * except + sql "drop view if exists test_view_star_except;" + sql """ + create view test_view_star_except as select * except(pk) from mal_test_view; + """ + qt_test_star_except "select * from test_view_star_except order by a, b;" + qt_test_star_except_sql "show create view test_view_star_except;" + + // test create view from view + sql "drop view if exists test_view_from_view" + sql "create view test_view_from_view(c1,c2,c3) as select * from test_view_with_as" + qt_test_create_view_from_view "select * from test_view_from_view order by c1,c2,c3" + qt_test_create_view_from_view_sql "show create view test_view_from_view" + + // test backquote in name + + sql "drop view if exists test_backquote_in_view_define;" + sql "create view test_backquote_in_view_define(`ab``c`, c2) as select a,b from mal_test_view;" + qt_test_backquote_in_view_define "select * from test_backquote_in_view_define order by `ab``c`, c2;" + qt_test_backquote_in_view_define_sql "show create view test_backquote_in_view_define;" + + sql "drop view if exists test_backquote_in_table_alias;" + sql "create view test_backquote_in_table_alias(c1, c2) as select * from (select a,b from mal_test_view) `ab``c`;" + qt_test_backquote_in_table_alias "select * from test_backquote_in_table_alias order by c1, c2;" + qt_test_backquote_in_table_alias_sql "show create view test_backquote_in_table_alias;" +} diff --git a/regression-test/suites/show_p0/test_show_create_table_and_views.groovy b/regression-test/suites/show_p0/test_show_create_table_and_views.groovy index 6c6322e5c4aa03..5872cc95d5ae5c 100644 --- a/regression-test/suites/show_p0/test_show_create_table_and_views.groovy +++ b/regression-test/suites/show_p0/test_show_create_table_and_views.groovy @@ -30,6 +30,7 @@ suite("test_show_create_table_and_views", "show") { String rollupName = "${suiteName}_rollup" String likeName = "${suiteName}_like" + sql "SET enable_nereids_planner=false;" sql "CREATE DATABASE IF NOT EXISTS ${dbName}" sql "DROP TABLE IF EXISTS ${dbName}.${tableName}" sql """ @@ -76,6 +77,7 @@ suite("test_show_create_table_and_views", "show") { qt_show "SHOW CREATE TABLE ${dbName}.${tableName}" qt_select "SELECT * FROM ${dbName}.${tableName} ORDER BY user_id, good_id" + sql "drop view if exists ${dbName}.${viewName};" // create view and show sql """ CREATE VIEW IF NOT EXISTS ${dbName}.${viewName} (user_id, cost) diff --git a/regression-test/suites/show_p0/test_show_create_table_and_views_nereids.groovy b/regression-test/suites/show_p0/test_show_create_table_and_views_nereids.groovy new file mode 100644 index 00000000000000..88fd00444adec6 --- /dev/null +++ b/regression-test/suites/show_p0/test_show_create_table_and_views_nereids.groovy @@ -0,0 +1,128 @@ +// 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_show_create_table_and_views_nereids", "show") { + def ret = sql "SHOW FRONTEND CONFIG like '%enable_feature_binlog%';" + logger.info("${ret}") + if (ret.size() != 0 && ret[0].size() > 1 && ret[0][1] == 'false') { + logger.info("enable_feature_binlog=false in frontend config, no need to run this case.") + return + } + + String suiteName = "show_create_table_and_views_nereids" + String dbName = "${suiteName}_db" + String tableName = "${suiteName}_table" + String viewName = "${suiteName}_view" + String rollupName = "${suiteName}_rollup" + String likeName = "${suiteName}_like" + + sql "SET enable_nereids_planner=true;" + sql "SET enable_fallback_to_original_planner=false;" + + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "DROP TABLE IF EXISTS ${dbName}.${tableName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `user_id` LARGEINT NOT NULL, + `good_id` LARGEINT NOT NULL, + `cost` BIGINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`user_id`, `good_id`) + PARTITION BY RANGE(`good_id`) + ( + PARTITION p1 VALUES LESS THAN ("100"), + PARTITION p2 VALUES LESS THAN ("200"), + PARTITION p3 VALUES LESS THAN ("300"), + PARTITION p4 VALUES LESS THAN ("400"), + PARTITION p5 VALUES LESS THAN ("500"), + PARTITION p6 VALUES LESS THAN ("600"), + PARTITION p7 VALUES LESS THAN MAXVALUE + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """INSERT INTO ${dbName}.${tableName} VALUES + (1, 1, 10), + (1, 1, 20), + (1, 2, 5), + (1, 3, 10), + (2, 1, 0), + (2, 1, 100), + (3, 1, 10), + (2, 2, 10), + (2, 3, 44), + (3, 2, 1), + (100, 100, 1), + (200, 20, 1), + (300, 20, 1), + (1, 300, 2), + (2, 200, 1111), + (23, 900, 1)""" + + qt_show "SHOW CREATE TABLE ${dbName}.${tableName}" + qt_select "SELECT * FROM ${dbName}.${tableName} ORDER BY user_id, good_id" + + sql "drop view if exists ${dbName}.${viewName};" + // create view and show + sql """ + CREATE VIEW IF NOT EXISTS ${dbName}.${viewName} (user_id, cost) + AS + SELECT user_id, cost FROM ${dbName}.${tableName} + WHERE good_id = 2 + """ + qt_select "SELECT * FROM ${dbName}.${viewName} ORDER BY user_id" + qt_show "SHOW CREATE VIEW ${dbName}.${viewName}" + + // create rollup + sql """ALTER TABLE ${dbName}.${tableName} + ADD ROLLUP ${rollupName} (user_id, cost) + """ + + def isAlterTableFinish = { -> + def records = sql """SHOW ALTER TABLE ROLLUP FROM ${dbName}""" + for (def row in records) { + if (row[5] == "${rollupName}" && row[8] == "FINISHED") { + return true + } + } + false + } + while (!isAlterTableFinish()) { + Thread.sleep(100) + } + + qt_select "SELECT user_id, SUM(cost) FROM ${dbName}.${tableName} GROUP BY user_id ORDER BY user_id" + qt_show "SHOW CREATE TABLE ${dbName}.${tableName}" + + // create like + sql "CREATE TABLE ${dbName}.${likeName} LIKE ${dbName}.${tableName}" + qt_show "SHOW CREATE TABLE ${dbName}.${likeName}" + + // create like with rollup + sql "CREATE TABLE ${dbName}.${likeName}_with_rollup LIKE ${dbName}.${tableName} WITH ROLLUP" + qt_show "SHOW CREATE TABLE ${dbName}.${likeName}_with_rollup" + + sql "DROP TABLE IF EXISTS ${dbName}.${likeName}_with_rollup FORCE" + sql "DROP TABLE ${dbName}.${likeName} FORCE" + sql "DROP VIEW ${dbName}.${viewName}" + sql "DROP TABLE ${dbName}.${tableName} FORCE" + sql "DROP DATABASE ${dbName} FORCE" +} + From e73ede42a32df10436693d441f4c89724cd5c877 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Fri, 12 Apr 2024 12:34:27 +0800 Subject: [PATCH 23/71] [opt](Nereids) support select async mv partition (#33560) --- .../apache/doris/nereids/rules/analysis/BindRelation.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index fcbdcbd64e60b9..c4175d96fd0dc0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -332,15 +332,15 @@ private List getPartitionIds(TableIf t, UnboundRelation unboundRelation) { if (CollectionUtils.isEmpty(parts)) { return ImmutableList.of(); } - if (!t.getType().equals(TableIf.TableType.OLAP)) { - throw new IllegalStateException(String.format( + if (!t.isManagedTable()) { + throw new AnalysisException(String.format( "Only OLAP table is support select by partition for now," + "Table: %s is not OLAP table", t.getName())); } return parts.stream().map(name -> { Partition part = ((OlapTable) t).getPartition(name, unboundRelation.isTempPart()); if (part == null) { - throw new IllegalStateException(String.format("Partition: %s is not exists", name)); + throw new AnalysisException(String.format("Partition: %s is not exists", name)); } return part.getId(); }).collect(ImmutableList.toImmutableList()); From 527a558fc471934e9445245092184516181bbc61 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Fri, 12 Apr 2024 12:34:45 +0800 Subject: [PATCH 24/71] [chore](test) update one join case for nereids because ambiguous name (#33562) --- regression-test/suites/query_p0/join/test_join.groovy | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/regression-test/suites/query_p0/join/test_join.groovy b/regression-test/suites/query_p0/join/test_join.groovy index 5f25ea52234a52..b3955d56f2751a 100644 --- a/regression-test/suites/query_p0/join/test_join.groovy +++ b/regression-test/suites/query_p0/join/test_join.groovy @@ -389,10 +389,10 @@ suite("test_join", "query,p0") { } sql"""select a.k1 k1, a.k2, a.k3, b.k1, b.k2, b.k3 from ${tbName1} a full outer join ${tbName2} b on a.k1 = b.k1 and a.k2 > b.k2 order by isnull(k1), 1, 2, 3, 4, 5 limit 65535""" - sql"""select a.k1 k1, a.k2, a.k3, b.k1, b.k2, b.k3 from ${tbName1} a left outer join ${tbName2} b - on a.k1 = b.k1 and a.k2 > b.k2 union (select a.k1, a.k2, a.k3, b.k1, b.k2, b.k3 + sql"""select a.k1 ak1, a.k2 ak2, a.k3 ak3, b.k1 bk1, b.k2 bk2, b.k3 bk3 from ${tbName1} a left outer join ${tbName2} b + on a.k1 = b.k1 and a.k2 > b.k2 union (select a.k1 ak1, a.k2 ak2, a.k3 ak3, b.k1 bk1, b.k2 bk2, b.k3 bk3 from ${tbName1} a right outer join ${tbName2} b on a.k1 = b.k1 and a.k2 > b.k2) - order by isnull(k1), 1, 2, 3, 4, 5 limit 65535""" + order by isnull(ak1), 1, 2, 3, 4, 5 limit 65535""" sql"""select count(*) from ${tbName1} a full outer join ${tbName2} b on a.k2 = b.k2 and a.k1 > 0 full outer join ${tbName3} c on a.k3 = c.k3 and b.k1 = c.k1 and c.k3 > 0""" sql"""select count(*) from ((select a.k1 as k1, b.k1 as k2, a.k2 as k3, b.k2 as k4, a.k3 as k5, b.k3 as k6, c.k1 as k7, c.k2 as k8, c.k3 as k9 from ${tbName1} a From e824895dccb7ee85b91ca63309139bd27f8811bf Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 12 Apr 2024 13:09:44 +0800 Subject: [PATCH 25/71] [Enhancement](hive-writer) Adjust table sink exchange rebalancer params. (#33397) Issue Number: #31442 Change table sink exchange rebalancer params to node level and adjust these params to improve write performance by better balance. rebalancer params: ``` DEFINE_mInt64(table_sink_partition_write_min_data_processed_rebalance_threshold, "26214400"); // 25MB // Minimum partition data processed to rebalance writers in exchange when partition writing DEFINE_mInt64(table_sink_partition_write_min_partition_data_processed_rebalance_threshold, "15728640"); // 15MB ``` --- be/src/common/config.cpp | 13 ++++++------ be/src/common/config.h | 8 ++++---- .../pipeline/exec/exchange_sink_operator.cpp | 20 ++++++++++++------- .../pipeline_x_fragment_context.cpp | 1 + be/src/runtime/runtime_state.h | 5 +++++ 5 files changed, 30 insertions(+), 17 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index f29ec307914eeb..212233dcbae6ee 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -1181,14 +1181,15 @@ DEFINE_mString(ca_cert_file_paths, "/etc/ssl/ca-bundle.pem"); /** Table sink configurations(currently contains only external table types) **/ -// Minimum data processed to scale writers when non partition writing +// Minimum data processed to scale writers in exchange when non partition writing DEFINE_mInt64(table_sink_non_partition_write_scaling_data_processed_threshold, - "125829120"); // 120MB -// Minimum data processed to start rebalancing in exchange when partition writing -DEFINE_mInt64(table_sink_partition_write_data_processed_threshold, "209715200"); // 200MB + "26214400"); // 25MB // Minimum data processed to trigger skewed partition rebalancing in exchange when partition writing -DEFINE_mInt64(table_sink_partition_write_skewed_data_processed_rebalance_threshold, - "209715200"); // 200MB +DEFINE_mInt64(table_sink_partition_write_min_data_processed_rebalance_threshold, + "26214400"); // 25MB +// Minimum partition data processed to rebalance writers in exchange when partition writing +DEFINE_mInt64(table_sink_partition_write_min_partition_data_processed_rebalance_threshold, + "15728640"); // 15MB // Maximum processed partition nums of per writer when partition writing DEFINE_mInt32(table_sink_partition_write_max_partition_nums_per_writer, "128"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 81fabfb9517879..b6b7e0c1d3660f 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -1265,12 +1265,12 @@ DECLARE_String(trino_connector_plugin_dir); DECLARE_mString(ca_cert_file_paths); /** Table sink configurations(currently contains only external table types) **/ -// Minimum data processed to scale writers when non partition writing +// Minimum data processed to scale writers in exchange when non partition writing DECLARE_mInt64(table_sink_non_partition_write_scaling_data_processed_threshold); -// Minimum data processed to start rebalancing in exchange when partition writing -DECLARE_mInt64(table_sink_partition_write_data_processed_threshold); // Minimum data processed to trigger skewed partition rebalancing in exchange when partition writing -DECLARE_mInt64(table_sink_partition_write_skewed_data_processed_rebalance_threshold); +DECLARE_mInt64(table_sink_partition_write_min_data_processed_rebalance_threshold); +// Minimum partition data processed to rebalance writers in exchange when partition writing +DECLARE_mInt64(table_sink_partition_write_min_partition_data_processed_rebalance_threshold); // Maximum processed partition nums of per writer when partition writing DECLARE_mInt32(table_sink_partition_write_max_partition_nums_per_writer); diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index bc55bc8f805803..8323e20cfd17c1 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -257,17 +257,23 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf _partitioner.reset( new vectorized::Crc32HashPartitioner(_partition_count)); _partition_function.reset(new HashPartitionFunction(_partitioner.get())); - // const long MEGABYTE = 1024 * 1024; - // const long MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD = 10000 * MEGABYTE; // 1MB - // const long MIN_DATA_PROCESSED_REBALANCE_THRESHOLD = 50000 * MEGABYTE; // 50MB - // const long MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD = 1; // 1MB - // const long MIN_DATA_PROCESSED_REBALANCE_THRESHOLD = 1; // 50MB scale_writer_partitioning_exchanger.reset(new vectorized::ScaleWriterPartitioningExchanger< HashPartitionFunction>( channels.size(), *_partition_function, _partition_count, channels.size(), 1, - config::table_sink_partition_write_data_processed_threshold, - config::table_sink_partition_write_skewed_data_processed_rebalance_threshold)); + config::table_sink_partition_write_min_partition_data_processed_rebalance_threshold / + state->task_num() == + 0 + ? config::table_sink_partition_write_min_partition_data_processed_rebalance_threshold + : config::table_sink_partition_write_min_partition_data_processed_rebalance_threshold / + state->task_num(), + config::table_sink_partition_write_min_data_processed_rebalance_threshold / + state->task_num() == + 0 + ? config::table_sink_partition_write_min_data_processed_rebalance_threshold + : config::table_sink_partition_write_min_data_processed_rebalance_threshold / + state->task_num())); + RETURN_IF_ERROR(_partitioner->init(p._texprs)); RETURN_IF_ERROR(_partitioner->prepare(state, p._row_desc)); _profile->add_info_string("Partitioner", diff --git a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp index ce2d4a507487eb..f23e39472ab993 100644 --- a/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp +++ b/be/src/pipeline/pipeline_x/pipeline_x_fragment_context.cpp @@ -593,6 +593,7 @@ Status PipelineXFragmentContext::_build_pipeline_tasks( init_runtime_state(task_runtime_state); auto cur_task_id = _total_tasks++; task_runtime_state->set_task_id(cur_task_id); + task_runtime_state->set_task_num(pipeline->num_tasks()); auto task = std::make_unique( pipeline, cur_task_id, get_task_runtime_state(cur_task_id), this, pipeline_id_to_profile[pip_idx].get(), get_local_exchange_state(pipeline), diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 07655c71b6cd16..644db3e32eb211 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -619,6 +619,10 @@ class RuntimeState { int task_id() const { return _task_id; } + void set_task_num(int task_num) { _task_num = task_num; } + + int task_num() const { return _task_num; } + private: Status create_error_log_file(); @@ -729,6 +733,7 @@ class RuntimeState { std::vector _error_tablet_infos; int _max_operator_id = 0; int _task_id = -1; + int _task_num = 0; std::vector _hive_partition_updates; From 6b6944820300f50c014490831295c3fb943f71a2 Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Fri, 12 Apr 2024 13:18:43 +0800 Subject: [PATCH 26/71] [chore](ci) cloud_p* checkout the same commit as in compile (#33215) * [chore](ci) cloud_p* checkout the same commit as in compile Co-authored-by: stephen --- regression-test/pipeline/cloud_p0/prepare.sh | 97 +++++++++++++------- regression-test/pipeline/cloud_p0/run.sh | 4 + 2 files changed, 68 insertions(+), 33 deletions(-) diff --git a/regression-test/pipeline/cloud_p0/prepare.sh b/regression-test/pipeline/cloud_p0/prepare.sh index 7f0e693b3ee424..89bfa3150bc09f 100644 --- a/regression-test/pipeline/cloud_p0/prepare.sh +++ b/regression-test/pipeline/cloud_p0/prepare.sh @@ -11,41 +11,14 @@ rm -rf ../.old/* export teamcity_build_checkoutDir="%teamcity.build.checkoutDir%" export commit_id_from_checkout="%build.vcs.number%" export target_branch='%teamcity.pullRequest.target.branch%' - -merge_pr_to_target_branch_latest() { - local pr_num_from_trigger="$1" - local target_branch="$2" - echo "INFO: merge pull request into ${target_branch}" - if [[ -z "${teamcity_build_checkoutDir}" ]]; then - echo "ERROR: env teamcity_build_checkoutDir not set" && return 1 - fi - cd "${teamcity_build_checkoutDir}" || return 1 - git reset --hard - git fetch origin "${target_branch}" - git checkout "${target_branch}" - git reset --hard origin/"${target_branch}" - git pull origin "${target_branch}" - git submodule update --init be/src/clucene - git submodule update --init be/src/apache-orc - # target_branch_commit_id=$(git rev-parse HEAD) - git config user.email "ci@selectdb.com" - git config user.name "ci" - echo "git fetch origin refs/pull/${pr_num_from_trigger}/head" - git fetch origin "refs/pull/${pr_num_from_trigger}/head" - git merge --no-edit --allow-unrelated-histories FETCH_HEAD - echo "INFO: merge refs/pull/${pr_num_from_trigger}/head into master: $(git rev-parse HEAD)" - # CONFLICTS=$(git ls-files -u | wc -l) - if [[ $(git ls-files -u | wc -l) -gt 0 ]]; then - echo "ERROR: merge refs/pull/${pr_num_from_trigger}/head into master failed. Aborting" - git merge --abort - return 1 - fi -} -# It should not merge, otherwise binary and cases may mismatch! -# if ! merge_pr_to_target_branch_latest "${pr_num_from_trigger}" "${target_branch}";then return 1; fi +export teamcity_buildType_id='%system.teamcity.buildType.id%' +export skip_pipeline='%skip_pipline%' export PATH=/usr/local/software/apache-maven-3.6.3/bin:${PATH} if [[ -f "${teamcity_build_checkoutDir:-}"/regression-test/pipeline/cloud_p0/prepare.sh ]]; then cd "${teamcity_build_checkoutDir}"/regression-test/pipeline/cloud_p0/ + if [[ "${skip_pipeline}" == "true" ]]; then + bash "${teamcity_build_checkoutDir}"/regression-test/pipeline/common/get-or-set-tmp-env.sh 'set' "export skip_pipeline=true" + fi bash prepare.sh else echo "Build Step file missing: regression-test/pipeline/cloud_p0/prepare.sh" && exit 1 @@ -65,7 +38,7 @@ fi # stop_doris, clean_fdb, install_fdb, install_java source "${teamcity_build_checkoutDir}"/regression-test/pipeline/common/doris-utils.sh # shellcheck source=/dev/null -# check_oss_file_exist +# check_oss_file_exist, download_oss_file source "${teamcity_build_checkoutDir}"/regression-test/pipeline/common/oss-utils.sh echo "#### Check env" @@ -133,5 +106,63 @@ install_fdb clean_fdb "cloud_instance_0" echo "#### 5. check if binary package ready" +merge_pr_to_master_commit() { + local pr_num_from_trigger="$1" + local target_branch="$2" + local master_commit="$3" + echo "INFO: merge pull request into ${target_branch} ${master_commit}" + if [[ -z "${teamcity_build_checkoutDir}" ]]; then + echo "ERROR: env teamcity_build_checkoutDir not set" && return 1 + fi + cd "${teamcity_build_checkoutDir}" || return 1 + git reset --hard + git fetch origin "${target_branch}" + git checkout "${target_branch}" + git reset --hard origin/"${target_branch}" + git checkout "${master_commit}" + returnValue=$? + if [[ ${returnValue} -ne 0 ]]; then + echo "ERROR: checkout ${target_branch} ${master_commit} failed. please rebase to the newest version." + return 1 + fi + git rev-parse HEAD + git config user.email "ci@selectdb.com" + git config user.name "ci" + echo "git fetch origin refs/pull/${pr_num_from_trigger}/head" + git fetch origin "refs/pull/${pr_num_from_trigger}/head" + git merge --no-edit --allow-unrelated-histories FETCH_HEAD + echo "INFO: merge refs/pull/${pr_num_from_trigger}/head into ${target_branch} ${master_commit}" + # CONFLICTS=$(git ls-files -u | wc -l) + if [[ $(git ls-files -u | wc -l) -gt 0 ]]; then + echo "ERROR: merge refs/pull/${pr_num_from_trigger}/head into failed. Aborting" + git merge --abort + return 1 + fi +} export OSS_DIR="${OSS_DIR:-"oss://opensource-pipeline/compile_result"}" if ! check_oss_file_exist "${pr_num_from_trigger}_${commit_id_from_trigger}.tar.gz"; then return 1; fi +if download_oss_file "${pr_num_from_trigger}_${commit_id_from_trigger}.tar.gz"; then + rm -rf "${teamcity_build_checkoutDir}"/output + tar -I pigz -xf "${pr_num_from_trigger}_${commit_id_from_trigger}.tar.gz" + master_commit_file="master.commit" + if [[ -e output/${master_commit_file} ]]; then + # checkout to master commit and merge this pr, to ensure binary and case are same version + master_commit=$(cat output/"${master_commit_file}") + if merge_pr_to_master_commit "${pr_num_from_trigger}" "${target_branch}" "${master_commit}"; then + echo "INFO: merged done" + if [[ "${teamcity_buildType_id:-}" == "Doris_DorisCloudRegression_CloudP1" ]]; then + echo "INFO: 用cloud_p1/conf覆盖cloud_p0/conf" + if [[ -d "${teamcity_build_checkoutDir:-}"/regression-test/pipeline/cloud_p1/conf ]]; then + cp -rf "${teamcity_build_checkoutDir}"/regression-test/pipeline/cloud_p1/conf/* \ + "${teamcity_build_checkoutDir}"/regression-test/pipeline/cloud_p0/conf/ + else + echo "ERROR: regression-test/pipeline/cloud_p1/conf not exist" && exit 1 + fi + fi + else + exit 1 + fi + fi +else + exit 1 +fi diff --git a/regression-test/pipeline/cloud_p0/run.sh b/regression-test/pipeline/cloud_p0/run.sh index 6dd594163d52e6..ef06c5cdf5e015 100644 --- a/regression-test/pipeline/cloud_p0/run.sh +++ b/regression-test/pipeline/cloud_p0/run.sh @@ -55,6 +55,10 @@ run() { echo "sk='${cos_sk}'" >>"${teamcity_build_checkoutDir}"/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy cp -f "${teamcity_build_checkoutDir}"/regression-test/pipeline/cloud_p0/conf/regression-conf-custom.groovy \ "${teamcity_build_checkoutDir}"/regression-test/conf/ + # start kafka docker to run case test_rountine_load + sed -i "s/^CONTAINER_UID=\"doris--\"/CONTAINER_UID=\"doris-external--\"/" "${teamcity_build_checkoutDir}"/docker/thirdparties/custom_settings.env + if bash "${teamcity_build_checkoutDir}"/docker/thirdparties/run-thirdparties-docker.sh --stop; then echo; fi + bash "${teamcity_build_checkoutDir}"/docker/thirdparties/run-thirdparties-docker.sh -c kafka JAVA_HOME="$(find /usr/lib/jvm -maxdepth 1 -type d -name 'java-8-*' | sed -n '1p')" export JAVA_HOME if "${teamcity_build_checkoutDir}"/run-regression-test.sh \ From 05467d34c5cdce09b1a436bb7fa3f6450b72c73d Mon Sep 17 00:00:00 2001 From: HHoflittlefish777 <77738092+HHoflittlefish777@users.noreply.github.com> Date: Fri, 12 Apr 2024 14:29:53 +0800 Subject: [PATCH 27/71] [fix](routine-load) fix consumer hang when kafka exception causing can not query (#33492) --- thirdparty/patches/librdkafka-1.9.2.patch | 31 +++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/thirdparty/patches/librdkafka-1.9.2.patch b/thirdparty/patches/librdkafka-1.9.2.patch index 38064e751dc0a2..b13e740bc5c36a 100644 --- a/thirdparty/patches/librdkafka-1.9.2.patch +++ b/thirdparty/patches/librdkafka-1.9.2.patch @@ -34,6 +34,37 @@ # Clear define name ($2): caller may have additional checks mkl_check_failed "$cname" "" "$3" "pkg-config --libs failed" return 1 +--- src/rdkafka.c ++++ src/rdkafka.c +@@ -3510,6 +3510,7 @@ rd_kafka_resp_err_t rd_kafka_query_watermark_offsets(rd_kafka_t *rk, + struct rd_kafka_partition_leader *leader; + rd_list_t leaders; + rd_kafka_resp_err_t err; ++ int tmout; + + partitions = rd_kafka_topic_partition_list_new(1); + rktpar = +@@ -3556,11 +3557,15 @@ rd_kafka_resp_err_t rd_kafka_query_watermark_offsets(rd_kafka_t *rk, + rd_list_destroy(&leaders); + + /* Wait for reply (or timeout) */ +- while (state.err == RD_KAFKA_RESP_ERR__IN_PROGRESS && +- rd_kafka_q_serve(rkq, 100, 0, RD_KAFKA_Q_CB_CALLBACK, +- rd_kafka_poll_cb, +- NULL) != RD_KAFKA_OP_RES_YIELD) +- ; ++ while (state.err == RD_KAFKA_RESP_ERR__IN_PROGRESS) { ++ tmout = rd_timeout_remains(ts_end); ++ if (rd_timeout_expired(tmout)) { ++ state.err = RD_KAFKA_RESP_ERR__TIMED_OUT; ++ break; ++ } ++ rd_kafka_q_serve(rkq, tmout, 0, RD_KAFKA_Q_CB_CALLBACK, ++ rd_kafka_poll_cb, NULL); ++ } + + rd_kafka_q_destroy_owner(rkq); + --- src/rdkafka_broker.c +++ src/rdkafka_broker.c @@ -5461,7 +5461,9 @@ static int rd_kafka_broker_thread_main(void *arg) { From 50f0ca80ed033d7880e65086fb68a0324c5412c8 Mon Sep 17 00:00:00 2001 From: Mryange <59914473+Mryange@users.noreply.github.com> Date: Fri, 12 Apr 2024 14:36:44 +0800 Subject: [PATCH 28/71] [fix](timeout) query timeout was not correctly set (#33444) --- .../org/apache/doris/qe/ConnectContext.java | 21 +++++++++- .../java/org/apache/doris/qe/Coordinator.java | 7 ++-- .../org/apache/doris/qe/ResultReceiver.java | 41 ++++++++++++------- .../org/apache/doris/qe/StmtExecutor.java | 14 +++++++ 4 files changed, 65 insertions(+), 18 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java index 5188eae3dfe427..79ec64e9e66125 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java @@ -56,6 +56,7 @@ import org.apache.doris.plsql.Exec; import org.apache.doris.plsql.executor.PlSqlOperation; import org.apache.doris.plugin.audit.AuditEvent.AuditEventBuilder; +import org.apache.doris.proto.Types; import org.apache.doris.resource.Tag; import org.apache.doris.service.arrowflight.results.FlightSqlChannel; import org.apache.doris.statistics.ColumnStatistic; @@ -891,6 +892,24 @@ public void kill(boolean killConnection) { cancelQuery(); } + // kill operation with no protect by timeout. + private void killByTimeout(boolean killConnection) { + LOG.warn("kill query from {}, kill mysql connection: {} reason time out", getRemoteHostPortString(), + killConnection); + + if (killConnection) { + isKilled = true; + // Close channel to break connection with client + closeChannel(); + } + // Now, cancel running query. + // cancelQuery by time out + StmtExecutor executorRef = executor; + if (executorRef != null) { + executorRef.cancel(Types.PPlanFragmentCancelReason.TIMEOUT); + } + } + public void cancelQuery() { StmtExecutor executorRef = executor; if (executorRef != null) { @@ -931,7 +950,7 @@ public void checkTimeout(long now) { } if (killFlag) { - kill(killConnection); + killByTimeout(killConnection); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index c389bd36240145..3dd48dfade06de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -1330,7 +1330,7 @@ public RowBatch getNext() throws Exception { resultBatch = receiver.getNext(status); if (!status.ok()) { LOG.warn("Query {} coordinator get next fail, {}, need cancel.", - DebugUtil.printId(queryId), status.toString()); + DebugUtil.printId(queryId), status.getErrorMsg()); } updateStatus(status); @@ -1474,7 +1474,8 @@ public void cancel(Types.PPlanFragmentCancelReason cancelReason) { } else { queryStatus.setStatus(Status.CANCELLED); } - LOG.warn("Cancel execution of query {}, this is a outside invoke", DebugUtil.printId(queryId)); + LOG.warn("Cancel execution of query {}, this is a outside invoke, cancelReason {}", + DebugUtil.printId(queryId), cancelReason.toString()); cancelInternal(cancelReason); } finally { unlock(); @@ -1501,7 +1502,7 @@ private void cancelLatch() { private void cancelInternal(Types.PPlanFragmentCancelReason cancelReason) { if (null != receiver) { - receiver.cancel(cancelReason.toString()); + receiver.cancel(cancelReason); } if (null != pointExec) { pointExec.cancel(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java index c473d74b919252..275ba0ffd78fb1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java @@ -42,8 +42,10 @@ public class ResultReceiver { private static final Logger LOG = LogManager.getLogger(ResultReceiver.class); - private boolean isDone = false; - private boolean isCancel = false; + private boolean isDone = false; + // runStatus represents the running status of the ResultReceiver. + // If it is not "OK," it indicates cancel. + private Status runStatus = new Status(); private long packetIdx = 0; private long timeoutTs = 0; private TNetworkAddress address; @@ -56,6 +58,14 @@ public class ResultReceiver { int maxMsgSizeOfResultReceiver; + private void setRunStatus(Status status) { + runStatus.setStatus(status); + } + + private boolean isCancel() { + return !runStatus.ok(); + } + public ResultReceiver(TUniqueId queryId, TUniqueId tid, Long backendId, TNetworkAddress address, long timeoutTs, int maxMsgSizeOfResultReceiver) { this.queryId = Types.PUniqueId.newBuilder().setHi(queryId.hi).setLo(queryId.lo).build(); @@ -72,15 +82,14 @@ public RowBatch getNext(Status status) throws TException { } final RowBatch rowBatch = new RowBatch(); try { - while (!isDone && !isCancel) { + while (!isDone && !isCancel()) { InternalService.PFetchDataRequest request = InternalService.PFetchDataRequest.newBuilder() .setFinstId(finstId) .setRespInAttachment(false) .build(); currentThread = Thread.currentThread(); - fetchDataAsyncFuture - = BackendServiceProxy.getInstance().fetchDataAsync(address, request); + fetchDataAsyncFuture = BackendServiceProxy.getInstance().fetchDataAsync(address, request); InternalService.PFetchDataResult pResult = null; while (pResult == null) { @@ -92,7 +101,7 @@ public RowBatch getNext(Status status) throws TException { pResult = fetchDataAsyncFuture.get(timeoutTs - currentTs, TimeUnit.MILLISECONDS); } catch (CancellationException e) { LOG.warn("Future of ResultReceiver of query {} is cancelled", DebugUtil.printId(this.queryId)); - if (!isCancel) { + if (!isCancel()) { LOG.warn("ResultReceiver is not set to cancelled state, this should not happen"); } else { status.setStatus(new Status(TStatusCode.CANCELLED, this.cancelReason)); @@ -101,7 +110,7 @@ public RowBatch getNext(Status status) throws TException { } catch (TimeoutException e) { LOG.warn("Query {} get result timeout, get result duration {} ms", DebugUtil.printId(this.queryId), (timeoutTs - currentTs) / 1000); - isCancel = true; + setRunStatus(Status.TIMEOUT); status.setStatus(Status.TIMEOUT); updateCancelReason("fetch data timeout"); return null; @@ -109,7 +118,7 @@ public RowBatch getNext(Status status) throws TException { // continue to get result LOG.warn("Future of ResultReceiver of query {} got interrupted Exception", DebugUtil.printId(this.queryId), e); - if (isCancel) { + if (isCancel()) { status.setStatus(Status.CANCELLED); return null; } @@ -148,7 +157,7 @@ public RowBatch getNext(Status status) throws TException { } catch (TException e) { if (e.getMessage().contains("MaxMessageSize reached")) { throw new TException( - "MaxMessageSize reached, try increase max_msg_size_of_result_receiver"); + "MaxMessageSize reached, try increase max_msg_size_of_result_receiver"); } else { throw e; } @@ -181,8 +190,8 @@ public RowBatch getNext(Status status) throws TException { } } - if (isCancel) { - status.setStatus(Status.CANCELLED); + if ((isCancel())) { + status.setStatus(runStatus); } return rowBatch; } @@ -196,9 +205,13 @@ private void updateCancelReason(String reason) { } } - public void cancel(String reason) { - isCancel = true; - updateCancelReason(reason); + public void cancel(Types.PPlanFragmentCancelReason reason) { + if (reason == Types.PPlanFragmentCancelReason.TIMEOUT) { + setRunStatus(Status.TIMEOUT); + } else { + setRunStatus(Status.CANCELLED); + } + updateCancelReason(reason.toString()); synchronized (this) { if (currentThread != null) { // TODO(cmy): we cannot interrupt this thread, or we may throw 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 d28d73c58e6e5e..6c84dbbaee6a23 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 @@ -1527,6 +1527,20 @@ public void cancel() { } } + // Because this is called by other thread + public void cancel(Types.PPlanFragmentCancelReason cancelReason) { + Coordinator coordRef = coord; + if (coordRef != null) { + coordRef.cancel(cancelReason); + } + if (mysqlLoadId != null) { + Env.getCurrentEnv().getLoadManager().getMysqlLoadManager().cancelMySqlLoad(mysqlLoadId); + } + if (parsedStmt instanceof AnalyzeTblStmt || parsedStmt instanceof AnalyzeDBStmt) { + Env.getCurrentEnv().getAnalysisManager().cancelSyncTask(context); + } + } + // Handle kill statement. private void handleKill() throws DdlException { KillStmt killStmt = (KillStmt) parsedStmt; From 3c078ad3736bdfb743438cb78ec28269c299f0b2 Mon Sep 17 00:00:00 2001 From: Mryange <59914473+Mryange@users.noreply.github.com> Date: Fri, 12 Apr 2024 14:39:05 +0800 Subject: [PATCH 29/71] [feature](profile) add non-zero counter in profile(#33342) add non-zero counter in profile --- .../pipeline/exec/exchange_sink_operator.cpp | 6 ++-- .../exec/exchange_source_operator.cpp | 4 +-- be/src/util/runtime_profile.cpp | 29 +++++++++++----- be/src/util/runtime_profile.h | 34 +++++++++++++++++++ 4 files changed, 60 insertions(+), 13 deletions(-) diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index 8323e20cfd17c1..ca50f7bd05399e 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -197,9 +197,9 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf if (channel->is_local()) { _local_channels_dependency[dep_id] = channel->get_local_channel_dependency(); DCHECK(_local_channels_dependency[dep_id] != nullptr); - _wait_channel_timer[dep_id] = ADD_CHILD_TIMER_WITH_LEVEL( - _profile, fmt::format("WaitForLocalExchangeBuffer{}", dep_id), timer_name, - 1); + _wait_channel_timer[dep_id] = _profile->add_nonzero_counter( + fmt::format("WaitForLocalExchangeBuffer{}", dep_id), TUnit ::TIME_NS, + timer_name, 1); dep_id++; } } diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp index 664e576e1ce341..fd44b23995d82b 100644 --- a/be/src/pipeline/exec/exchange_source_operator.cpp +++ b/be/src/pipeline/exec/exchange_source_operator.cpp @@ -85,8 +85,8 @@ Status ExchangeLocalState::init(RuntimeState* state, LocalStateInfo& info) { static const std::string timer_name = "WaitForDependencyTime"; _wait_for_dependency_timer = ADD_TIMER_WITH_LEVEL(_runtime_profile, timer_name, 1); for (size_t i = 0; i < queues.size(); i++) { - metrics[i] = ADD_CHILD_TIMER_WITH_LEVEL(_runtime_profile, fmt::format("WaitForData{}", i), - timer_name, 1); + metrics[i] = _runtime_profile->add_nonzero_counter(fmt::format("WaitForData{}", i), + TUnit ::TIME_NS, timer_name, 1); } RETURN_IF_ERROR(_parent->cast()._vsort_exec_exprs.clone( state, vsort_exec_exprs)); diff --git a/be/src/util/runtime_profile.cpp b/be/src/util/runtime_profile.cpp index 647c6e966c5810..782009d4438f76 100644 --- a/be/src/util/runtime_profile.cpp +++ b/be/src/util/runtime_profile.cpp @@ -429,6 +429,25 @@ RuntimeProfile::Counter* RuntimeProfile::add_counter(const std::string& name, TU return counter; } +RuntimeProfile::NonZeroCounter* RuntimeProfile::add_nonzero_counter( + const std::string& name, TUnit::type type, const std::string& parent_counter_name, + int64_t level) { + std::lock_guard l(_counter_map_lock); + if (_counter_map.find(name) != _counter_map.end()) { + DCHECK(dynamic_cast(_counter_map[name])); + return static_cast(_counter_map[name]); + } + + DCHECK(parent_counter_name == ROOT_COUNTER || + _counter_map.find(parent_counter_name) != _counter_map.end()); + NonZeroCounter* counter = _pool->add(new NonZeroCounter(type, level, parent_counter_name)); + _counter_map[name] = counter; + std::set* child_counters = + find_or_insert(&_child_counter_map, parent_counter_name, std::set()); + child_counters->insert(name); + return counter; +} + RuntimeProfile::DerivedCounter* RuntimeProfile::add_derived_counter( const std::string& name, TUnit::type type, const DerivedCounterFunction& counter_fn, const std::string& parent_counter_name) { @@ -576,14 +595,8 @@ void RuntimeProfile::to_thrift(std::vector* nodes) { node.child_counters_map = _child_counter_map; } - for (std::map::const_iterator iter = counter_map.begin(); - iter != counter_map.end(); ++iter) { - TCounter counter; - counter.name = iter->first; - counter.value = iter->second->value(); - counter.type = iter->second->type(); - counter.__set_level(iter->second->level()); - node.counters.push_back(counter); + for (auto&& [name, counter] : counter_map) { + counter->to_thrift(name, node.counters, node.child_counters_map); } { diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h index 4cc2c2617ec45f..c1756f6c63e111 100644 --- a/be/src/util/runtime_profile.h +++ b/be/src/util/runtime_profile.h @@ -116,6 +116,16 @@ class RuntimeProfile { return binary_cast(_value.load(std::memory_order_relaxed)); } + virtual void to_thrift(const std::string& name, std::vector& tcounters, + std::map>& child_counters_map) { + TCounter counter; + counter.name = name; + counter.value = this->value(); + counter.type = this->type(); + counter.__set_level(this->level()); + tcounters.push_back(counter); + } + TUnit::type type() const { return _type; } virtual int64_t level() { return _level; } @@ -201,6 +211,26 @@ class RuntimeProfile { DerivedCounterFunction _counter_fn; }; + // NonZeroCounter will not be converted to Thrift if the value is 0. + class NonZeroCounter : public Counter { + public: + NonZeroCounter(TUnit::type type, int64_t level, const std::string& parent_name) + : Counter(type, 0, level), _parent_name(parent_name) {} + + void to_thrift(const std::string& name, std::vector& tcounters, + std::map>& child_counters_map) override { + if (this->_value > 0) { + Counter::to_thrift(name, tcounters, child_counters_map); + } else { + // remove it + child_counters_map[_parent_name].erase(name); + } + } + + private: + const std::string _parent_name; + }; + // An EventSequence captures a sequence of events (each added by // calling MarkEvent). Each event has a text label, and a time // (measured relative to the moment start() was called as t=0). It is @@ -299,6 +329,10 @@ class RuntimeProfile { return add_counter(name, type, "", level); } + NonZeroCounter* add_nonzero_counter(const std::string& name, TUnit::type type, + const std::string& parent_counter_name = "", + int64_t level = 2); + // Add a derived counter with 'name'/'type'. The counter is owned by the // RuntimeProfile object. // If parent_counter_name is a non-empty string, the counter is added as a child of From d05dca0cd77b83edc0575e2eb05c0ad4b0d3cbd1 Mon Sep 17 00:00:00 2001 From: feiniaofeiafei <53502832+feiniaofeiafei@users.noreply.github.com> Date: Fri, 12 Apr 2024 15:01:17 +0800 Subject: [PATCH 30/71] [Fix] fix compile problem (#33578) Co-authored-by: feiniaofeiafei --- .../nereids/trees/plans/commands/info/CreateViewInfo.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java index 632e884685652f..6edca0a9877a24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateViewInfo.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.ColWithComment; import org.apache.doris.analysis.CreateViewStmt; +import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.common.ErrorCode; @@ -112,8 +113,8 @@ public void validate(ConnectContext ctx) throws UserException { // disallow external catalog Util.prohibitExternalCatalog(viewName.getCtl(), "CreateViewStmt"); // check privilege - if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ConnectContext.get(), viewName.getDb(), - viewName.getTbl(), PrivPredicate.CREATE)) { + if (!Env.getCurrentEnv().getAccessManager().checkTblPriv(ctx, new TableName(viewName.getCtl(), viewName.getDb(), + viewName.getTbl()), PrivPredicate.CREATE)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "CREATE"); } Set colSets = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); From 7e485874495810c844d69e4b27f01091577ff231 Mon Sep 17 00:00:00 2001 From: LiBinfeng <46676950+LiBinfeng-01@users.noreply.github.com> Date: Fri, 12 Apr 2024 15:06:02 +0800 Subject: [PATCH 31/71] [Fix](Nereids) fix leading hint should have all tables in one query block (#33517) --- .../doris/nereids/hint/LeadingHint.java | 59 +++++++++---------- .../nereids/rules/analysis/LeadingJoin.java | 2 +- .../data/nereids_p0/hint/fix_leading.out | 25 ++++++++ .../data/nereids_p0/hint/multi_leading.out | 4 +- .../suites/nereids_p0/hint/fix_leading.groovy | 4 ++ .../nereids_p0/hint/multi_leading.groovy | 8 +-- 6 files changed, 65 insertions(+), 37 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/LeadingHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/LeadingHint.java index 1f48e4873cbf07..3ef5217566a44d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/LeadingHint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/LeadingHint.java @@ -17,7 +17,6 @@ package org.apache.doris.nereids.hint; -import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Pair; import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -226,14 +225,14 @@ public RelationId findRelationIdAndTableName(String name) { return null; } - private boolean hasSameName() { + private Optional hasSameName() { Set tableSet = Sets.newHashSet(); for (String table : tablelist) { if (!tableSet.add(table)) { - return true; + return Optional.of(table); } } - return false; + return Optional.empty(); } public Map getExprIdToTableNameMap() { @@ -287,12 +286,14 @@ public Long getTotalBitmap() { /** * set total bitmap used in leading before we get into leading join */ - public void setTotalBitmap() { + public void setTotalBitmap(Set inputRelationSets) { Long totalBitmap = 0L; - if (hasSameName()) { + Optional duplicateTableName = hasSameName(); + if (duplicateTableName.isPresent()) { this.setStatus(HintStatus.SYNTAX_ERROR); - this.setErrorMessage("duplicated table"); + this.setErrorMessage("duplicated table:" + duplicateTableName.get()); } + Set existRelationSets = new HashSet<>(); for (int index = 0; index < getTablelist().size(); index++) { RelationId id = findRelationIdAndTableName(getTablelist().get(index)); if (id == null) { @@ -300,11 +301,32 @@ public void setTotalBitmap() { this.setErrorMessage("can not find table: " + getTablelist().get(index)); return; } + existRelationSets.add(id); totalBitmap = LongBitmap.set(totalBitmap, id.asInt()); } + if (getTablelist().size() < inputRelationSets.size()) { + Set missRelationIds = new HashSet<>(); + missRelationIds.addAll(inputRelationSets); + missRelationIds.removeAll(existRelationSets); + String missingTablenames = getMissingTableNames(missRelationIds); + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("leading should have all tables in query block, missing tables: " + missingTablenames); + } this.totalBitmap = totalBitmap; } + private String getMissingTableNames(Set missRelationIds) { + String missTableNames = ""; + for (RelationId id : missRelationIds) { + for (Pair pair : relationIdAndTableName) { + if (pair.first.equals(id)) { + missTableNames += pair.second + " "; + } + } + } + return missTableNames; + } + /** * try to get join constraint, if can not get, it means join is inner join, * @param joinTableBitmap table bitmap below this join @@ -612,27 +634,4 @@ private Long getBitmap(LogicalPlan root) { return null; } } - - /** - * get leading containing tables which means leading wants to combine tables into joins - * @return long value represent tables we included - */ - public Long getLeadingTableBitmap(List tables) { - Long totalBitmap = 0L; - if (hasSameName()) { - this.setStatus(HintStatus.SYNTAX_ERROR); - this.setErrorMessage("duplicated table"); - return totalBitmap; - } - for (int index = 0; index < getTablelist().size(); index++) { - RelationId id = findRelationIdAndTableName(getTablelist().get(index)); - if (id == null) { - this.setStatus(HintStatus.SYNTAX_ERROR); - this.setErrorMessage("can not find table: " + getTablelist().get(index)); - return totalBitmap; - } - totalBitmap = LongBitmap.set(totalBitmap, id.asInt()); - } - return totalBitmap; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LeadingJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LeadingJoin.java index c8b19a1e9d5398..9e94c745f779c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LeadingJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LeadingJoin.java @@ -43,7 +43,7 @@ public List buildRules() { return ctx.root; } Hint leadingHint = ctx.cascadesContext.getHintMap().get("Leading"); - ((LeadingHint) leadingHint).setTotalBitmap(); + ((LeadingHint) leadingHint).setTotalBitmap(ctx.root.getInputRelations()); Long currentBitMap = LongBitmap.computeTableBitmap(ctx.root.getInputRelations()); if (((LeadingHint) leadingHint).getTotalBitmap().equals(currentBitMap) && leadingHint.isSuccess()) { diff --git a/regression-test/data/nereids_p0/hint/fix_leading.out b/regression-test/data/nereids_p0/hint/fix_leading.out index 2a83ff6eafe174..c7bfec92f2dd0d 100644 --- a/regression-test/data/nereids_p0/hint/fix_leading.out +++ b/regression-test/data/nereids_p0/hint/fix_leading.out @@ -256,3 +256,28 @@ Used: leading(t1 t2 t3 ) UnUsed: SyntaxError: +-- !select5_1 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[LOCAL] +--------PhysicalProject +----------NestedLoopJoin[LEFT_OUTER_JOIN](t3.c3 > 500) +------------PhysicalProject +--------------PhysicalOlapScan[t3] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------NestedLoopJoin[LEFT_OUTER_JOIN](t1.c1 > 500) +------------------PhysicalProject +--------------------filter((t1.c1 < 200)) +----------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------filter((t2.c2 > 500)) +------------------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: +SyntaxError: leading(t1 t2) Msg:leading should have all tables in query block, missing tables: t3 + diff --git a/regression-test/data/nereids_p0/hint/multi_leading.out b/regression-test/data/nereids_p0/hint/multi_leading.out index 0efabe2002e20b..b602e97bcb94fc 100644 --- a/regression-test/data/nereids_p0/hint/multi_leading.out +++ b/regression-test/data/nereids_p0/hint/multi_leading.out @@ -264,7 +264,7 @@ PhysicalResultSink ----------------------PhysicalOlapScan[t1] Hint log: -Used: leading(t2 t1 ) leading(t3 alias1 ) +Used: leading(t2 t1 ) leading(t3 alias1 cte ) UnUsed: SyntaxError: @@ -330,7 +330,7 @@ PhysicalResultSink ----------------------PhysicalOlapScan[t1] Hint log: -Used: leading(t2 t1 ) leading(t2 t1 ) leading(t2 t1 ) leading(t3 alias1 ) +Used: leading(t2 t1 ) leading(t2 t1 ) leading(t2 t1 ) leading(t3 alias1 cte ) UnUsed: SyntaxError: diff --git a/regression-test/suites/nereids_p0/hint/fix_leading.groovy b/regression-test/suites/nereids_p0/hint/fix_leading.groovy index 6185789d2baa99..49823e9769830a 100644 --- a/regression-test/suites/nereids_p0/hint/fix_leading.groovy +++ b/regression-test/suites/nereids_p0/hint/fix_leading.groovy @@ -170,4 +170,8 @@ suite("fix_leading") { qt_select4_1 """select count(*) from t1 left join t2 on c1 > 500 and c2 >500 right join t3 on c3 > 500 and c1 < 200;""" qt_select4_2 """select /*+ leading(t1 t2 t3)*/ count(*) from t1 left join t2 on c1 > 500 and c2 >500 right join t3 on c3 > 500 and c1 < 200;""" qt_select4_3 """explain shape plan select /*+ leading(t1 t2 t3)*/ count(*) from t1 left join t2 on c1 > 500 and c2 >500 right join t3 on c3 > 500 and c1 < 200;""" + + // check whether we have all tables + qt_select5_1 """explain shape plan select /*+ leading(t1 t2)*/ count(*) from t1 left join t2 on c1 > 500 and c2 >500 right join t3 on c3 > 500 and c1 < 200;""" + } diff --git a/regression-test/suites/nereids_p0/hint/multi_leading.groovy b/regression-test/suites/nereids_p0/hint/multi_leading.groovy index d1f83d15ebb7cc..bd567ecdefa378 100644 --- a/regression-test/suites/nereids_p0/hint/multi_leading.groovy +++ b/regression-test/suites/nereids_p0/hint/multi_leading.groovy @@ -98,14 +98,14 @@ suite("multi_leading") { // test subquery + cte qt_sql3_1 """explain shape plan with cte as (select c11, c1 from t1 join t2 on c1 = c2) select count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;""" - qt_sql3_2 """explain shape plan with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1) */ count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" + qt_sql3_2 """explain shape plan with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1 cte) */ count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" qt_sql3_3 """explain shape plan with cte as (select c11, c1 from t1 join t2 on c1 = c2) select count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" - qt_sql3_4 """explain shape plan with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1) */ count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" + qt_sql3_4 """explain shape plan with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1 cte) */ count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" qt_sql3_res_1 """with cte as (select c11, c1 from t1 join t2 on c1 = c2) select count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" - qt_sql3_res_2 """with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1) */ count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" + qt_sql3_res_2 """with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1 cte) */ count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" qt_sql3_res_3 """with cte as (select c11, c1 from t1 join t2 on c1 = c2) select count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" - qt_sql3_res_4 """with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1) */ count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" + qt_sql3_res_4 """with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1 cte) */ count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" // test multi level subqueries qt_sql4_0 """explain shape plan select count(*) from (select c1, c11 from t1 join (select c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" From 8dc977df1d92ff9ea3927ef0b5f47e0785d9b6f7 Mon Sep 17 00:00:00 2001 From: Gabriel Date: Fri, 12 Apr 2024 16:22:35 +0800 Subject: [PATCH 32/71] [pipelineX](broadcast) Set dependency ready if a limited exchange returns EOS (#33525) --- be/src/pipeline/exec/exchange_sink_buffer.cpp | 2 ++ be/src/pipeline/exec/exchange_sink_buffer.h | 9 +++++++-- be/src/pipeline/exec/exchange_sink_operator.cpp | 1 + 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 9f692d0beeb589..44b655150af8f2 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -464,6 +464,8 @@ void ExchangeSinkBuffer::_set_receiver_eof(InstanceLoId id) { _rpc_channel_is_idle[id] = true; _set_ready_to_finish(_busy_channels.fetch_sub(1) == 1); } + std::queue, std::list>> empty; + swap(empty, _instance_to_broadcast_package_queue[id]); } template diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index 8c0375499c3f86..43fdc98d24c00e 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -216,6 +216,10 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx { _finish_dependency = finish_dependency; } + void set_broadcast_dependency(std::shared_ptr broadcast_dependency) { + _broadcast_dependency = broadcast_dependency; + } + void set_should_stop() { _should_stop = true; _set_ready_to_finish(_busy_channels == 0); @@ -270,8 +274,9 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx { int64_t get_sum_rpc_time(); std::atomic _total_queue_size = 0; - std::shared_ptr _queue_dependency; - std::shared_ptr _finish_dependency; + std::shared_ptr _queue_dependency = nullptr; + std::shared_ptr _finish_dependency = nullptr; + std::shared_ptr _broadcast_dependency = nullptr; std::atomic _should_stop {false}; }; diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index ca50f7bd05399e..07c7130894a336 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -181,6 +181,7 @@ Status ExchangeSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& inf _broadcast_dependency = Dependency::create_shared(_parent->operator_id(), _parent->node_id(), "BroadcastDependency", true, state->get_query_ctx()); + _sink_buffer->set_broadcast_dependency(_broadcast_dependency); _broadcast_pb_blocks = vectorized::BroadcastPBlockHolderQueue::create_shared(_broadcast_dependency); for (int i = 0; i < config::num_broadcast_buffer; ++i) { From 607b33356fae330c2d17149f2d4915e973e1820e Mon Sep 17 00:00:00 2001 From: Ashin Gau Date: Fri, 12 Apr 2024 17:26:27 +0800 Subject: [PATCH 33/71] [fix](schema change) CastStringConverter is compiled failed in g++ (#33546) follow #32873, CastStringConverter is compiled failed in g++ for uninitialized value, which is ok in clang: --- be/src/vec/exec/format/column_type_convert.h | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/be/src/vec/exec/format/column_type_convert.h b/be/src/vec/exec/format/column_type_convert.h index 65742671f729cf..43cae065fca711 100644 --- a/be/src/vec/exec/format/column_type_convert.h +++ b/be/src/vec/exec/format/column_type_convert.h @@ -384,7 +384,7 @@ class CastStringConverter : public ColumnTypeConverter { to_col->resize(start_idx + rows); auto& data = static_cast(to_col.get())->get_data(); for (int i = 0; i < rows; ++i) { - DstCppType value; + DstCppType& value = data[start_idx + i]; auto string_value = string_col.get_data_at(i); bool can_cast = false; if constexpr (is_decimal_type_const()) { @@ -398,9 +398,7 @@ class CastStringConverter : public ColumnTypeConverter { can_cast = SafeCastString::safe_cast_string( string_value.data, string_value.size, &value); } - if (can_cast) { - data[start_idx + i] = value; - } else { + if (!can_cast) { if (null_map == nullptr) { return Status::InternalError("Failed to cast string '{}' to not null column", string_value.to_string()); From e6899a048d6ca954560018ca9690f2b755cb84c6 Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Fri, 12 Apr 2024 17:38:52 +0800 Subject: [PATCH 34/71] [test](bi) add dbeaver and datagrip connect doris test (#33487) --- .../suites/query_p0/bi_p0/datagrip.groovy | 43 ++++++++++++++++++ .../suites/query_p0/bi_p0/dbeaver.groovy | 44 +++++++++++++++++++ 2 files changed, 87 insertions(+) create mode 100644 regression-test/suites/query_p0/bi_p0/datagrip.groovy create mode 100644 regression-test/suites/query_p0/bi_p0/dbeaver.groovy diff --git a/regression-test/suites/query_p0/bi_p0/datagrip.groovy b/regression-test/suites/query_p0/bi_p0/datagrip.groovy new file mode 100644 index 00000000000000..b7402686fe0fe1 --- /dev/null +++ b/regression-test/suites/query_p0/bi_p0/datagrip.groovy @@ -0,0 +1,43 @@ +// 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("datagrip") { + sql """SET net_write_timeout=600""" + sql """select table_name, table_type, table_comment, engine, table_collation, create_options from information_schema.tables where table_schema = '__internal_schema'""" + sql """select ordinal_position, column_name, column_type, column_default, generation_expression, table_name, column_comment, is_nullable, extra, collation_name from information_schema.columns where table_schema = '__internal_schema' order by table_name, ordinal_position""" + sql """select table_name, auto_increment from information_schema.tables where table_schema = '__internal_schema'""" + sql """select table_name, auto_increment from information_schema.tables where table_schema = '__internal_schema' and auto_increment is not null""" + sql """select table_name, index_name, index_comment, index_type, non_unique, column_name, sub_part, collation, null expression from information_schema.statistics where table_schema = '__internal_schema' and index_schema = '__internal_schema' order by index_schema, table_name, index_name, index_type, seq_in_index""" + sql """select c.constraint_name, c.constraint_schema, c.table_name, c.constraint_type, false enforced from information_schema.table_constraints c where c.table_schema = '__internal_schema'""" + sql """select constraint_name, table_name, column_name, referenced_table_schema, referenced_table_name, referenced_column_name from information_schema.key_column_usage where table_schema = '__internal_schema' and referenced_column_name is not null order by table_name , constraint_name , ordinal_position""" + sql """select table_name, partition_name, subpartition_name, partition_ordinal_position, subpartition_ordinal_position, partition_method, subpartition_method, partition_expression, subpartition_expression, partition_description, partition_comment/*, tablespace_name*/ from information_schema.partitions where partition_name is not null and table_schema = 'demo'""" + sql """select trigger_name, event_object_table, event_manipulation, action_timing, definer from information_schema.triggers where trigger_schema = 'demo'""" + sql """select event_name, event_comment, definer, event_type = 'RECURRING' recurring, interval_value, interval_field, cast(coalesce(starts, execute_at) as char) starts, cast(ends as char) ends, status, on_completion = 'PRESERVE' preserve, last_executed from information_schema.events where event_schema = 'demo'""" + sql """select routine_name, routine_type, routine_definition, routine_comment, dtd_identifier, definer, is_deterministic = 'YES' is_deterministic, cast(sql_data_access as char(1)) sql_data_access, cast(security_type as char(1)) security_type from information_schema.routines where routine_schema = 'demo'""" + sql """select Host, User, Routine_name, Proc_priv, Routine_type = 'PROCEDURE' as is_proc from mysql.procs_priv where Db = 'demo';""" + sql """select grantee, table_name, column_name, privilege_type, is_grantable from information_schema.column_privileges where table_schema = 'demo' union all select grantee, table_name, '' column_name, privilege_type, is_grantable from information_schema.table_privileges where table_schema = 'demo' order by table_name, grantee, privilege_type""" + sql """select specific_name, ordinal_position, parameter_name, parameter_mode, dtd_identifier from information_schema.parameters where specific_schema = 'demo' and ordinal_position > 0 order by specific_name, ordinal_position""" + sql """select table_name, view_definition, definer from information_schema.views where table_schema = 'demo'""" + sql """select table_name, table_type, table_comment, engine, table_collation, create_options from information_schema.tables where table_schema = 'information_schema'""" + sql """select table_name, auto_increment from information_schema.tables where table_schema = 'mysql' and auto_increment is not null""" + sql """select table_name, partition_name, subpartition_name, partition_ordinal_position, subpartition_ordinal_position, partition_method, subpartition_method, partition_expression, subpartition_expression, partition_description, partition_comment/*, tablespace_name*/ from information_schema.partitions where partition_name is not null and table_schema = 'mysql'""" + sql """select trigger_name, event_object_table, event_manipulation, action_timing, definer from information_schema.triggers where trigger_schema = 'mysql'""" + sql """select event_name, event_comment, definer, event_type = 'RECURRING' recurring, interval_value, interval_field, cast(coalesce(starts, execute_at) as char) starts, cast(ends as char) ends, status, on_completion = 'PRESERVE' preserve, last_executed from information_schema.events where event_schema = 'mysql'""" + sql """select routine_name, routine_type, routine_definition, routine_comment, dtd_identifier, definer, is_deterministic = 'YES' is_deterministic, cast(sql_data_access as char(1)) sql_data_access, cast(security_type as char(1)) security_type from information_schema.routines where routine_schema = 'mysql'""" + sql """select Host, User, Routine_name, Proc_priv, Routine_type = 'PROCEDURE' as is_proc from mysql.procs_priv where Db = 'mysql';""" + sql """select grantee, table_name, column_name, privilege_type, is_grantable from information_schema.column_privileges where table_schema = 'mysql' union all select grantee, table_name, '' column_name, privilege_type, is_grantable from information_schema.table_privileges where table_schema = 'mysql' order by table_name, grantee, privilege_type""" +} \ No newline at end of file diff --git a/regression-test/suites/query_p0/bi_p0/dbeaver.groovy b/regression-test/suites/query_p0/bi_p0/dbeaver.groovy new file mode 100644 index 00000000000000..ddf5461e5b92f3 --- /dev/null +++ b/regression-test/suites/query_p0/bi_p0/dbeaver.groovy @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("dbeaver") { + sql """SELECT @@session.auto_increment_increment AS auto_increment_increment, @@character_set_client AS character_set_client, @@character_set_connection AS character_set_connection, @@character_set_results AS character_set_results, @@character_set_server AS character_set_server, @@collation_server AS collation_server, @@collation_connection AS collation_connection, @@init_connect AS init_connect, @@interactive_timeout AS interactive_timeout, @@license AS license, @@lower_case_table_names AS lower_case_table_names, @@max_allowed_packet AS max_allowed_packet, @@net_write_timeout AS net_write_timeout, @@performance_schema AS performance_schema, @@query_cache_size AS query_cache_size, @@query_cache_type AS query_cache_type, @@sql_mode AS sql_mode, @@system_time_zone AS system_time_zone, @@time_zone AS time_zone, @@transaction_isolation AS transaction_isolation, @@wait_timeout AS wait_timeout""" + sql """SET character_set_results = NULL""" + sql """SET autocommit=1""" + sql """SET sql_mode='STRICT_TRANS_TABLES'""" + sql """SET autocommit=1""" + sql """SELECT DATABASE()""" + sql """SHOW ENGINES""" + sql """SHOW CHARSET""" + sql """SHOW COLLATION""" + sql """SELECT @@GLOBAL.character_set_server,@@GLOBAL.collation_server""" + sql """SHOW PLUGINS""" + sql """SHOW VARIABLES LIKE 'lower_case_table_names'""" + sql """show databases""" + sql """SELECT * FROM information_schema.TABLES t WHERE t.TABLE_SCHEMA = 'information_schema' AND t.TABLE_NAME = 'CHECK_CONSTRAINTS'""" + sql """SHOW FULL PROCESSLIST""" + sql """SHOW PRIVILEGES""" + sql """SHOW GRANTS FOR 'admin'@'%'""" + sql """SHOW STATUS""" + sql """SHOW GLOBAL STATUS""" + sql """SHOW VARIABLES""" + sql """SHOW GLOBAL VARIABLES""" + sql """SHOW FULL TABLES FROM `__internal_schema`""" + sql """SHOW TABLE STATUS FROM `__internal_schema`""" + sql """SELECT * FROM information_schema.COLUMNS WHERE TABLE_SCHEMA='__internal_schema' AND TABLE_NAME='column_statistics' ORDER BY ORDINAL_POSITION""" + sql """SELECT * FROM mysql.user ORDER BY user""" +} \ No newline at end of file From c5a81e98d74f973ea8ce524b618b154e429128c7 Mon Sep 17 00:00:00 2001 From: Liqf <109049295+LemonLiTree@users.noreply.github.com> Date: Fri, 12 Apr 2024 17:59:39 +0800 Subject: [PATCH 35/71] [k8s](improve)add docker resource script for k8s (#33329) --- docker/runtime/be/Dockerfile | 5 +- docker/runtime/be/resource/be_entrypoint.sh | 244 +++++++++++ docker/runtime/be/resource/be_prestop.sh | 21 + docker/runtime/be/resource/entry_point.sh | 0 docker/runtime/be/resource/init_be.sh | 2 +- docker/runtime/broker/Dockerfile | 54 ++- .../broker/resource/broker_entrypoint.sh | 222 ++++++++++ .../broker/resource/broker_is_alive.sh | 39 ++ .../runtime/broker/resource/broker_prestop.sh | 21 + docker/runtime/broker/resource/init_broker.sh | 2 +- docker/runtime/fe/Dockerfile | 6 +- docker/runtime/fe/resource/fe_check_master.sh | 42 ++ docker/runtime/fe/resource/fe_entrypoint.sh | 395 ++++++++++++++++++ docker/runtime/fe/resource/fe_prestop.sh | 21 + docker/runtime/fe/resource/init_fe.sh | 6 +- 15 files changed, 1055 insertions(+), 25 deletions(-) create mode 100755 docker/runtime/be/resource/be_entrypoint.sh create mode 100755 docker/runtime/be/resource/be_prestop.sh mode change 100644 => 100755 docker/runtime/be/resource/entry_point.sh create mode 100755 docker/runtime/broker/resource/broker_entrypoint.sh create mode 100755 docker/runtime/broker/resource/broker_is_alive.sh create mode 100755 docker/runtime/broker/resource/broker_prestop.sh create mode 100755 docker/runtime/fe/resource/fe_check_master.sh create mode 100755 docker/runtime/fe/resource/fe_entrypoint.sh create mode 100755 docker/runtime/fe/resource/fe_prestop.sh diff --git a/docker/runtime/be/Dockerfile b/docker/runtime/be/Dockerfile index e890f0532375fc..1e1b450576454b 100644 --- a/docker/runtime/be/Dockerfile +++ b/docker/runtime/be/Dockerfile @@ -1,4 +1,3 @@ -#!/bin/bash # 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 @@ -51,10 +50,14 @@ ADD resource/apache-doris-${DORIS_VERSION}-bin-${TARGETARCH:-amd64}/be /opt/apac COPY resource/be_*.sh /opt/apache-doris/ +COPY --from=selectdb/doris-debug-ubuntu:latest /doris-debug /opt/apache-doris/ + COPY resource/entry_point.sh /usr/local/bin/ COPY resource/init_be.sh /usr/local/bin/ +#RUN ln -sf /usr/share/zoneinfo/Asia/Shanghai /etc/localtime + WORKDIR /opt/apache-doris ENTRYPOINT ["bash","entry_point.sh"] diff --git a/docker/runtime/be/resource/be_entrypoint.sh b/docker/runtime/be/resource/be_entrypoint.sh new file mode 100755 index 00000000000000..4c1ca21ba94a54 --- /dev/null +++ b/docker/runtime/be/resource/be_entrypoint.sh @@ -0,0 +1,244 @@ +#!/bin/bash +# 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. + + +# the fe query port for mysql. +FE_QUERY_PORT=${FE_QUERY_PORT:-9030} +# timeout for probe fe master. +PROBE_TIMEOUT=60 +# interval time to probe fe. +PROBE_INTERVAL=2 +# rpc port for fe communicate with be. +HEARTBEAT_PORT=9050 +# fqdn or ip +MY_SELF= +MY_IP=`hostname -i` +MY_HOSTNAME=`hostname -f` +DORIS_ROOT=${DORIS_ROOT:-"/opt/apache-doris"} +# if config secret for basic auth about operate node of doris, the path must be `/etc/basic_auth`. This is set by operator and the key of password must be `password`. +AUTH_PATH="/etc/basic_auth" +DORIS_HOME=${DORIS_ROOT}/be +BE_CONFIG=$DORIS_HOME/conf/be.conf +# represents self in fe meta or not. +REGISTERED=false + +DB_ADMIN_USER=${USER:-"root"} + +DB_ADMIN_PASSWD=$PASSWD + +log_stderr() +{ + echo "[`date`] $@" >&2 +} + +update_conf_from_configmap() +{ + if [[ "x$CONFIGMAP_MOUNT_PATH" == "x" ]] ; then + log_stderr '[info] Empty $CONFIGMAP_MOUNT_PATH env var, skip it!' + return 0 + fi + if ! test -d $CONFIGMAP_MOUNT_PATH ; then + log_stderr "[info] $CONFIGMAP_MOUNT_PATH not exist or not a directory, ignore ..." + return 0 + fi + local tgtconfdir=$DORIS_HOME/conf + for conffile in `ls $CONFIGMAP_MOUNT_PATH` + do + log_stderr "[info] Process conf file $conffile ..." + local tgt=$tgtconfdir/$conffile + if test -e $tgt ; then + # make a backup + mv -f $tgt ${tgt}.bak + fi + ln -sfT $CONFIGMAP_MOUNT_PATH/$conffile $tgt + done +} + +# resolve password for root +resolve_password_from_secret() +{ + if [[ -f "$AUTH_PATH/password" ]]; then + DB_ADMIN_PASSWD=`cat $AUTH_PATH/password` + fi + if [[ -f "$AUTH_PATH/username" ]]; then + DB_ADMIN_USER=`cat $AUTH_PATH/username` + fi +} + +# get all backends info to check self exist or not. +show_backends(){ + local svc=$1 + backends=`timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -uroot --skip-column-names --batch -e 'SHOW BACKENDS;' 2>&1` + log_stderr "[info] use root no password show backends result $backends ." + if echo $backends | grep -w "1045" | grep -q -w "28000" &>/dev/null; then + log_stderr "[info] use username and password that configured to show backends." + backends=`timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e 'SHOW BACKENDS;'` + fi + + echo "$backends" + + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e 'SHOW BACKENDS;' + #else + # timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER --skip-column-names --batch -e 'SHOW BACKENDS;' + #fi +} + +# get all registered fe in cluster, for check the fe have `MASTER`. +function show_frontends() +{ + local addr=$1 + frontends=`timeout 15 mysql --connect-timeout 2 -h $addr -P $FE_QUERY_PORT -uroot --batch -e 'show frontends;' 2>&1` + log_stderr "[info] use root no password show frontends result $frontends ." + if echo $frontends | grep -w "1045" | grep -q -w "28000" &>/dev/null; then + log_stderr "[info] use username and passwore that configured to show frontends." + frontends=`timeout 15 mysql --connect-timeout 2 -h $addr -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --batch -e 'show frontends;'` + fi + + echo "$frontends" + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # timeout 15 mysql --connect-timeout 2 -h $addr -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --batch -e 'show frontends;' + #else + # timeout 15 mysql --connect-timeout 2 -h $addr -P $FE_QUERY_PORT -u$DB_ADMIN_USER --batch -e 'show frontends;' + #fi +} + +#parse the `$BE_CONFIG` file, passing the key need resolve as parameter. +parse_confval_from_conf() +{ + # a naive script to grep given confkey from fe conf file + # assume conf format: ^\s*\s*=\s*\s*$ + local confkey=$1 + local confvalue=`grep "\<$confkey\>" $BE_CONFIG | grep -v '^\s*#' | sed 's|^\s*'$confkey'\s*=\s*\(.*\)\s*$|\1|g'` + echo "$confvalue" +} + +collect_env_info() +{ + # heartbeat_port from conf file + local heartbeat_port=`parse_confval_from_conf "heartbeat_service_port"` + if [[ "x$heartbeat_port" != "x" ]] ; then + HEARTBEAT_PORT=$heartbeat_port + fi + + if [[ "x$HOST_TYPE" == "xIP" ]] ; then + MY_SELF=$MY_IP + else + MY_SELF=$MY_HOSTNAME + fi +} + +add_self() +{ + local svc=$1 + start=`date +%s` + local timeout=$PROBE_TIMEOUT + + while true + do + memlist=`show_backends $svc` + if echo "$memlist" | grep -q -w "$MY_SELF" &>/dev/null ; then + log_stderr "[info] Check myself ($MY_SELF:$HEARTBEAT_PORT) exist in FE, start be directly ..." + break; + fi + + # check fe cluster have master, if fe have not master wait. + fe_memlist=`show_frontends $svc` + local pos=`echo "$fe_memlist" | grep '\' | awk -F '\t' '{for(i=1;i' | awk -v p="$pos" -F '\t' '{if ($p=="true") print $2}'` + log_stderr "'IsMaster' sequence in columns is $pos master=$leader ." + + if [[ "x$leader" == "x" ]]; then + log_stderr "[info] resolve the eighth column for finding master !" + leader=`echo "$fe_memlist" | grep '\' | awk -F '\t' '{if ($8=="true") print $2}'` + fi + if [[ "x$leader" == "x" ]]; then + # compatible 2.1.0 + log_stderr "[info] resoluve the ninth column for finding master!" + leader=`echo "$fe_memlist" | grep '\' | awk -F '\t' '{if ($9=="true") print $2}'` + fi + + if [[ "x$leader" != "x" ]]; then + log_stderr "[info] myself ($MY_SELF:$HEARTBEAT_PORT) not exist in FE and fe have leader register myself into fe." + add_result=`timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -uroot --skip-column-names --batch -e "ALTER SYSTEM ADD BACKEND \"$MY_SELF:$HEARTBEAT_PORT\";" 2>&1` + if echo $add_result | grep -w "1045" | grep -q -w "28000" &>/dev/null ; then + timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e "ALTER SYSTEM ADD BACKEND \"$MY_SELF:$HEARTBEAT_PORT\";" + fi + + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e "ALTER SYSTEM ADD BACKEND \"$MY_SELF:$HEARTBEAT_PORT\";" + #else + # timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER --skip-column-names --batch -e "ALTER SYSTEM ADD BACKEND \"$MY_SELF:$HEARTBEAT_PORT\";" + #fi + + let "expire=start+timeout" + now=`date +%s` + if [[ $expire -le $now ]] ; then + log_stderr "[error] exit probe master for probing timeout." + return 0 + fi + else + log_stderr "[info] not have leader wait fe cluster elect a master, sleep 2s..." + sleep $PROBE_INTERVAL + fi + done +} + +# check be exist or not, if exist return 0, or register self in fe cluster. when all fe address failed exit script. +# `xxx1:port,xxx2:port` as parameter to function. +function check_and_register() +{ + addrs=$1 + local addrArr=(${addrs//,/ }) + for addr in ${addrArr[@]} + do + add_self $addr + + if [[ $REGISTERED ]]; then + break; + fi + done + + if [[ $REGISTERED ]]; then + return 0 + else + log_stderr "not find master in fe cluster, please use mysql connect to fe for verfing the master exist and verify domain connectivity with two pods in different node. " + exit 1 + fi +} + +fe_addrs=$1 +if [[ "x$fe_addrs" == "x" ]]; then + echo "need fe address as paramter!" + echo " Example $0 " + exit 1 +fi + +update_conf_from_configmap +# resolve password for root to manage nodes in doris. +resolve_password_from_secret +collect_env_info +#add_self $fe_addr || exit $? +check_and_register $fe_addrs +./doris-debug --component be +log_stderr "run start_be.sh" +# the server will start in the current terminal session, and the log output and console interaction will be printed to that terminal +# befor doris 2.0.2 ,doris start with : start_xx.sh +# sine doris 2.0.2 ,doris start with : start_xx.sh --console doc: https://doris.apache.org/docs/dev/install/standard-deployment/#version--202 +$DORIS_HOME/bin/start_be.sh --console + diff --git a/docker/runtime/be/resource/be_prestop.sh b/docker/runtime/be/resource/be_prestop.sh new file mode 100755 index 00000000000000..af9ab1001c604f --- /dev/null +++ b/docker/runtime/be/resource/be_prestop.sh @@ -0,0 +1,21 @@ +#!/bin/bash +# 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. + +DORIS_ROOT=${DORIS_ROOT:-"/opt/apache-doris"} +DORIS_HOME=${DORIS_ROOT}/be +$DORIS_HOME/bin/stop_be.sh diff --git a/docker/runtime/be/resource/entry_point.sh b/docker/runtime/be/resource/entry_point.sh old mode 100644 new mode 100755 diff --git a/docker/runtime/be/resource/init_be.sh b/docker/runtime/be/resource/init_be.sh index 5c8a4dd0563362..42afd1f6754a02 100644 --- a/docker/runtime/be/resource/init_be.sh +++ b/docker/runtime/be/resource/init_be.sh @@ -161,7 +161,7 @@ _main() { fi check_be_status doris_note "Ready to start BE!" - start_be.sh --console & + ${DORIS_HOME}/be/bin/start_be.sh --console & child_pid=$! wait $child_pid exec "$@" diff --git a/docker/runtime/broker/Dockerfile b/docker/runtime/broker/Dockerfile index bf1e6650f7ef36..827b635d24f084 100644 --- a/docker/runtime/broker/Dockerfile +++ b/docker/runtime/broker/Dockerfile @@ -1,4 +1,3 @@ -#!/bin/bash # 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 @@ -16,26 +15,45 @@ # specific language governing permissions and limitations # under the License. -# choose a base image -FROM openjdk:8u342-jdk +# how to use Dockerfile. +# this is dockerfile for build doris broker image on amd64. +# when build youself image. +# 1. pull binary from official website and decompress into resource directory that the level equals with Dockerfile_broker_ubuntu. +# 2. untar xxxx.tar.gz in resource directory, update the dockerfile field `apache-doris-xxx`, replace with real version. +# 3. run commad docker build -t xxx.doris.broker:xx -f Dockerfile_broker_ubuntu. + +# we have support buildx for amd64 and arm64 architecture image build. +# get the binary from doris github and utar into resource, update the directory as apache-`version(example:2.0.1)`-bin-`architecture(amd64/arm64)` mode. +# + +FROM ubuntu:22.04 + +ARG TARGETARCH + +ARG DORIS_VERSION="x.x.x" + +RUN apt-get update -y && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends \ + patchelf gdb binutils binutils-common mysql-client \ + curl wget less vim htop iproute2 numactl jq iotop sysstat \ + tcpdump iputils-ping dnsutils strace lsof blktrace tzdata \ + bpfcc-tools linux-headers-realtime linux-tools-realtime silversearcher-ag \ + net-tools openjdk-8-jdk && \ + rm -rf /var/lib/apt/lists/* # set environment variables -ENV JAVA_HOME="/usr/local/openjdk-8/" \ - PATH="/opt/apache-doris/broker/bin:$PATH" +ENV JAVA_HOME=/usr/lib/jvm/java-8-openjdk-${TARGETARCH:-amd64} + +# apache-doris/broker from doris release xxxx.tar.gz +ADD resource/apache-doris-${DORIS_VERSION}-bin-${TARGETARCH:-amd64}/extensions/apache_hdfs_broker /opt/apache-doris/apache_hdfs_broker + +COPY resource/broker_*.sh /opt/apache-doris/ + +RUN chmod +x /opt/apache-doris/broker_*.sh -# Download the software to the mirror, where the broker directory is synchronously compressed to the binary package of FE, -# which needs to be decompressed and repackaged by itself, and can be replaced as needed -ADD ./resource/apache_hdfs_broker.tar.gz /opt/ +ADD ./resource/init_broker.sh /usr/local/bin/ -# deploy software -RUN apt-get update && \ - apt-get install -y default-mysql-client && \ - apt-get clean && \ - mkdir /opt/apache-doris && \ - cd /opt && \ - mv apache_hdfs_broker /opt/apache-doris/broker +RUN chmod 755 /usr/local/bin/init_broker.sh -ADD ./resource/init_broker.sh /opt/apache-doris/broker/bin -RUN chmod 755 /opt/apache-doris/broker/bin/init_broker.sh +WORKDIR /opt/apache-doris -ENTRYPOINT ["/opt/apache-doris/broker/bin/init_broker.sh"] +ENTRYPOINT ["bash","init_broker.sh"] diff --git a/docker/runtime/broker/resource/broker_entrypoint.sh b/docker/runtime/broker/resource/broker_entrypoint.sh new file mode 100755 index 00000000000000..ad06d6c958c3ae --- /dev/null +++ b/docker/runtime/broker/resource/broker_entrypoint.sh @@ -0,0 +1,222 @@ +#!/bin/bash +# 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. + +# the fe query port for mysql. +FE_QUERY_PORT=${FE_QUERY_PORT:-9030} +# timeout for probe fe master. +PROBE_TIMEOUT=60 +# interval time to probe fe. +PROBE_INTERVAL=2 +# ipc port for fe/be communicate with broker. +IPC_PORT=8000 +# fqdn or ip +MY_SELF= +MY_IP=`hostname -i` +MY_HOSTNAME=`hostname -f` +DORIS_ROOT=${DORIS_ROOT:-"/opt/apache-doris"} +AUTH_PATH="/etc/basic_auth" +DORIS_HOME=${DORIS_ROOT}/apache_hdfs_broker +BK_CONFIG=$DORIS_HOME/conf/apache_hdfs_broker.conf +BK_NAME=broker001 +# represents self in fe meta or not. +REGISTERED=false + +DB_ADMIN_USER=${USER:-"root"} + +DB_ADMIN_PASSWD=$PASSWD + +log_stderr() +{ + echo "[`date`] $@" >&2 +} + +parse_confval_from_bk_conf() +{ + # a naive script to grep given confkey from broker conf file + # assume conf format: ^\s*\s*=\s*\s*$ + local confkey=$1 + local confvalue=`grep "\<$confkey\>" $BK_CONFIG | grep -v '^\s*#' | sed 's|^\s*'$confkey'\s*=\s*\(.*\)\s*$|\1|g'` + echo "$confvalue" +} + +update_conf_from_configmap() +{ + if [[ "x$CONFIGMAP_MOUNT_PATH" == "x" ]] ; then + log_stderr '[info] Empty $CONFIGMAP_MOUNT_PATH env var, skip it!' + return 0 + fi + if ! test -d $CONFIGMAP_MOUNT_PATH ; then + log_stderr "[info] $CONFIGMAP_MOUNT_PATH not exist or not a directory, ignore ..." + return 0 + fi + local tgtconfdir=$DORIS_HOME/conf + for conffile in `ls $CONFIGMAP_MOUNT_PATH` + do + log_stderr "[info] Process conf file $conffile ..." + local tgt=$tgtconfdir/$conffile + if test -e $tgt ; then + # make a backup + mv -f $tgt ${tgt}.bak + fi + ln -sfT $CONFIGMAP_MOUNT_PATH/$conffile $tgt + done +} + + + +# get all brokers info to check self exist or not. +show_brokers(){ + local svc=$1 + brokers=`timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -uroot --skip-column-names --batch -e 'SHOW BROKER;' 2>&1` + if echo $brokers | grep -w "1045" | grep -q -w "28000" &>/dev/null ; then + brokers=`timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e 'SHOW BROKER;' 2>&1` + fi + echo "$brokers" + + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e 'SHOW BROKER;' + #else + # timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER --skip-column-names --batch -e 'SHOW BROKER;' + #fi +} + + + ## ALTER SYSTEM ADD BROKER broker_name "broker_host1:broker_ipc_port1","broker_host2:broker_ipc_port2",...; + +# get all registered fe in cluster, for check the fe have `MASTER`. +function show_frontends() +{ + local addr=$1 + frontends=`timeout 15 mysql --connect-timeout 2 -h $addr -P $FE_QUERY_PORT -uroot --batch -e 'show frontends;' 2>&1` + if echo $frontends | grep -w "1045" | grep -q -w "28000" &>/dev/nulll ; then + frontends=`timeout 15 mysql --connect-timeout 2 -h $addr -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --batch -e 'show frontends;'` + fi + echo "$frontends" + + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # timeout 15 mysql --connect-timeout 2 -h $addr -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e 'show frontends;' + #else + # timeout 15 mysql --connect-timeout 2 -h $addr -P $FE_QUERY_PORT -u$DB_ADMIN_USER --skip-column-names --batch -e 'show frontends;' + #fi +} + +collect_env_info() +{ + # IPC_PORT from conf file + local ipc_port=`parse_confval_from_bk_conf "broker_ipc_port"` + if [[ "x$ipc_port" != "x" ]] ; then + IPC_PORT=$ipc_port + fi + + if [[ "x$HOST_TYPE" == "xIP" ]] ; then + MY_SELF=$MY_IP + else + MY_SELF=$MY_HOSTNAME + fi +} + +add_self() +{ + local svc=$1 + start=`date +%s` + local timeout=$PROBE_TIMEOUT + + while true + do + memlist=`show_brokers $svc` + if echo "$memlist" | grep -q -w "$MY_SELF" &>/dev/null ; then + log_stderr "[info] Check myself ($MY_SELF:$IPC_PORT) exist in FE start broker directly ..." + break; + fi + + # check fe cluster have master, if fe have not master wait. + fe_memlist=`show_frontends $svc` + + #local leader=`echo "$fe_memlist" | grep '\' | awk -F '\t' '{if ($8=="true") print $2}'` + local pos=`echo "$fe_memlist" | grep '\' | awk -F '\t' '{for(i=1;i' | awk -v p="$pos" -F '\t' '{if ($p=="true") print $2}'` + if [[ "x$leader" != "x" ]]; then + log_stderr "[info] myself ($MY_SELF:$IPC_PORT) not exist in FE and fe have leader register myself into fe..." + + add_result=`timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -uroot --skip-column-names --batch -e "ALTER SYSTEM ADD BROKER $BK_NAME \"$MY_SELF:$IPC_PORT\";" 2>&1` + if echo $add_result | grep -w "1045" | grep -q -w "28000" &>/dev/null ; then + timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e "ALTER SYSTEM ADD BROKER $BK_NAME \"$MY_SELF:$IPC_PORT\";" + fi + + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e "ALTER SYSTEM ADD BROKER $BK_NAME \"$MY_SELF:$IPC_PORT\";" + #else + # timeout 15 mysql --connect-timeout 2 -h $svc -P $FE_QUERY_PORT -u$DB_ADMIN_USER --skip-column-names --batch -e "ALTER SYSTEM ADD BROKER $BK_NAME \"$MY_SELF:$IPC_PORT\";" + #fi + + let "expire=start+timeout" + now=`date +%s` + if [[ $expire -le $now ]] ; then + log_stderr "[error] exit probe master for probing timeout." + return 0 + fi + else + log_stderr "[info] not have leader wait fe cluster select a master, sleep 2s..." + sleep $PROBE_INTERVAL + fi + done +} + +# check be exist or not, if exist return 0, or register self in fe cluster. when all fe address failed exit script. +# `xxx1:port,xxx2:port` as parameter to function. +function check_and_register() +{ + addrs=$1 + local addrArr=(${addrs//,/ }) + for addr in ${addrArr[@]} + do + add_self $addr + done + + if [[ $REGISTERED ]]; then + return 0 + else + exit 1 + fi +} + +resolve_password_from_secret() +{ + if [[ -f "$AUTH_PATH/password" ]]; then + DB_ADMIN_PASSWD=`cat $AUTH_PATH/password` + fi + + if [[ -f "$AUTH_PATH/username" ]]; then + DB_ADMIN_USER=`cat $AUTH_PATH/username` + fi +} + +fe_addrs=$1 +if [[ "x$fe_addrs" == "x" ]]; then + echo "need fe address as paramter!" + echo " Example $0 " + exit 1 +fi + +update_conf_from_configmap +resolve_password_from_secret +collect_env_info +#add_self $fe_addr || exit $? +check_and_register $fe_addrs +log_stderr "run start_broker.sh" +$DORIS_HOME/bin/start_broker.sh diff --git a/docker/runtime/broker/resource/broker_is_alive.sh b/docker/runtime/broker/resource/broker_is_alive.sh new file mode 100755 index 00000000000000..0b55c746afad98 --- /dev/null +++ b/docker/runtime/broker/resource/broker_is_alive.sh @@ -0,0 +1,39 @@ +#!/bin/bash +# 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. + + +log_stderr() +{ + echo "[`date`] $@" >&2 +} + +rpc_port=$1 +if [[ "x$rpc_port" == "x" ]]; then + echo "need broker rpc_port as paramter!" + exit 1 +fi + +netstat -nltu | grep ":$rpc_port " > /dev/null + +if [ $? -eq 0 ]; then +# log_stderr "broker ($rpc_port)alive,ProbeHandler ExecAction get exit 0" + exit 0 +else +# log_stderr "broker($rpc_port)not alive,ProbeHandler ExecAction get exit 1" + exit 1 +fi diff --git a/docker/runtime/broker/resource/broker_prestop.sh b/docker/runtime/broker/resource/broker_prestop.sh new file mode 100755 index 00000000000000..a0081dec6d7bf3 --- /dev/null +++ b/docker/runtime/broker/resource/broker_prestop.sh @@ -0,0 +1,21 @@ +#!/bin/bash +# 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. + +DORIS_ROOT=${DORIS_ROOT:-"/opt/apache-doris"} +DORIS_HOME=${DORIS_ROOT}/apache_hdfs_broker +$DORIS_HOME/bin/stop_broker.sh diff --git a/docker/runtime/broker/resource/init_broker.sh b/docker/runtime/broker/resource/init_broker.sh index 958449d8354a4f..7567e02e3f07b6 100644 --- a/docker/runtime/broker/resource/init_broker.sh +++ b/docker/runtime/broker/resource/init_broker.sh @@ -183,7 +183,7 @@ _main() { register_broker_to_fe check_broker_status doris_note "Ready to start BROKER!" - start_broker.sh + ${DORIS_HOME}/apache_hdfs_broker/bin/start_broker.sh exec "$@" } diff --git a/docker/runtime/fe/Dockerfile b/docker/runtime/fe/Dockerfile index 6ef16b9ccd77bf..293f72b3053dc1 100644 --- a/docker/runtime/fe/Dockerfile +++ b/docker/runtime/fe/Dockerfile @@ -1,4 +1,3 @@ -#!/bin/bash # 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 @@ -52,6 +51,11 @@ ADD resource/init_fe.sh /usr/local/bin/ COPY resource/fe_*.sh /opt/apache-doris/ +COPY --from=selectdb/doris-debug-ubuntu:latest /doris-debug /opt/apache-doris/ + +# when you use beijing zone, please enable the set. +# RUN ln -sf /usr/share/zoneinfo/Asia/Shanghai /etc/localtime + WORKDIR /opt/apache-doris ENTRYPOINT ["bash","init_fe.sh"] diff --git a/docker/runtime/fe/resource/fe_check_master.sh b/docker/runtime/fe/resource/fe_check_master.sh new file mode 100755 index 00000000000000..dfa814014a8fa0 --- /dev/null +++ b/docker/runtime/fe/resource/fe_check_master.sh @@ -0,0 +1,42 @@ +#!/bin/bash +# 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. + +check_first_fe_status() +{ + local start_time=`date +%s` + local expire_timeout=120 + local helper=$1 + while true; do + output=`timeout 15 mysql --connect-timeout 2 -h $helper -P $FE_QUERY_PORT -u root --skip-column-names --batch -e "SHOW FRONTENDS;"` + if [[ "x$output" != "x" ]]; then + return 0 + fi + + let "expire=start_time+expire_timeout" + local now=`date +%s` + if [[ $expire -le $now ]]; then + echo "[`date`] the first container is not started" >& 2 + exit 1 + fi + + sleep 2 + done +} + +check_first_fe_status $1 + diff --git a/docker/runtime/fe/resource/fe_entrypoint.sh b/docker/runtime/fe/resource/fe_entrypoint.sh new file mode 100755 index 00000000000000..16db02aa571c08 --- /dev/null +++ b/docker/runtime/fe/resource/fe_entrypoint.sh @@ -0,0 +1,395 @@ +#!/bin/bash +# 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. + +DORIS_ROOT=${DORIS_ROOT:-"/opt/apache-doris"} +# if config secret for basic auth about operate node of doris, the path must be `/etc/doris/basic_auth`. This is set by operator and the key of password must be `password`. +AUTH_PATH="/etc/basic_auth" +# annotations_for_recovery_start +ANNOTATION_PATH="/etc/podinfo/annotations" +RECOVERY_KEY="" +# fe location +DORIS_HOME=${DORIS_ROOT}/fe +# participant election number of fe. +ELECT_NUMBER=${ELECT_NUMBER:=3} +# query port for mysql connection. +QUERY_PORT=${FE_QUERY_PORT:-9030} +EDIT_LOG_PORT=9010 +# location of fe config store. +FE_CONFFILE=$DORIS_HOME/conf/fe.conf +# represents the type for fe communication: domain or IP. +START_TYPE= +# the master node in fe cluster. +FE_MASTER= +# pod ordinal of statefulset deployed pod. +POD_INDEX= +# probe interval: 2 seconds +PROBE_INTERVAL=2 +# timeout for probe master: 60 seconds +PROBE_MASTER_POD0_TIMEOUT=60 # at most 30 attempts, no less than the times needed for an election +# no-0 ordinal pod timeout for probe master: 90 times +PROBE_MASTER_PODX_TIMEOUT=180 # at most 90 attempts +# administrator for administrate the cluster. +DB_ADMIN_USER=${USER:-"root"} + +DB_ADMIN_PASSWD=$PASSWD +# myself as IP or FQDN +MYSELF= + +function log_stderr() +{ + echo "[`date`] $@" >& 2 +} + +#parse the `$FE_CONFFILE` file, passing the key need resolve as parameter. +parse_confval_from_fe_conf() +{ + # a naive script to grep given confkey from fe conf file + # assume conf format: ^\s*\s*=\s*\s*$ + local confkey=$1 + local confvalue=`grep "\<$confkey\>" $FE_CONFFILE | grep -v '^\s*#' | sed 's|^\s*'$confkey'\s*=\s*\(.*\)\s*$|\1|g'` + echo "$confvalue" +} + +# when image exist int doris-meta, use exist meta to start. +function start_fe_with_meta() +{ + log_stderr "start with meta run start_fe.sh" + # the server will start in the current terminal session, and the log output and console interaction will be printed to that terminal + # befor doris 2.0.2 ,doris start with : start_xx.sh + # sine doris 2.0.2 ,doris start with : start_xx.sh --console doc: https://doris.apache.org/docs/dev/install/standard-deployment/#version--202 + $DORIS_HOME/fe/bin/start_fe.sh --console +} + +collect_env_info() +{ + # set POD_IP, POD_FQDN, POD_INDEX, EDIT_LOG_PORT, QUERY_PORT + if [[ "x$POD_IP" == "x" ]] ; then + POD_IP=`hostname -i | awk '{print $1}'` + fi + + if [[ "x$POD_FQDN" == "x" ]] ; then + POD_FQDN=`hostname -f` + fi + + # example: fe-sr-deploy-1.fe-svc.kc-sr.svc.cluster.local + POD_INDEX=`echo $POD_FQDN | awk -F'.' '{print $1}' | awk -F'-' '{print $NF}'` + + # since selectdb/doris.fe-ubuntu:2.0.2 , fqdn is forced to open without using ip method(enable_fqdn_mode = true). + # Therefore START_TYPE is true + START_TYPE=`parse_confval_from_fe_conf "enable_fqdn_mode"` + + if [[ "x$START_TYPE" == "xtrue" ]]; then + MYSELF=$POD_FQDN + else + MYSELF=$POD_IP + fi + + # edit_log_port from conf file + local edit_log_port=`parse_confval_from_fe_conf "edit_log_port"` + if [[ "x$edit_log_port" != "x" ]] ; then + EDIT_LOG_PORT=$edit_log_port + fi + + # query_port from conf file + local query_port=`parse_confval_from_fe_conf "query_port"` + if [[ "x$query_port" != "x" ]] ; then + QUERY_PORT=$query_port + fi +} + +# get all registered fe in cluster. +function show_frontends() +{ + local addr=$1 + # fist start use root and no password check. avoid use pre setted username and password. + frontends=`timeout 15 mysql --connect-timeout 2 -h $addr -P $QUERY_PORT -uroot --batch -e 'show frontends;' 2>&1` + log_stderr "[info] use root no password show frotends result '$frontends'" + if echo $frontends | grep -w "1045" | grep -q -w "28000" &>/dev/null ; then + log_stderr "[info] use username and password that configured show frontends." + frontends=`timeout 15 mysql --connect-timeout 2 -h $addr -P $QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --batch -e 'show frontends;' 2>&1` + fi + echo "$frontends" + + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # timeout 15 mysql --connect-timeout 2 -h $addr -P $QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --batch -e 'show frontends;' + #else + # timeout 15 mysql --connect-timeout 2 -h $addr -P $QUERY_PORT -u$DB_ADMIN_USER --batch -e 'show frontends;' + #fi +} + +# add myself in cluster for FOLLOWER. +function add_self_follower() +{ + add_result=`mysql --connect-timeout 2 -h $FE_MASTER -P $QUERY_PORT -uroot --skip-column-names --batch -e "ALTER SYSTEM ADD FOLLOWER \"$MYSELF:$EDIT_LOG_PORT\";" 2>&1` + log_stderr "[info] use root no password to add follower result '$add_result'" + if echo $add_result | grep -w "1045" | grep -q -w "28000" &>/dev/null ; then + log_stderr "[info] use username and password that configured to add self as follower." + mysql --connect-timeout 2 -h $FE_MASTER -P $QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e "ALTER SYSTEM ADD FOLLOWER \"$MYSELF:$EDIT_LOG_PORT\";" + fi + + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # mysql --connect-timeout 2 -h $FE_MASTER -P $QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e "ALTER SYSTEM ADD FOLLOWER \"$MYSELF:$EDIT_LOG_PORT\";" + #else + # mysql --connect-timeout 2 -h $FE_MASTER -P $QUERY_PORT -u$DB_ADMIN_USER --skip-column-names --batch -e "ALTER SYSTEM ADD FOLLOWER \"$MYSELF:$EDIT_LOG_PORT\";" + #fi +} + +# add myself in cluster for OBSERVER. +function add_self_observer() +{ + add_result=`mysql --connect-timeout 2 -h $FE_MASTER -P $QUERY_PORT -uroot --skip-column-names --batch -e "ALTER SYSTEM ADD OBSERVER \"$MYSELF:$EDIT_LOG_PORT\";" 2>&1` + log_stderr "[info] use root no password to add self as observer result '$add_result'." + if echo $add_result | grep -w "1045" | grep -q -w "28000" &>/dev/null ; then + log_stderr "[info] use username and password that configed to add self as observer." + mysql --connect-timeout 2 -h $FE_MASTER -P $QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e "ALTER SYSTEM ADD OBSERVER \"$MYSELF:$EDIT_LOG_PORT\";" + fi + + #if [[ "x$DB_ADMIN_PASSWD" != "x" ]]; then + # mysql --connect-timeout 2 -h $FE_MASTER -P $QUERY_PORT -u$DB_ADMIN_USER -p$DB_ADMIN_PASSWD --skip-column-names --batch -e "ALTER SYSTEM ADD OBSERVER \"$MYSELF:$EDIT_LOG_PORT\";" + #else + # mysql --connect-timeout 2 -h $FE_MASTER -P $QUERY_PORT -u$DB_ADMIN_USER --skip-column-names --batch -e "ALTER SYSTEM ADD OBSERVER \"$MYSELF:$EDIT_LOG_PORT\";" + #fi +} + +# `dori-meta/image` not exist start as first time. +function start_fe_no_meta() +{ + # the server will start in the current terminal session, and the log output and console interaction will be printed to that terminal + # befor doris 2.0.2 ,doris start with : start_xx.sh + # sine doris 2.0.2 ,doris start with : start_xx.sh --console doc: https://doris.apache.org/docs/dev/install/standard-deployment/#version--202 + local opts="--console" + local start=`date +%s` + local has_member=false + local member_list= + if [[ "x$FE_MASTER" != "x" ]] ; then + opts+=" --helper $FE_MASTER:$EDIT_LOG_PORT" + local start=`date +%s` + while true + do + # for statefulset manage fe pods, when `ELECT_NUMBER` greater than `POD_INDEX` + if [[ ELECT_NUMBER -gt $POD_INDEX ]]; then + log_stderr "Add myself($MYSELF:$EDIT_LOG_PORT) to master as follower ..." + add_self_follower + else + log_stderr "Add myself($MYSELF:$EDIT_LOG_PORT) to master as observer ..." + add_self_observer + fi + # if successfully exit circulate register logic and start fe. + if show_frontends $addr | grep -q -w "$MYSELF" &>/dev/null ; then + break; + fi + + local now=`date +%s` + let "expire=start+30" # 30s timeout + # if timeout for register self exit 1. + if [[ $expire -le $now ]] ; then + log_stderr "Timed out, abort!" + exit 1 + fi + + log_stderr "Sleep a while and retry adding ..." + sleep $PROBE_INTERVAL + done + fi + log_stderr "first start with no meta run start_fe.sh with additional options: '$opts'" + $DORIS_HOME/bin/start_fe.sh $opts +} + +# the ordinal is 0, probe timeout as 60s, when have not meta and not `MASTER` in fe cluster, 0 start as master. +probe_master_for_pod() +{ + # possible to have no result at all, because myself is the first FE instance in the cluster + local svc=$1 + local start=`date +%s` + local has_member=false + local memlist= + while true + do + memlist=`show_frontends $svc` + # find master by column `IsMaster` + local pos=`echo "$memlist" | grep '\' | awk -F '\t' '{for(i=1;i' | awk -v p="$pos" -F '\t' '{if ($p=="true") print $2}'` + + log_stderr "'IsMaster' sequence in columns is $pos, master=$master." + if [[ "x$master" == "x" ]]; then + log_stderr "[info] resolve the eighth column for finding master !" + master=`echo "$memlist" | grep '\' | awk -F '\t' '{if ($8=="true") print $2}'` + fi + + if [[ "x$master" == "x" ]]; then + # compatible 2.1.0 + log_stderr "[info] resoluve the ninth column for finding master!" + master=`echo "$memlist" | grep '\' | awk -F '\t' '{if ($9=="true") print $2}'` + fi + + if [[ "x$master" != "x" ]] ; then + # has master, done + log_stderr "Find master: $master!" + FE_MASTER=$master + return 0 + fi + + # show frontens has members + if [[ "x$memlist" != "x" && "x$pos" != "x" ]] ; then + # has member list ever before + has_member=true + fi + + # no master yet, check if needs timeout and quit + log_stderr "[info] master is not elected, has_member: $has_member, this may be first start or master changing, wait $PROBE_INTERVAL s to next probe..." + local timeout=$PROBE_MASTER_POD0_TIMEOUT + if "$has_member" == true || [ "$POD_INDEX" -ne "0" ] ; then + # set timeout to the same as PODX since there are other members + timeout=$PROBE_MASTER_PODX_TIMEOUT + fi + + local now=`date +%s` + let "expire=start+timeout" + if [[ $expire -le $now ]] ; then + log_stderr "[info] exit probe master for probing timeout, if it is the first pod will start as master. .." + # empty FE_MASTER + FE_MASTER="" + return 0 + fi + sleep $PROBE_INTERVAL + done +} + +# when not meta exist, fe start should probe +probe_master() +{ + local svc=$1 + # resolve svc as array. + local addArr=${svc//,/ } + for addr in ${addArr[@]} + do + # if have master break for register or check. + if [[ "x$FE_MASTER" != "x" ]]; then + break + fi + + # find master under current service and set to FE_MASTER + probe_master_for_pod $addr + done + + # if first pod assume first start should as master. others first start have not master exit. + if [[ "x$FE_MASTER" == "x" ]]; then + if [[ "$POD_INDEX" -eq 0 ]]; then + return 0 + else + log_stderr "the pod can't connect to pod 0, the network may be not work. please verify domain connectivity with two pods in different node and verify the pod 0 ready or not." + exit 1 + fi + fi +} + +function add_fqdn_config() +{ + # TODO(user):since selectdb/doris.fe-ubuntu:2.0.2 , `enable_fqdn_mode` is forced to set `true` for starting doris. (enable_fqdn_mode = true). + local enable_fqdn=`parse_confval_from_fe_conf "enable_fqdn_mode"` + log_stderr "enable_fqdn is : $enable_fqdn" + if [[ "x$enable_fqdn" != "xtrue" ]] ; then + log_stderr "add enable_fqdn_mode = true to ${DORIS_HOME}/conf/fe.conf" + echo "enable_fqdn_mode = true" >>${DORIS_HOME}/conf/fe.conf + fi +} + +update_conf_from_configmap() +{ + if [[ "x$CONFIGMAP_MOUNT_PATH" == "x" ]] ; then + log_stderr '[info] Empty $CONFIGMAP_MOUNT_PATH env var, skip it!' + add_fqdn_config + return 0 + fi + if ! test -d $CONFIGMAP_MOUNT_PATH ; then + log_stderr "[info] $CONFIGMAP_MOUNT_PATH not exist or not a directory, ignore ..." + add_fqdn_config + return 0 + fi + local tgtconfdir=$DORIS_HOME/conf + for conffile in `ls $CONFIGMAP_MOUNT_PATH` + do + log_stderr "[info] Process conf file $conffile ..." + local tgt=$tgtconfdir/$conffile + if test -e $tgt ; then + # make a backup + mv -f $tgt ${tgt}.bak + fi + ln -sfT $CONFIGMAP_MOUNT_PATH/$conffile $tgt + done + add_fqdn_config +} + +# resolve password for root +resolve_password_from_secret() +{ + if [[ -f "$AUTH_PATH/password" ]]; then + DB_ADMIN_PASSWD=`cat $AUTH_PATH/password` + fi + + if [[ -f "$AUTH_PATH/username" ]]; then + DB_ADMIN_USER=`cat $AUTH_PATH/username` + fi +} + +start_fe_with_meta() +{ + # the server will start in the current terminal session, and the log output and console interaction will be printed to that terminal + # befor doris 2.0.2 ,doris start with : start_xx.sh + local opts="--console" + local recovery=`grep "\" $ANNOTATION_PATH | grep -v '^\s*#' | sed 's|^\s*'$confkey'\s*=\s*\(.*\)\s*$|\1|g'` + if [[ "x$recovery" != "x" ]]; then + opts=${opts}" --metadata_failure_recovery" + fi + + log_stderr "start with meta run start_fe.sh with additional options: '$opts'" + # sine doris 2.0.2 ,doris start with : start_xx.sh --console doc: https://doris.apache.org/docs/dev/install/standard-deployment/#version--202 + $DORIS_HOME/bin/start_fe.sh $opts +} + +print_vlsn() +{ + local doirs_meta_path=`parse_confval_from_fe_conf "meta_dir"` + if [[ "x$doirs_meta_path" == "x" ]] ; then + doris_meta_path="/opt/apache-doris/fe/doris-meta" + fi + + vlsns=`grep -rn "VLSN:" $doris_meta_path/bdb/je* | tail -n 10` + echo "$vlsns" +} + +fe_addrs=$1 +if [[ "x$fe_addrs" == "x" ]]; then + echo "need fe address as parameter!" + exit +fi + +update_conf_from_configmap +# resolve password for root to manage nodes in doris. +resolve_password_from_secret +if [[ -f "/opt/apache-doris/fe/doris-meta/image/ROLE" ]]; then + log_stderr "start fe with exist meta." + ./doris-debug --component fe + print_vlsn + start_fe_with_meta +else + log_stderr "first start fe with meta not exist." + collect_env_info + probe_master $fe_addrs + start_fe_no_meta +fi diff --git a/docker/runtime/fe/resource/fe_prestop.sh b/docker/runtime/fe/resource/fe_prestop.sh new file mode 100755 index 00000000000000..4a2eb59df90252 --- /dev/null +++ b/docker/runtime/fe/resource/fe_prestop.sh @@ -0,0 +1,21 @@ +#!/bin/bash +# 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. + +DORIS_ROOT=${DORIS_ROOT:-"/opt/apache-doris"} +DORIS_HOME=${DORIS_ROOT}/fe +$DORIS_HOME/bin/stop_fe.sh diff --git a/docker/runtime/fe/resource/init_fe.sh b/docker/runtime/fe/resource/init_fe.sh index c1e1a2c282b0d8..8cbd80ea237fd2 100644 --- a/docker/runtime/fe/resource/init_fe.sh +++ b/docker/runtime/fe/resource/init_fe.sh @@ -295,7 +295,7 @@ _main() { docker_required_variables_env trap 'cleanup' SIGTERM SIGINT if [[ $RUN_TYPE == "K8S" ]]; then - start_fe.sh --console & + ${DORIS_HOME}/fe/bin/start_fe.sh --console & child_pid=$! else docker_setup_env @@ -310,10 +310,10 @@ _main() { doris_note "Ready to start CURRENT_FE!" if [ $CURRENT_FE_IS_MASTER == true ]; then - start_fe.sh --console & + ${DORIS_HOME}/fe/bin/start_fe.sh --console & child_pid=$! else - start_fe.sh --helper ${MASTER_FE_IP}:${MASTER_FE_EDIT_PORT} --console & + ${DORIS_HOME}/fe/bin/start_fe.sh --helper ${MASTER_FE_IP}:${MASTER_FE_EDIT_PORT} --console & child_pid=$! fi fi From 03bd2a337d4a56ea9c91673b3bd4ae518ed10f20 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Fri, 12 Apr 2024 19:54:36 +0800 Subject: [PATCH 36/71] [enhancement](Nereids) Enable parse sql from sql cache (#33262) Before this pr, the query must pass through parser, analyzer, rewriter, optimizer and translator, then we can check whether this query can use sql cache, if the query is too long, or the number of join tables too big, the plan time usually >= 500ms. This pr reduce this time by skip the fashion plan path, because we can reuse the previous physical plan and query result if no any changed. In some cases we should not parse sql from sql cache, e.g. table structure changed, data changed, user policies changed, privileges changed, contains non-deterministic functions, and user variables changed. In my test case: query a view which has lots of join and union, and the tables has empty partition, the query latency is about 3ms. if not parse sql from sql cache, the plan time is about 550ms ## Features 1. use Config.sql_cache_manage_num to control how many sql cache be reused in on fe 2. if explain plan appear some plans contains `LogicalSqlCache` or `PhysicalSqlCache`, it means the query can use sql cache, like this: ```sql mysql> set enable_sql_cache=true; Query OK, 0 rows affected (0.00 sec) mysql> explain physical plan select * from test.t; +----------------------------------------------------------------------------------+ | Explain String(Nereids Planner) | +----------------------------------------------------------------------------------+ | cost = 3.135 | | PhysicalResultSink[53] ( outputExprs=[c1#0, c2#1] ) | | +--PhysicalDistribute[50]@0 ( stats=3, distributionSpec=DistributionSpecGather ) | | +--PhysicalOlapScan[t]@0 ( stats=3 ) | +----------------------------------------------------------------------------------+ 4 rows in set (0.02 sec) mysql> select * from test.t; +------+------+ | c1 | c2 | +------+------+ | 1 | 2 | | -2 | -2 | | NULL | 30 | +------+------+ 3 rows in set (0.05 sec) mysql> explain physical plan select * from test.t; +-------------------------------------------------------------------------------------------+ | Explain String(Nereids Planner) | +-------------------------------------------------------------------------------------------+ | cost = 0.0 | | PhysicalSqlCache[2] ( queryId=78511f515cda466b-95385d892d6c68d0, backend=127.0.0.1:9050 ) | | +--PhysicalResultSink[52] ( outputExprs=[c1#0, c2#1] ) | | +--PhysicalDistribute[49]@0 ( stats=3, distributionSpec=DistributionSpecGather ) | | +--PhysicalOlapScan[t]@0 ( stats=3 ) | +-------------------------------------------------------------------------------------------+ 5 rows in set (0.01 sec) ``` --- .../java/org/apache/doris/common/Config.java | 10 +- .../java/org/apache/doris/common/Pair.java | 4 + .../doris/blockrule/SqlBlockRuleMgr.java | 8 +- .../java/org/apache/doris/catalog/Env.java | 10 + .../doris/common/NereidsSqlCacheManager.java | 360 ++++++++++++ .../doris/common/profile/SummaryProfile.java | 2 +- .../apache/doris/nereids/CascadesContext.java | 6 +- .../apache/doris/nereids/NereidsPlanner.java | 223 ++++--- .../apache/doris/nereids/SqlCacheContext.java | 353 +++++++++++ .../doris/nereids/StatementContext.java | 118 +++- .../doris/nereids/parser/NereidsParser.java | 113 ++++ .../rules/analysis/BindExpression.java | 16 +- .../nereids/rules/analysis/BindRelation.java | 120 ++-- .../rules/analysis/ExpressionAnalyzer.java | 73 ++- .../expression/rules/FunctionBinder.java | 4 +- .../rules/ReplaceVariableByLiteral.java | 13 +- .../rules/rewrite/CheckPrivileges.java | 12 +- .../expressions/ExpressionEvaluator.java | 13 +- .../functions/AggCombinerFunctionBuilder.java | 17 +- .../functions/BuiltinFunctionBuilder.java | 7 +- .../functions/FunctionBuilder.java | 10 +- .../functions/scalar/ConnectionId.java | 3 +- .../functions/scalar/CurrentUser.java | 3 +- .../functions/scalar/Database.java | 3 +- .../expressions/functions/scalar/User.java | 3 +- .../functions/table/TableValuedFunction.java | 4 +- .../functions/udf/AliasUdfBuilder.java | 5 +- .../functions/udf/JavaUdafBuilder.java | 21 +- .../functions/udf/JavaUdfBuilder.java | 7 +- .../expressions/literal/DateLiteral.java | 40 +- .../expressions/literal/DateTimeLiteral.java | 77 ++- .../literal/DateTimeV2Literal.java | 109 ++-- .../expressions/literal/DateV2Literal.java | 13 +- .../nereids/trees/plans/AbstractPlan.java | 30 +- .../doris/nereids/trees/plans/PlanType.java | 2 + .../nereids/trees/plans/TreeStringPlan.java | 112 ++++ .../nereids/trees/plans/algebra/SqlCache.java | 36 ++ .../plans/logical/LogicalCheckPolicy.java | 15 +- .../trees/plans/logical/LogicalSqlCache.java | 147 +++++ .../plans/physical/PhysicalSqlCache.java | 139 +++++ .../trees/plans/visitor/PlanVisitor.java | 8 + .../org/apache/doris/qe/ConnectProcessor.java | 191 ++++-- .../org/apache/doris/qe/StmtExecutor.java | 60 +- .../java/org/apache/doris/qe/cache/Cache.java | 19 +- .../apache/doris/qe/cache/CacheAnalyzer.java | 156 ++++- .../doris/qe/cache/CacheCoordinator.java | 5 +- .../org/apache/doris/qe/cache/SqlCache.java | 56 +- .../rules/analysis/FunctionRegistryTest.java | 2 +- .../regression/action/ExplainAction.groovy | 4 +- .../doris/regression/suite/Suite.groovy | 74 ++- regression-test/plugins/test_helper.groovy | 63 ++ .../cache/parse_sql_from_sql_cache.groovy | 550 ++++++++++++++++++ .../suites/query_p0/cache/sql_cache.groovy | 2 +- .../test_schema_change_duplicate.groovy | 1 - 54 files changed, 3025 insertions(+), 427 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/TreeStringPlan.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/SqlCache.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java create mode 100644 regression-test/plugins/test_helper.groovy create mode 100644 regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy 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 aacbf0bd2bc05a..f5d60377c62a18 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 @@ -17,8 +17,6 @@ package org.apache.doris.common; -import org.apache.doris.common.ConfigBase.ConfField; - public class Config extends ConfigBase { @ConfField(description = {"用户自定义配置文件的路径,用于存放 fe_custom.conf。该文件中的配置会覆盖 fe.conf 中的配置", @@ -2004,6 +2002,14 @@ public class Config extends ConfigBase { + "the old load statement will be degraded."}) public static boolean enable_nereids_load = false; + /** + * the plan cache num which can be reused for the next query + */ + @ConfField(mutable = false, varType = VariableAnnotation.EXPERIMENTAL, description = { + "当前默认设置为 100,用来控制控制NereidsSqlCacheManager管理的sql cache数量。", + "Now default set to 100, this config is used to control the number of " + + "sql cache managed by NereidsSqlCacheManager"}) + public static int sql_cache_manage_num = 100; /** * Maximum number of events to poll in each RPC. diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Pair.java b/fe/fe-common/src/main/java/org/apache/doris/common/Pair.java index a699284676131c..3a8b1940d59759 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Pair.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Pair.java @@ -43,6 +43,10 @@ private Pair(F first, S second) { this.second = second; } + public static Pair ofSame(K same) { + return new Pair<>(same, same); + } + public static Pair of(F first, S second) { return new Pair<>(first, second); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java index ca4e68a6ae2f9c..2eba1d4fae3385 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/blockrule/SqlBlockRuleMgr.java @@ -269,10 +269,10 @@ public void checkLimitations(Long partitionNum, Long tabletNum, Long cardinality return; } // match global rule - List globalRules = - nameToSqlBlockRuleMap.values().stream().filter(SqlBlockRule::getGlobal).collect(Collectors.toList()); - for (SqlBlockRule rule : globalRules) { - checkLimitations(rule, partitionNum, tabletNum, cardinality); + for (SqlBlockRule rule : nameToSqlBlockRuleMap.values()) { + if (rule.getGlobal()) { + checkLimitations(rule, partitionNum, tabletNum, cardinality); + } } // match user rule String[] bindSqlBlockRules = Env.getCurrentEnv().getAuth().getSqlBlockRules(user); 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 cf72de4ebf7290..dee9b54db6332b 100755 --- 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 @@ -105,6 +105,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.MetaNotFoundException; +import org.apache.doris.common.NereidsSqlCacheManager; import org.apache.doris.common.Pair; import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.UserException; @@ -530,6 +531,8 @@ public class Env { private DNSCache dnsCache; + private final NereidsSqlCacheManager sqlCacheManager; + public List getFrontendInfos() { List res = new ArrayList<>(); @@ -766,6 +769,9 @@ public Env(boolean isCheckpointCatalog) { this.mtmvService = new MTMVService(); this.insertOverwriteManager = new InsertOverwriteManager(); this.dnsCache = new DNSCache(); + this.sqlCacheManager = new NereidsSqlCacheManager( + Config.sql_cache_manage_num, Config.cache_last_version_interval_second + ); } public static void destroyCheckpoint() { @@ -6092,6 +6098,10 @@ public MasterDaemon getTabletStatMgr() { return tabletStatMgr; } + public NereidsSqlCacheManager getSqlCacheManager() { + return sqlCacheManager; + } + public void alterMTMVRefreshInfo(AlterMTMVRefreshInfo info) { AlterMTMV alter = new AlterMTMV(info.getMvName(), info.getRefreshInfo(), MTMVAlterOpType.ALTER_REFRESH_INFO); this.alter.processAlterMTMV(alter, false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java new file mode 100644 index 00000000000000..f3643046cade04 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java @@ -0,0 +1,360 @@ +// 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.common; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.View; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.metric.MetricRepo; +import org.apache.doris.mysql.privilege.DataMaskPolicy; +import org.apache.doris.mysql.privilege.RowFilterPolicy; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; +import org.apache.doris.nereids.SqlCacheContext.FullColumnName; +import org.apache.doris.nereids.SqlCacheContext.FullTableName; +import org.apache.doris.nereids.SqlCacheContext.ScanTable; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.analyzer.UnboundVariable; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.analysis.ExpressionAnalyzer; +import org.apache.doris.nereids.rules.analysis.UserAuthentication; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnFE; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Variable; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; +import org.apache.doris.proto.InternalService; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.cache.CacheAnalyzer; +import org.apache.doris.qe.cache.SqlCache; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.google.common.collect.ImmutableList; +import org.apache.commons.collections.CollectionUtils; + +import java.time.Duration; +import java.util.List; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** NereidsSqlCacheManager */ +public class NereidsSqlCacheManager { + // key: : + // value: CacheAnalyzer + private final Cache sqlCache; + + public NereidsSqlCacheManager(int sqlCacheNum, long cacheIntervalSeconds) { + sqlCache = Caffeine.newBuilder() + .maximumSize(sqlCacheNum) + .expireAfterAccess(Duration.ofSeconds(cacheIntervalSeconds)) + // auto evict cache when jvm memory too low + .softValues() + .build(); + } + + /** tryAddCache */ + public void tryAddCache( + ConnectContext connectContext, String sql, + CacheAnalyzer analyzer, boolean currentMissParseSqlFromSqlCache) { + Optional sqlCacheContextOpt = connectContext.getStatementContext().getSqlCacheContext(); + if (!sqlCacheContextOpt.isPresent()) { + return; + } + if (!(analyzer.getCache() instanceof SqlCache)) { + return; + } + SqlCacheContext sqlCacheContext = sqlCacheContextOpt.get(); + UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); + String key = currentUserIdentity.toString() + ":" + sql.trim(); + if (analyzer.getCache() instanceof SqlCache + && (currentMissParseSqlFromSqlCache || sqlCache.getIfPresent(key) == null)) { + SqlCache cache = (SqlCache) analyzer.getCache(); + sqlCacheContext.setCacheKeyMd5(cache.getOrComputeCacheMd5()); + sqlCacheContext.setSumOfPartitionNum(cache.getSumOfPartitionNum()); + sqlCacheContext.setLatestPartitionId(cache.getLatestId()); + sqlCacheContext.setLatestPartitionVersion(cache.getLatestVersion()); + sqlCacheContext.setLatestPartitionTime(cache.getLatestTime()); + sqlCacheContext.setCacheProxy(cache.getProxy()); + + for (ScanTable scanTable : analyzer.getScanTables()) { + sqlCacheContext.addScanTable(scanTable); + } + + sqlCache.put(key, sqlCacheContext); + } + } + + /** invalidateCache */ + public void invalidateCache(ConnectContext connectContext, String sql) { + UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); + String key = currentUserIdentity.toString() + ":" + sql.trim(); + sqlCache.invalidate(key); + } + + /** tryParseSql */ + public Optional tryParseSql(ConnectContext connectContext, String sql) { + UserIdentity currentUserIdentity = connectContext.getCurrentUserIdentity(); + Env env = connectContext.getEnv(); + String key = currentUserIdentity.toString() + ":" + sql.trim(); + SqlCacheContext sqlCacheContext = sqlCache.getIfPresent(key); + if (sqlCacheContext == null) { + return Optional.empty(); + } + + // LOG.info("Total size: " + GraphLayout.parseInstance(sqlCacheContext).totalSize()); + + // check table and view and their columns authority + if (privilegeChanged(connectContext, env, sqlCacheContext)) { + return invalidateCache(key); + } + if (tablesOrDataChanged(env, sqlCacheContext)) { + return invalidateCache(key); + } + if (viewsChanged(env, sqlCacheContext)) { + return invalidateCache(key); + } + if (usedVariablesChanged(sqlCacheContext)) { + return invalidateCache(key); + } + + LogicalEmptyRelation whateverPlan = new LogicalEmptyRelation(new RelationId(0), ImmutableList.of()); + if (nondeterministicFunctionChanged(whateverPlan, connectContext, sqlCacheContext)) { + return invalidateCache(key); + } + + // table structure and data not changed, now check policy + if (rowPoliciesChanged(currentUserIdentity, env, sqlCacheContext)) { + return invalidateCache(key); + } + if (dataMaskPoliciesChanged(currentUserIdentity, env, sqlCacheContext)) { + return invalidateCache(key); + } + + try { + Status status = new Status(); + InternalService.PFetchCacheResult cacheData = + SqlCache.getCacheData(sqlCacheContext.getCacheProxy(), + sqlCacheContext.getCacheKeyMd5(), sqlCacheContext.getLatestPartitionId(), + sqlCacheContext.getLatestPartitionVersion(), sqlCacheContext.getLatestPartitionTime(), + sqlCacheContext.getSumOfPartitionNum(), status); + + if (status.ok() && cacheData != null && cacheData.getStatus() == InternalService.PCacheStatus.CACHE_OK) { + List cacheValues = cacheData.getValuesList(); + String cachedPlan = sqlCacheContext.getPhysicalPlan(); + String backendAddress = SqlCache.findCacheBe(sqlCacheContext.getCacheKeyMd5()).getAddress(); + + MetricRepo.COUNTER_CACHE_HIT_SQL.increase(1L); + + LogicalSqlCache logicalSqlCache = new LogicalSqlCache( + sqlCacheContext.getQueryId(), sqlCacheContext.getColLabels(), + sqlCacheContext.getResultExprs(), cacheValues, backendAddress, cachedPlan); + return Optional.of(logicalSqlCache); + } + return Optional.empty(); + } catch (Throwable t) { + return Optional.empty(); + } + } + + private boolean tablesOrDataChanged(Env env, SqlCacheContext sqlCacheContext) { + long latestPartitionTime = sqlCacheContext.getLatestPartitionTime(); + long latestPartitionVersion = sqlCacheContext.getLatestPartitionVersion(); + + if (sqlCacheContext.hasUnsupportedTables()) { + return true; + } + + for (ScanTable scanTable : sqlCacheContext.getScanTables()) { + FullTableName fullTableName = scanTable.fullTableName; + TableIf tableIf = findTableIf(env, fullTableName); + if (!(tableIf instanceof OlapTable)) { + return true; + } + OlapTable olapTable = (OlapTable) tableIf; + long currentTableTime = olapTable.getVisibleVersionTime(); + long cacheTableTime = scanTable.latestTimestamp; + long currentTableVersion = olapTable.getVisibleVersion(); + long cacheTableVersion = scanTable.latestVersion; + // some partitions have been dropped, or delete or update or insert rows into new partition? + if (currentTableTime > cacheTableTime + || (currentTableTime == cacheTableTime && currentTableVersion > cacheTableVersion)) { + return true; + } + + for (Long scanPartitionId : scanTable.getScanPartitions()) { + Partition partition = olapTable.getPartition(scanPartitionId); + // partition == null: is this partition truncated? + if (partition == null || partition.getVisibleVersionTime() > latestPartitionTime + || (partition.getVisibleVersionTime() == latestPartitionTime + && partition.getVisibleVersion() > latestPartitionVersion)) { + return true; + } + } + } + return false; + } + + private boolean viewsChanged(Env env, SqlCacheContext sqlCacheContext) { + for (Entry cacheView : sqlCacheContext.getUsedViews().entrySet()) { + TableIf currentView = findTableIf(env, cacheView.getKey()); + if (currentView == null) { + return true; + } + + String cacheValueDdlSql = cacheView.getValue(); + if (currentView instanceof View) { + if (!((View) currentView).getInlineViewDef().equals(cacheValueDdlSql)) { + return true; + } + } else { + return true; + } + } + return false; + } + + private boolean rowPoliciesChanged(UserIdentity currentUserIdentity, Env env, SqlCacheContext sqlCacheContext) { + for (Entry> kv : sqlCacheContext.getRowPolicies().entrySet()) { + FullTableName qualifiedTable = kv.getKey(); + List cachedPolicies = kv.getValue(); + + List rowPolicies = env.getAccessManager().evalRowFilterPolicies( + currentUserIdentity, qualifiedTable.catalog, qualifiedTable.db, qualifiedTable.table); + if (!CollectionUtils.isEqualCollection(cachedPolicies, rowPolicies)) { + return true; + } + } + return false; + } + + private boolean dataMaskPoliciesChanged( + UserIdentity currentUserIdentity, Env env, SqlCacheContext sqlCacheContext) { + for (Entry> kv : sqlCacheContext.getDataMaskPolicies().entrySet()) { + FullColumnName qualifiedColumn = kv.getKey(); + Optional cachedPolicy = kv.getValue(); + + Optional dataMaskPolicy = env.getAccessManager() + .evalDataMaskPolicy(currentUserIdentity, qualifiedColumn.catalog, + qualifiedColumn.db, qualifiedColumn.table, qualifiedColumn.column); + if (!Objects.equals(cachedPolicy, dataMaskPolicy)) { + return true; + } + } + return false; + } + + private boolean privilegeChanged(ConnectContext connectContext, Env env, SqlCacheContext sqlCacheContext) { + StatementContext currentStatementContext = connectContext.getStatementContext(); + for (Entry> kv : sqlCacheContext.getCheckPrivilegeTablesOrViews().entrySet()) { + Set usedColumns = kv.getValue(); + TableIf tableIf = findTableIf(env, kv.getKey()); + if (tableIf == null) { + return true; + } + // release when close statementContext + currentStatementContext.addTableReadLock(tableIf); + try { + UserAuthentication.checkPermission(tableIf, connectContext, usedColumns); + } catch (Throwable t) { + return true; + } + } + return false; + } + + private boolean usedVariablesChanged(SqlCacheContext sqlCacheContext) { + for (Variable variable : sqlCacheContext.getUsedVariables()) { + Variable currentVariable = ExpressionAnalyzer.resolveUnboundVariable( + new UnboundVariable(variable.getName(), variable.getType())); + if (!Objects.equals(currentVariable, variable) + || variable.getRealExpression().anyMatch(Nondeterministic.class::isInstance)) { + return true; + } + } + return false; + } + + private boolean nondeterministicFunctionChanged( + Plan plan, ConnectContext connectContext, SqlCacheContext sqlCacheContext) { + if (sqlCacheContext.containsCannotProcessExpression()) { + return true; + } + + List> nondeterministicFunctions = sqlCacheContext.getFoldNondeterministicPairs(); + if (nondeterministicFunctions.isEmpty()) { + return false; + } + + CascadesContext tempCascadeContext = CascadesContext.initContext( + connectContext.getStatementContext(), plan, PhysicalProperties.ANY); + ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(tempCascadeContext); + for (Pair foldPair : nondeterministicFunctions) { + Expression nondeterministic = foldPair.first; + Expression deterministic = foldPair.second; + Expression fold = nondeterministic.accept(FoldConstantRuleOnFE.VISITOR_INSTANCE, rewriteContext); + if (!Objects.equals(deterministic, fold)) { + return true; + } + } + return false; + } + + private boolean isValidDbAndTable(TableIf tableIf, Env env) { + return getTableFromEnv(tableIf, env) != null; + } + + private TableIf getTableFromEnv(TableIf tableIf, Env env) { + Optional db = env.getInternalCatalog().getDb(tableIf.getDatabase().getId()); + if (!db.isPresent()) { + return null; + } + Optional table = db.get().getTable(tableIf.getId()); + return table.orElse(null); + } + + private Optional invalidateCache(String key) { + sqlCache.invalidate(key); + return Optional.empty(); + } + + private TableIf findTableIf(Env env, FullTableName fullTableName) { + CatalogIf> catalog = env.getCatalogMgr().getCatalog(fullTableName.catalog); + if (catalog == null) { + return null; + } + Optional> db = catalog.getDb(fullTableName.db); + if (!db.isPresent()) { + return null; + } + return db.get().getTable(fullTableName.table).orElse(null); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java index b3e8898ed40a05..f5ef529b4d97f3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java @@ -631,7 +631,7 @@ public Map build() { } public String getPrettyParseSqlTime() { - return getPrettyTime(parseSqlStartTime, parseSqlFinishTime, TUnit.TIME_MS); + return getPrettyTime(parseSqlFinishTime, parseSqlStartTime, TUnit.TIME_MS); } public String getPrettyNereidsAnalysisTime() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 60b7c0343a7001..dd569ef8f7519a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -102,7 +102,7 @@ public class CascadesContext implements ScheduleContext { private Optional currentRootRewriteJobContext; // in optimize stage, the plan will storage in the memo private Memo memo; - private final StatementContext statementContext; + private StatementContext statementContext; private final CTEContext cteContext; private final RuleSet ruleSet; @@ -265,6 +265,10 @@ public Memo getMemo() { return memo; } + public void releaseMemo() { + this.memo = null; + } + public void setTables(List tables) { this.tables = tables.stream().collect(Collectors.toMap(TableIf::getId, t -> t, (t1, t2) -> t1)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 97cb7f32379fb2..ef9734d4535eea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -52,10 +52,12 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSqlCache; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.Planner; import org.apache.doris.planner.RuntimeFilter; @@ -95,6 +97,7 @@ public class NereidsPlanner extends Planner { private PhysicalPlan physicalPlan; // The cost of optimized plan private double cost = 0; + private LogicalPlanAdapter logicalPlanAdapter; private List hooks = new ArrayList<>(); public NereidsPlanner(StatementContext statementContext) { @@ -112,7 +115,7 @@ public void plan(StatementBase queryStmt, org.apache.doris.thrift.TQueryOptions throw new RuntimeException("Wrong type of queryStmt, expected: "); } - LogicalPlanAdapter logicalPlanAdapter = (LogicalPlanAdapter) queryStmt; + logicalPlanAdapter = (LogicalPlanAdapter) queryStmt; ExplainLevel explainLevel = getExplainLevel(queryStmt.getExplainOptions()); @@ -129,35 +132,7 @@ public void plan(StatementBase queryStmt, org.apache.doris.thrift.TQueryOptions return; } physicalPlan = (PhysicalPlan) resultPlan; - PlanTranslatorContext planTranslatorContext = new PlanTranslatorContext(cascadesContext); - PhysicalPlanTranslator physicalPlanTranslator = new PhysicalPlanTranslator(planTranslatorContext, - statementContext.getConnectContext().getStatsErrorEstimator()); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsTranslateTime(); - } - if (cascadesContext.getConnectContext().getSessionVariable().isEnableNereidsTrace()) { - CounterEvent.clearCounter(); - } - if (cascadesContext.getConnectContext().getSessionVariable().isPlayNereidsDump()) { - return; - } - PlanFragment root = physicalPlanTranslator.translatePlan(physicalPlan); - - scanNodeList.addAll(planTranslatorContext.getScanNodes()); - descTable = planTranslatorContext.getDescTable(); - fragments = new ArrayList<>(planTranslatorContext.getPlanFragments()); - for (int seq = 0; seq < fragments.size(); seq++) { - fragments.get(seq).setFragmentSequenceNum(seq); - } - // set output exprs - logicalPlanAdapter.setResultExprs(root.getOutputExprs()); - ArrayList columnLabelList = physicalPlan.getOutput().stream().map(NamedExpression::getName) - .collect(Collectors.toCollection(ArrayList::new)); - logicalPlanAdapter.setColLabels(columnLabelList); - logicalPlanAdapter.setViewDdlSqls(statementContext.getViewDdlSqls()); - - // update scan nodes visible version at the end of plan phase. - ScanNode.setVisibleVersionForOlapScanNodes(getScanNodes()); + translate(physicalPlan); } @VisibleForTesting @@ -187,84 +162,98 @@ public Plan plan(LogicalPlan plan, PhysicalProperties requireProperties, Explain */ public Plan plan(LogicalPlan plan, PhysicalProperties requireProperties, ExplainLevel explainLevel, boolean showPlanProcess) { - if (explainLevel == ExplainLevel.PARSED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { - parsedPlan = plan; - if (explainLevel == ExplainLevel.PARSED_PLAN) { - return parsedPlan; + try { + if (plan instanceof LogicalSqlCache) { + rewrittenPlan = analyzedPlan = plan; + LogicalSqlCache logicalSqlCache = (LogicalSqlCache) plan; + physicalPlan = new PhysicalSqlCache( + logicalSqlCache.getQueryId(), logicalSqlCache.getColumnLabels(), + logicalSqlCache.getResultExprs(), logicalSqlCache.getCacheValues(), + logicalSqlCache.getBackendAddress(), logicalSqlCache.getPlanBody() + ); + return physicalPlan; + } + if (explainLevel == ExplainLevel.PARSED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { + parsedPlan = plan; + if (explainLevel == ExplainLevel.PARSED_PLAN) { + return parsedPlan; + } } - } - // pre-process logical plan out of memo, e.g. process SET_VAR hint - plan = preprocess(plan); + // pre-process logical plan out of memo, e.g. process SET_VAR hint + plan = preprocess(plan); - initCascadesContext(plan, requireProperties); + initCascadesContext(plan, requireProperties); - try (Lock lock = new Lock(plan, cascadesContext)) { - // resolve column, table and function - // analyze this query - analyze(showAnalyzeProcess(explainLevel, showPlanProcess)); - // minidump of input must be serialized first, this process ensure minidump string not null - try { - MinidumpUtils.serializeInputsToDumpFile(plan, cascadesContext.getTables()); - } catch (IOException e) { - throw new RuntimeException(e); - } + try (Lock lock = new Lock(plan, cascadesContext)) { + // resolve column, table and function + // analyze this query + analyze(showAnalyzeProcess(explainLevel, showPlanProcess)); + // minidump of input must be serialized first, this process ensure minidump string not null + try { + MinidumpUtils.serializeInputsToDumpFile(plan, cascadesContext.getTables()); + } catch (IOException e) { + throw new RuntimeException(e); + } - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setQueryAnalysisFinishTime(); - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsAnalysisTime(); - } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setQueryAnalysisFinishTime(); + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsAnalysisTime(); + } - if (explainLevel == ExplainLevel.ANALYZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { - analyzedPlan = cascadesContext.getRewritePlan(); - if (explainLevel == ExplainLevel.ANALYZED_PLAN) { - return analyzedPlan; + if (explainLevel == ExplainLevel.ANALYZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { + analyzedPlan = cascadesContext.getRewritePlan(); + if (explainLevel == ExplainLevel.ANALYZED_PLAN) { + return analyzedPlan; + } } - } - // rule-based optimize - rewrite(showRewriteProcess(explainLevel, showPlanProcess)); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsRewriteTime(); - } + // rule-based optimize + rewrite(showRewriteProcess(explainLevel, showPlanProcess)); + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsRewriteTime(); + } - if (explainLevel == ExplainLevel.REWRITTEN_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { - rewrittenPlan = cascadesContext.getRewritePlan(); - if (explainLevel == ExplainLevel.REWRITTEN_PLAN) { - return rewrittenPlan; + if (explainLevel == ExplainLevel.REWRITTEN_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { + rewrittenPlan = cascadesContext.getRewritePlan(); + if (explainLevel == ExplainLevel.REWRITTEN_PLAN) { + return rewrittenPlan; + } } - } - optimize(); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsOptimizeTime(); - } + optimize(); + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsOptimizeTime(); + } - // print memo before choose plan. - // if chooseNthPlan failed, we could get memo to debug - if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { - String memo = cascadesContext.getMemo().toString(); - LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + memo); - } + // print memo before choose plan. + // if chooseNthPlan failed, we could get memo to debug + if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { + String memo = cascadesContext.getMemo().toString(); + LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + memo); + } - int nth = cascadesContext.getConnectContext().getSessionVariable().getNthOptimizedPlan(); - PhysicalPlan physicalPlan = chooseNthPlan(getRoot(), requireProperties, nth); + int nth = cascadesContext.getConnectContext().getSessionVariable().getNthOptimizedPlan(); + PhysicalPlan physicalPlan = chooseNthPlan(getRoot(), requireProperties, nth); - physicalPlan = postProcess(physicalPlan); - if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { - String tree = physicalPlan.treeString(); - LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + tree); - } - if (explainLevel == ExplainLevel.OPTIMIZED_PLAN - || explainLevel == ExplainLevel.ALL_PLAN - || explainLevel == ExplainLevel.SHAPE_PLAN) { - optimizedPlan = physicalPlan; - } - // serialize optimized plan to dumpfile, dumpfile do not have this part means optimize failed - MinidumpUtils.serializeOutputToDumpFile(physicalPlan); - NereidsTracer.output(statementContext.getConnectContext()); + physicalPlan = postProcess(physicalPlan); + if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { + String tree = physicalPlan.treeString(); + LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + tree); + } + if (explainLevel == ExplainLevel.OPTIMIZED_PLAN + || explainLevel == ExplainLevel.ALL_PLAN + || explainLevel == ExplainLevel.SHAPE_PLAN) { + optimizedPlan = physicalPlan; + } + // serialize optimized plan to dumpfile, dumpfile do not have this part means optimize failed + MinidumpUtils.serializeOutputToDumpFile(physicalPlan); + NereidsTracer.output(statementContext.getConnectContext()); - return physicalPlan; + return physicalPlan; + } + } finally { + statementContext.releasePlannerResources(); } } @@ -317,6 +306,50 @@ private void optimize() { } } + private void translate(PhysicalPlan resultPlan) throws UserException { + if (resultPlan instanceof PhysicalSqlCache) { + return; + } + + PlanTranslatorContext planTranslatorContext = new PlanTranslatorContext(cascadesContext); + PhysicalPlanTranslator physicalPlanTranslator = new PhysicalPlanTranslator(planTranslatorContext, + statementContext.getConnectContext().getStatsErrorEstimator()); + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsTranslateTime(); + } + if (cascadesContext.getConnectContext().getSessionVariable().isEnableNereidsTrace()) { + CounterEvent.clearCounter(); + } + if (cascadesContext.getConnectContext().getSessionVariable().isPlayNereidsDump()) { + return; + } + PlanFragment root = physicalPlanTranslator.translatePlan(physicalPlan); + + scanNodeList.addAll(planTranslatorContext.getScanNodes()); + descTable = planTranslatorContext.getDescTable(); + fragments = new ArrayList<>(planTranslatorContext.getPlanFragments()); + for (int seq = 0; seq < fragments.size(); seq++) { + fragments.get(seq).setFragmentSequenceNum(seq); + } + // set output exprs + logicalPlanAdapter.setResultExprs(root.getOutputExprs()); + ArrayList columnLabelList = physicalPlan.getOutput().stream().map(NamedExpression::getName) + .collect(Collectors.toCollection(ArrayList::new)); + logicalPlanAdapter.setColLabels(columnLabelList); + logicalPlanAdapter.setViewDdlSqls(statementContext.getViewDdlSqls()); + if (statementContext.getSqlCacheContext().isPresent()) { + SqlCacheContext sqlCacheContext = statementContext.getSqlCacheContext().get(); + sqlCacheContext.setColLabels(columnLabelList); + sqlCacheContext.setResultExprs(root.getOutputExprs()); + sqlCacheContext.setPhysicalPlan(resultPlan.treeString()); + } + + cascadesContext.releaseMemo(); + + // update scan nodes visible version at the end of plan phase. + ScanNode.setVisibleVersionForOlapScanNodes(getScanNodes()); + } + private PhysicalPlan postProcess(PhysicalPlan physicalPlan) { return new PlanPostProcessors(cascadesContext).process(physicalPlan); } @@ -573,6 +606,10 @@ public PhysicalPlan getPhysicalPlan() { return physicalPlan; } + public LogicalPlanAdapter getLogicalPlanAdapter() { + return logicalPlanAdapter; + } + public List getHooks() { return hooks; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java new file mode 100644 index 00000000000000..6afda4e4fe96d6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java @@ -0,0 +1,353 @@ +// 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; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.DatabaseIf; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.mysql.privilege.DataMaskPolicy; +import org.apache.doris.mysql.privilege.RowFilterPolicy; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Variable; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.proto.Types.PUniqueId; +import org.apache.doris.qe.cache.CacheProxy; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +/** SqlCacheContext */ +public class SqlCacheContext { + private final UserIdentity userIdentity; + private final TUniqueId queryId; + // if contains udf/udaf/tableValuesFunction we can not process it and skip use sql cache + private volatile boolean cannotProcessExpression; + private volatile String physicalPlan; + private volatile long latestPartitionId = -1; + private volatile long latestPartitionTime = -1; + private volatile long latestPartitionVersion = -1; + private volatile long sumOfPartitionNum = -1; + private final Set usedTables = Sets.newLinkedHashSet(); + // value: ddl sql + private final Map usedViews = Maps.newLinkedHashMap(); + // value: usedColumns + private final Map> checkPrivilegeTablesOrViews = Maps.newLinkedHashMap(); + private final Map> rowPolicies = Maps.newLinkedHashMap(); + private final Map> dataMaskPolicies = Maps.newLinkedHashMap(); + private final Set usedVariables = Sets.newLinkedHashSet(); + // key: the expression which contains nondeterministic function, e.g. date(now()) + // value: the expression which already try to fold nondeterministic function, e.g. '2024-01-01' + // note that value maybe contains nondeterministic function too, when fold failed + private final List> foldNondeterministicPairs = Lists.newArrayList(); + private volatile boolean hasUnsupportedTables; + private final List scanTables = Lists.newArrayList(); + private volatile CacheProxy cacheProxy; + + private volatile List resultExprs; + private volatile List colLabels; + + private volatile PUniqueId cacheKeyMd5; + + public SqlCacheContext(UserIdentity userIdentity, TUniqueId queryId) { + this.userIdentity = Objects.requireNonNull(userIdentity, "userIdentity cannot be null"); + this.queryId = Objects.requireNonNull(queryId, "queryId cannot be null"); + } + + public String getPhysicalPlan() { + return physicalPlan; + } + + public void setPhysicalPlan(String physicalPlan) { + this.physicalPlan = physicalPlan; + } + + public void setCannotProcessExpression(boolean cannotProcessExpression) { + this.cannotProcessExpression = cannotProcessExpression; + } + + public boolean containsCannotProcessExpression() { + return cannotProcessExpression; + } + + public boolean hasUnsupportedTables() { + return hasUnsupportedTables; + } + + public void setHasUnsupportedTables(boolean hasUnsupportedTables) { + this.hasUnsupportedTables = hasUnsupportedTables; + } + + /** addUsedTable */ + public synchronized void addUsedTable(TableIf tableIf) { + if (tableIf == null) { + return; + } + DatabaseIf database = tableIf.getDatabase(); + if (database == null) { + setCannotProcessExpression(true); + return; + } + CatalogIf catalog = database.getCatalog(); + if (catalog == null) { + setCannotProcessExpression(true); + return; + } + + usedTables.add( + new FullTableName(database.getCatalog().getName(), database.getFullName(), tableIf.getName()) + ); + } + + /** addUsedView */ + public synchronized void addUsedView(TableIf tableIf, String ddlSql) { + if (tableIf == null) { + return; + } + DatabaseIf database = tableIf.getDatabase(); + if (database == null) { + setCannotProcessExpression(true); + return; + } + CatalogIf catalog = database.getCatalog(); + if (catalog == null) { + setCannotProcessExpression(true); + return; + } + + usedViews.put( + new FullTableName(database.getCatalog().getName(), database.getFullName(), tableIf.getName()), + ddlSql + ); + } + + /** addNeedCheckPrivilegeTablesOrViews */ + public synchronized void addCheckPrivilegeTablesOrViews(TableIf tableIf, Set usedColumns) { + if (tableIf == null) { + return; + } + DatabaseIf database = tableIf.getDatabase(); + if (database == null) { + setCannotProcessExpression(true); + return; + } + CatalogIf catalog = database.getCatalog(); + if (catalog == null) { + setCannotProcessExpression(true); + return; + } + FullTableName fullTableName = new FullTableName(catalog.getName(), database.getFullName(), tableIf.getName()); + Set existsColumns = checkPrivilegeTablesOrViews.get(fullTableName); + if (existsColumns == null) { + checkPrivilegeTablesOrViews.put(fullTableName, usedColumns); + } else { + ImmutableSet.Builder allUsedColumns = ImmutableSet.builderWithExpectedSize( + existsColumns.size() + usedColumns.size()); + allUsedColumns.addAll(existsColumns); + allUsedColumns.addAll(usedColumns); + checkPrivilegeTablesOrViews.put(fullTableName, allUsedColumns.build()); + } + } + + public synchronized void setRowFilterPolicy( + String catalog, String db, String table, List rowFilterPolicy) { + rowPolicies.put(new FullTableName(catalog, db, table), Utils.fastToImmutableList(rowFilterPolicy)); + } + + public synchronized Map> getRowFilterPolicies() { + return ImmutableMap.copyOf(rowPolicies); + } + + public synchronized void addDataMaskPolicy( + String catalog, String db, String table, String columnName, Optional dataMaskPolicy) { + dataMaskPolicies.put( + new FullColumnName(catalog, db, table, columnName.toLowerCase(Locale.ROOT)), dataMaskPolicy + ); + } + + public synchronized Map> getDataMaskPolicies() { + return ImmutableMap.copyOf(dataMaskPolicies); + } + + public synchronized void addUsedVariable(Variable value) { + usedVariables.add(value); + } + + public synchronized List getUsedVariables() { + return ImmutableList.copyOf(usedVariables); + } + + public synchronized void addFoldNondeterministicPair(Expression unfold, Expression fold) { + foldNondeterministicPairs.add(Pair.of(unfold, fold)); + } + + public synchronized List> getFoldNondeterministicPairs() { + return ImmutableList.copyOf(foldNondeterministicPairs); + } + + public boolean isCannotProcessExpression() { + return cannotProcessExpression; + } + + public UserIdentity getUserIdentity() { + return userIdentity; + } + + public long getLatestPartitionTime() { + return latestPartitionTime; + } + + public void setLatestPartitionTime(long latestPartitionTime) { + this.latestPartitionTime = latestPartitionTime; + } + + public long getLatestPartitionVersion() { + return latestPartitionVersion; + } + + public void setLatestPartitionVersion(long latestPartitionVersion) { + this.latestPartitionVersion = latestPartitionVersion; + } + + public long getLatestPartitionId() { + return latestPartitionId; + } + + public void setLatestPartitionId(long latestPartitionId) { + this.latestPartitionId = latestPartitionId; + } + + public long getSumOfPartitionNum() { + return sumOfPartitionNum; + } + + public void setSumOfPartitionNum(long sumOfPartitionNum) { + this.sumOfPartitionNum = sumOfPartitionNum; + } + + public CacheProxy getCacheProxy() { + return cacheProxy; + } + + public void setCacheProxy(CacheProxy cacheProxy) { + this.cacheProxy = cacheProxy; + } + + public Set getUsedTables() { + return ImmutableSet.copyOf(usedTables); + } + + public Map getUsedViews() { + return ImmutableMap.copyOf(usedViews); + } + + public synchronized Map> getCheckPrivilegeTablesOrViews() { + return ImmutableMap.copyOf(checkPrivilegeTablesOrViews); + } + + public synchronized Map> getRowPolicies() { + return ImmutableMap.copyOf(rowPolicies); + } + + public boolean isHasUnsupportedTables() { + return hasUnsupportedTables; + } + + public synchronized void addScanTable(ScanTable scanTable) { + this.scanTables.add(scanTable); + } + + public synchronized List getScanTables() { + return ImmutableList.copyOf(scanTables); + } + + public List getResultExprs() { + return resultExprs; + } + + public void setResultExprs(List resultExprs) { + this.resultExprs = ImmutableList.copyOf(resultExprs); + } + + public List getColLabels() { + return colLabels; + } + + public void setColLabels(List colLabels) { + this.colLabels = ImmutableList.copyOf(colLabels); + } + + public TUniqueId getQueryId() { + return queryId; + } + + public PUniqueId getCacheKeyMd5() { + return cacheKeyMd5; + } + + public void setCacheKeyMd5(PUniqueId cacheKeyMd5) { + this.cacheKeyMd5 = cacheKeyMd5; + } + + /** FullTableName */ + @lombok.Data + @lombok.AllArgsConstructor + public static class FullTableName { + public final String catalog; + public final String db; + public final String table; + } + + /** FullColumnName */ + @lombok.Data + @lombok.AllArgsConstructor + public static class FullColumnName { + public final String catalog; + public final String db; + public final String table; + public final String column; + } + + /** ScanTable */ + @lombok.Data + @lombok.AllArgsConstructor + public static class ScanTable { + public final FullTableName fullTableName; + public final long latestTimestamp; + public final long latestVersion; + public final List scanPartitions = Lists.newArrayList(); + + public void addScanPartition(Long partitionId) { + this.scanPartitions.add(partitionId); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 403c605ba75992..5baa59a35dec1b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids; import org.apache.doris.analysis.StatementBase; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; import org.apache.doris.nereids.hint.Hint; @@ -37,6 +38,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.qe.cache.CacheAnalyzer; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; @@ -44,7 +46,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.sparkproject.guava.base.Throwables; +import java.io.Closeable; import java.util.ArrayList; import java.util.BitSet; import java.util.Collection; @@ -53,14 +59,18 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.Stack; import java.util.TreeMap; +import java.util.concurrent.TimeUnit; import javax.annotation.concurrent.GuardedBy; /** * Statement context for nereids */ -public class StatementContext { +public class StatementContext implements Closeable { + private static final Logger LOG = LogManager.getLogger(StatementContext.class); private ConnectContext connectContext; @@ -101,6 +111,8 @@ public class StatementContext { private final Map rewrittenCteProducer = new HashMap<>(); private final Map rewrittenCteConsumer = new HashMap<>(); private final Set viewDdlSqlSet = Sets.newHashSet(); + private final SqlCacheContext sqlCacheContext; + private Map> checkedPrivilegedTableAndUsedColumns = Maps.newLinkedHashMap(); // collect all hash join conditions to compute node connectivity in join graph private final List joinFilters = new ArrayList<>(); @@ -122,18 +134,30 @@ public class StatementContext { private BitSet disableRules; + // table locks + private Stack plannerResources = new Stack<>(); + // for create view support in nereids // key is the start and end position of the sql substring that needs to be replaced, // and value is the new string used for replacement. private TreeMap, String> indexInSqlToString = new TreeMap<>(new Pair.PairComparator<>()); public StatementContext() { - this.connectContext = ConnectContext.get(); + this(ConnectContext.get(), null); } + /** StatementContext */ public StatementContext(ConnectContext connectContext, OriginStatement originStatement) { this.connectContext = connectContext; this.originStatement = originStatement; + if (connectContext != null && connectContext.getSessionVariable() != null + && connectContext.queryId() != null + && CacheAnalyzer.canUseSqlCache(connectContext.getSessionVariable())) { + this.sqlCacheContext = new SqlCacheContext( + connectContext.getCurrentUserIdentity(), connectContext.queryId()); + } else { + this.sqlCacheContext = null; + } } public void setConnectContext(ConnectContext connectContext) { @@ -172,6 +196,10 @@ public void setMaxContinuousJoin(int joinCount) { } } + public Optional getSqlCacheContext() { + return Optional.ofNullable(sqlCacheContext); + } + public int getMaxContinuousJoin() { return joinCount; } @@ -368,4 +396,90 @@ public TreeMap, String> getIndexInSqlToString() { public void addIndexInSqlToString(Pair pair, String replacement) { indexInSqlToString.put(pair, replacement); } + + /** addTableReadLock */ + public synchronized void addTableReadLock(TableIf tableIf) { + if (!tableIf.needReadLockWhenPlan()) { + return; + } + if (!tableIf.tryReadLock(1, TimeUnit.MINUTES)) { + close(); + throw new RuntimeException(String.format("Failed to get read lock on table: %s", tableIf.getName())); + } + + String fullTableName = tableIf.getNameWithFullQualifiers(); + String resourceName = "tableReadLock(" + fullTableName + ")"; + plannerResources.push(new CloseableResource( + resourceName, Thread.currentThread().getName(), originStatement.originStmt, tableIf::readUnlock)); + } + + /** releasePlannerResources */ + public synchronized void releasePlannerResources() { + Throwable throwable = null; + while (!plannerResources.isEmpty()) { + try { + plannerResources.pop().close(); + } catch (Throwable t) { + if (throwable == null) { + throwable = t; + } + } + } + if (throwable != null) { + Throwables.propagateIfInstanceOf(throwable, RuntimeException.class); + throw new IllegalStateException("Release resource failed", throwable); + } + } + + // CHECKSTYLE OFF + @Override + protected void finalize() throws Throwable { + if (!plannerResources.isEmpty()) { + String msg = "Resources leak: " + plannerResources; + LOG.error(msg); + throw new IllegalStateException(msg); + } + } + // CHECKSTYLE ON + + @Override + public void close() { + releasePlannerResources(); + } + + private static class CloseableResource implements Closeable { + public final String resourceName; + public final String threadName; + public final String sql; + + private final Closeable resource; + + private boolean closed; + + public CloseableResource(String resourceName, String threadName, String sql, Closeable resource) { + this.resourceName = resourceName; + this.threadName = threadName; + this.sql = sql; + this.resource = resource; + } + + @Override + public void close() { + if (!closed) { + try { + resource.close(); + } catch (Throwable t) { + Throwables.propagateIfInstanceOf(t, RuntimeException.class); + throw new IllegalStateException("Close resource failed: " + t.getMessage(), t); + } + closed = true; + } + } + + @Override + public String toString() { + return "\nResource {\n name: " + resourceName + ",\n thread: " + threadName + + ",\n sql:\n" + sql + "\n}"; + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java index da6881a4905468..35e1a6a354ba33 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.parser; +import org.apache.doris.analysis.ExplainOptions; import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.Env; import org.apache.doris.common.Pair; @@ -26,6 +27,7 @@ import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.parser.plsql.PLSqlLogicalPlanBuilder; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.types.DataType; import org.apache.doris.plugin.DialectConverterPlugin; @@ -37,14 +39,18 @@ import org.antlr.v4.runtime.CharStreams; import org.antlr.v4.runtime.CommonTokenStream; import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.TokenSource; import org.antlr.v4.runtime.atn.PredictionMode; import org.antlr.v4.runtime.misc.ParseCancellationException; import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.BitSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.function.Function; import javax.annotation.Nullable; @@ -56,6 +62,21 @@ public class NereidsParser { private static final ParseErrorListener PARSE_ERROR_LISTENER = new ParseErrorListener(); private static final PostProcessor POST_PROCESSOR = new PostProcessor(); + private static final BitSet EXPLAIN_TOKENS = new BitSet(); + + static { + EXPLAIN_TOKENS.set(DorisLexer.EXPLAIN); + EXPLAIN_TOKENS.set(DorisLexer.PARSED); + EXPLAIN_TOKENS.set(DorisLexer.ANALYZED); + EXPLAIN_TOKENS.set(DorisLexer.LOGICAL); + EXPLAIN_TOKENS.set(DorisLexer.REWRITTEN); + EXPLAIN_TOKENS.set(DorisLexer.PHYSICAL); + EXPLAIN_TOKENS.set(DorisLexer.OPTIMIZED); + EXPLAIN_TOKENS.set(DorisLexer.PLAN); + EXPLAIN_TOKENS.set(DorisLexer.PROCESS); + + } + /** * In MySQL protocol, client could send multi-statement in a single packet. * see docs for more information. @@ -83,6 +104,98 @@ public List parseSQL(String originStr, @Nullable LogicalPlanBuild return statementBases; } + /** + * scan to token + * for example: select id from tbl return Tokens: ['select', 'id', 'from', 'tbl'] + */ + public static TokenSource scan(String sql) { + return new DorisLexer(new CaseInsensitiveStream(CharStreams.fromString(sql))); + } + + /** + * tryParseExplainPlan + * @param sql sql + * @return key: ExplainOptions, value: explain body + */ + public static Optional> tryParseExplainPlan(String sql) { + try { + TokenSource tokenSource = scan(sql); + if (expect(tokenSource, DorisLexer.EXPLAIN) == null) { + return Optional.empty(); + } + + Token token = readUntilNonComment(tokenSource); + if (token == null) { + return Optional.empty(); + } + + int tokenType = token.getType(); + ExplainLevel explainLevel = ExplainLevel.ALL_PLAN; + if (tokenType == DorisLexer.PARSED) { + explainLevel = ExplainLevel.PARSED_PLAN; + token = readUntilNonComment(tokenSource); + } else if (tokenType == DorisLexer.ANALYZED) { + explainLevel = ExplainLevel.ANALYZED_PLAN; + token = readUntilNonComment(tokenSource); + } else if (tokenType == DorisLexer.LOGICAL || tokenType == DorisLexer.REWRITTEN) { + explainLevel = ExplainLevel.REWRITTEN_PLAN; + token = readUntilNonComment(tokenSource); + } else if (tokenType == DorisLexer.PHYSICAL || tokenType == DorisLexer.OPTIMIZED) { + explainLevel = ExplainLevel.OPTIMIZED_PLAN; + token = readUntilNonComment(tokenSource); + } + + if (token == null) { + return Optional.empty(); + } + tokenType = token.getType(); + if (tokenType != DorisLexer.PLAN) { + return Optional.empty(); + } + + token = readUntilNonComment(tokenSource); + Token explainPlanBody; + boolean showPlanProcess = false; + if (token.getType() == DorisLexer.PROCESS) { + showPlanProcess = true; + explainPlanBody = readUntilNonComment(tokenSource); + } else { + explainPlanBody = token; + } + + if (explainPlanBody == null) { + return Optional.empty(); + } + ExplainOptions explainOptions = new ExplainOptions(explainLevel, showPlanProcess); + return Optional.of(Pair.of(explainOptions, sql.substring(explainPlanBody.getStartIndex()))); + } catch (Throwable t) { + return Optional.empty(); + } + } + + private static Token expect(TokenSource tokenSource, int tokenType) { + Token nextToken = readUntilNonComment(tokenSource); + if (nextToken == null) { + return null; + } + return nextToken.getType() == tokenType ? nextToken : null; + } + + private static Token readUntilNonComment(TokenSource tokenSource) { + Token token = tokenSource.nextToken(); + while (token != null) { + int tokenType = token.getType(); + if (tokenType == DorisLexer.BRACKETED_COMMENT + || tokenType == DorisLexer.SIMPLE_COMMENT + || tokenType == DorisLexer.WS) { + token = tokenSource.nextToken(); + continue; + } + break; + } + return token; + } + private List parseSQLWithDialect(String sql, SessionVariable sessionVariable) { @Nullable Dialect sqlDialect = Dialect.getByName(sessionVariable.getSqlDialect()); 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 e20d3e8d551852..a99de00e378f8a 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 @@ -19,8 +19,10 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.FunctionRegistry; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.MappingSlot; import org.apache.doris.nereids.analyzer.Scope; @@ -49,6 +51,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.FunctionBuilder; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; @@ -833,11 +836,16 @@ private LogicalTVFRelation bindTableValuedFunction(MatchingContext bindResult + = functionBuilder.build(functionName, arguments); + if (!(bindResult.first instanceof TableValuedFunction)) { + throw new AnalysisException(bindResult.first.toSql() + " is not a TableValuedFunction"); } - return new LogicalTVFRelation(unboundTVFRelation.getRelationId(), (TableValuedFunction) function); + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().setCannotProcessExpression(true); + } + return new LogicalTVFRelation(unboundTVFRelation.getRelationId(), (TableValuedFunction) bindResult.first); } private void checkSameNameSlot(List childOutputs, String subQueryAlias) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index c4175d96fd0dc0..0e6d940891ebce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -31,6 +31,7 @@ import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.nereids.CTEContext; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.analyzer.UnboundRelation; @@ -249,69 +250,90 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio }); List qualifierWithoutTableName = Lists.newArrayList(); qualifierWithoutTableName.addAll(tableQualifier.subList(0, tableQualifier.size() - 1)); - switch (table.getType()) { - case OLAP: - case MATERIALIZED_VIEW: - return makeOlapScan(table, unboundRelation, qualifierWithoutTableName); - case VIEW: - View view = (View) table; - String inlineViewDef = view.getInlineViewDef(); - Plan viewBody = parseAndAnalyzeView(inlineViewDef, cascadesContext); - LogicalView logicalView = new LogicalView<>(view, viewBody); - return new LogicalSubQueryAlias<>(tableQualifier, logicalView); - case HMS_EXTERNAL_TABLE: - HMSExternalTable hmsTable = (HMSExternalTable) table; - if (Config.enable_query_hive_views && hmsTable.isView()) { - String hiveCatalog = hmsTable.getCatalog().getName(); - String ddlSql = hmsTable.getViewText(); - Plan hiveViewPlan = parseAndAnalyzeHiveView(hiveCatalog, ddlSql, cascadesContext); - return new LogicalSubQueryAlias<>(tableQualifier, hiveViewPlan); - } - hmsTable.setScanParams(unboundRelation.getScanParams()); - return new LogicalFileScan(unboundRelation.getRelationId(), (HMSExternalTable) table, - qualifierWithoutTableName, unboundRelation.getTableSample()); - case ICEBERG_EXTERNAL_TABLE: - case PAIMON_EXTERNAL_TABLE: - case MAX_COMPUTE_EXTERNAL_TABLE: - case TRINO_CONNECTOR_EXTERNAL_TABLE: - return new LogicalFileScan(unboundRelation.getRelationId(), (ExternalTable) table, - qualifierWithoutTableName, unboundRelation.getTableSample()); - case SCHEMA: - return new LogicalSchemaScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); - case JDBC_EXTERNAL_TABLE: - case JDBC: - return new LogicalJdbcScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); - case ODBC: - return new LogicalOdbcScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); - case ES_EXTERNAL_TABLE: - return new LogicalEsScan(unboundRelation.getRelationId(), (EsExternalTable) table, - qualifierWithoutTableName); - case TEST_EXTERNAL_TABLE: - return new LogicalTestScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); - default: - try { - // TODO: support other type table, such as ELASTICSEARCH - cascadesContext.getConnectContext().getSessionVariable().enableFallbackToOriginalPlannerOnce(); - } catch (Exception e) { - // ignore + boolean isView = false; + try { + switch (table.getType()) { + case OLAP: + case MATERIALIZED_VIEW: + return makeOlapScan(table, unboundRelation, qualifierWithoutTableName); + case VIEW: + View view = (View) table; + isView = true; + String inlineViewDef = view.getInlineViewDef(); + Plan viewBody = parseAndAnalyzeView(view, inlineViewDef, cascadesContext); + LogicalView logicalView = new LogicalView<>(view, viewBody); + return new LogicalSubQueryAlias<>(tableQualifier, logicalView); + case HMS_EXTERNAL_TABLE: + HMSExternalTable hmsTable = (HMSExternalTable) table; + if (Config.enable_query_hive_views && hmsTable.isView()) { + isView = true; + String hiveCatalog = hmsTable.getCatalog().getName(); + String ddlSql = hmsTable.getViewText(); + Plan hiveViewPlan = parseAndAnalyzeHiveView(hmsTable, hiveCatalog, ddlSql, cascadesContext); + return new LogicalSubQueryAlias<>(tableQualifier, hiveViewPlan); + } + hmsTable.setScanParams(unboundRelation.getScanParams()); + return new LogicalFileScan(unboundRelation.getRelationId(), (HMSExternalTable) table, + qualifierWithoutTableName, unboundRelation.getTableSample()); + case ICEBERG_EXTERNAL_TABLE: + case PAIMON_EXTERNAL_TABLE: + case MAX_COMPUTE_EXTERNAL_TABLE: + case TRINO_CONNECTOR_EXTERNAL_TABLE: + return new LogicalFileScan(unboundRelation.getRelationId(), (ExternalTable) table, + qualifierWithoutTableName, unboundRelation.getTableSample()); + case SCHEMA: + return new LogicalSchemaScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); + case JDBC_EXTERNAL_TABLE: + case JDBC: + return new LogicalJdbcScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); + case ODBC: + return new LogicalOdbcScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); + case ES_EXTERNAL_TABLE: + return new LogicalEsScan(unboundRelation.getRelationId(), (EsExternalTable) table, + qualifierWithoutTableName); + case TEST_EXTERNAL_TABLE: + return new LogicalTestScan(unboundRelation.getRelationId(), table, qualifierWithoutTableName); + default: + try { + // TODO: support other type table, such as ELASTICSEARCH + cascadesContext.getConnectContext().getSessionVariable().enableFallbackToOriginalPlannerOnce(); + } catch (Exception e) { + // ignore + } + throw new AnalysisException("Unsupported tableType " + table.getType()); + } + } finally { + if (!isView) { + Optional sqlCacheContext = cascadesContext.getStatementContext().getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + if (table instanceof OlapTable) { + sqlCacheContext.get().addUsedTable(table); + } else { + sqlCacheContext.get().setHasUnsupportedTables(true); + } } - throw new AnalysisException("Unsupported tableType " + table.getType()); + } } } - private Plan parseAndAnalyzeHiveView(String hiveCatalog, String ddlSql, CascadesContext cascadesContext) { + private Plan parseAndAnalyzeHiveView( + HMSExternalTable table, String hiveCatalog, String ddlSql, CascadesContext cascadesContext) { ConnectContext ctx = cascadesContext.getConnectContext(); String previousCatalog = ctx.getCurrentCatalog().getName(); String previousDb = ctx.getDatabase(); ctx.changeDefaultCatalog(hiveCatalog); - Plan hiveViewPlan = parseAndAnalyzeView(ddlSql, cascadesContext); + Plan hiveViewPlan = parseAndAnalyzeView(table, ddlSql, cascadesContext); ctx.changeDefaultCatalog(previousCatalog); ctx.setDatabase(previousDb); return hiveViewPlan; } - private Plan parseAndAnalyzeView(String ddlSql, CascadesContext parentContext) { + private Plan parseAndAnalyzeView(TableIf view, String ddlSql, CascadesContext parentContext) { parentContext.getStatementContext().addViewDdlSql(ddlSql); + Optional sqlCacheContext = parentContext.getStatementContext().getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().addUsedView(view, ddlSql); + } LogicalPlan parsedViewPlan = new NereidsParser().parseSingle(ddlSql); // TODO: use a good to do this, such as eliminate UnboundResultSink if (parsedViewPlan instanceof UnboundResultSink) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java index 56ca1b3a8c4822..607d9fc1912a1c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java @@ -22,8 +22,10 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.FunctionRegistry; import org.apache.doris.common.DdlException; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.Util; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.Scope; import org.apache.doris.nereids.analyzer.UnboundAlias; @@ -34,6 +36,7 @@ import org.apache.doris.nereids.analyzer.UnboundVariable.VariableType; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.rules.FoldConstantRuleOnFE; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.ArrayItemReference; import org.apache.doris.nereids.trees.expressions.BinaryArithmetic; @@ -60,12 +63,15 @@ import org.apache.doris.nereids.trees.expressions.WhenClause; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; import org.apache.doris.nereids.trees.expressions.functions.FunctionBuilder; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; import org.apache.doris.nereids.trees.expressions.functions.scalar.ElementAt; import org.apache.doris.nereids.trees.expressions.functions.scalar.Lambda; import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl; import org.apache.doris.nereids.trees.expressions.functions.scalar.PushDownToProjectionFunction; import org.apache.doris.nereids.trees.expressions.functions.udf.AliasUdfBuilder; +import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdaf; +import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdf; import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; @@ -83,6 +89,7 @@ import org.apache.doris.qe.SessionVariable; import org.apache.doris.qe.VariableMgr; import org.apache.doris.qe.VariableVarConverters; +import org.apache.doris.qe.cache.CacheAnalyzer; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -112,6 +119,7 @@ public class ExpressionAnalyzer extends SubExprAnalyzer buildResult = builder.build(functionName, arguments); + StatementContext statementContext = context.cascadesContext.getStatementContext(); + if (buildResult.second instanceof Nondeterministic) { + hasNondeterministic = true; + } + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + if (builder instanceof AliasUdfBuilder + || buildResult.second instanceof JavaUdf || buildResult.second instanceof JavaUdaf) { + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().setCannotProcessExpression(true); + } + } if (builder instanceof AliasUdfBuilder) { + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().setCannotProcessExpression(true); + } // we do type coercion in build function in alias function, so it's ok to return directly. - return builder.build(functionName, arguments); + return buildResult.first; } else { - Expression boundFunction = TypeCoercionUtils - .processBoundFunction((BoundFunction) builder.build(functionName, arguments)); - if (boundFunction instanceof Count + Expression castFunction = TypeCoercionUtils.processBoundFunction((BoundFunction) buildResult.first); + if (castFunction instanceof Count && context.cascadesContext.getOuterScope().isPresent() && !context.cascadesContext.getOuterScope().get().getCorrelatedSlots() .isEmpty()) { @@ -338,20 +391,20 @@ public Expression visitUnboundFunction(UnboundFunction unboundFunction, Expressi // if there is no match, the row from right table is filled with nulls // but COUNT function is always not nullable. // so wrap COUNT with Nvl to ensure it's result is 0 instead of null to get the correct result - boundFunction = new Nvl(boundFunction, new BigIntLiteral(0)); + castFunction = new Nvl(castFunction, new BigIntLiteral(0)); } if (currentElementAtLevel == 1 - && PushDownToProjectionFunction.validToPushDown(boundFunction)) { + && PushDownToProjectionFunction.validToPushDown(castFunction)) { // Only rewrite the top level of PushDownToProjectionFunction, otherwise invalid slot will be generated // currentElementAtLevel == 1 means at the top of element_at function, other levels will be ignored. currentElementAtLevel = 0; - return visitElementAt((ElementAt) boundFunction, context); + return visitElementAt((ElementAt) castFunction, context); } - if (boundFunction instanceof ElementAt) { + if (castFunction instanceof ElementAt) { --currentElementAtLevel; } - return boundFunction; + return castFunction; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java index e8ed474a1be733..8e79e60abad1bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FunctionBinder.java @@ -181,10 +181,10 @@ public Expression visitUnboundFunction(UnboundFunction unboundFunction, Expressi unboundFunction.getDbName(), functionName, arguments); if (builder instanceof AliasUdfBuilder) { // we do type coercion in build function in alias function, so it's ok to return directly. - return builder.build(functionName, arguments); + return builder.build(functionName, arguments).first; } else { Expression boundFunction = TypeCoercionUtils - .processBoundFunction((BoundFunction) builder.build(functionName, arguments)); + .processBoundFunction((BoundFunction) builder.build(functionName, arguments).first); if (boundFunction instanceof Count && context.cascadesContext.getOuterScope().isPresent() && !context.cascadesContext.getOuterScope().get().getCorrelatedSlots() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java index b4c5552706c589..74f41e17cac49d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/ReplaceVariableByLiteral.java @@ -17,6 +17,8 @@ package org.apache.doris.nereids.rules.expression.rules; +import org.apache.doris.nereids.SqlCacheContext; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.rules.expression.ExpressionPatternMatcher; import org.apache.doris.nereids.rules.expression.ExpressionPatternRuleFactory; import org.apache.doris.nereids.trees.expressions.Expression; @@ -25,6 +27,7 @@ import com.google.common.collect.ImmutableList; import java.util.List; +import java.util.Optional; /** * replace varaible to real expression @@ -35,7 +38,15 @@ public class ReplaceVariableByLiteral implements ExpressionPatternRuleFactory { @Override public List> buildRules() { return ImmutableList.of( - matchesType(Variable.class).then(Variable::getRealExpression) + matchesType(Variable.class).thenApply(ctx -> { + StatementContext statementContext = ctx.cascadesContext.getStatementContext(); + Variable variable = ctx.expr; + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().addUsedVariable(variable); + } + return variable.getRealExpression(); + }) ); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java index 70a5c593ee3dc8..713a9404dc08c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckPrivileges.java @@ -19,6 +19,9 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.UserException; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.rules.analysis.UserAuthentication; @@ -34,6 +37,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; /** CheckPrivileges */ @@ -84,11 +88,17 @@ private Set computeUsedColumns(Plan plan, Set requiredSlots) { } private void checkColumnPrivileges(TableIf table, Set usedColumns) { - ConnectContext connectContext = jobContext.getCascadesContext().getConnectContext(); + CascadesContext cascadesContext = jobContext.getCascadesContext(); + ConnectContext connectContext = cascadesContext.getConnectContext(); try { UserAuthentication.checkPermission(table, connectContext, usedColumns); } catch (UserException e) { throw new AnalysisException(e.getMessage(), e); } + StatementContext statementContext = cascadesContext.getStatementContext(); + Optional sqlCacheContext = statementContext.getSqlCacheContext(); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().addCheckPrivilegeTablesOrViews(table, usedColumns); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java index adced61c7b7508..566798ec2d4e46 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/ExpressionEvaluator.java @@ -20,7 +20,6 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; -import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.executable.DateTimeAcquire; import org.apache.doris.nereids.trees.expressions.functions.executable.DateTimeArithmetic; @@ -79,7 +78,10 @@ public Expression eval(Expression expression) { } else if (expression instanceof BoundFunction) { BoundFunction function = ((BoundFunction) expression); fnName = function.getName(); - args = function.children().stream().map(ExpressionTrait::getDataType).toArray(DataType[]::new); + args = new DataType[function.arity()]; + for (int i = 0; i < function.children().size(); i++) { + args[i] = function.child(i).getDataType(); + } } if ((Env.getCurrentEnv().isNullResultWithOneNullParamFunction(fnName))) { @@ -166,8 +168,11 @@ private void registerFEFunction(ImmutableMultimap.Builder arguments) { } private AggregateFunction buildState(String nestedName, List arguments) { - return (AggregateFunction) nestedBuilder.build(nestedName, arguments); + return (AggregateFunction) nestedBuilder.build(nestedName, arguments).first; } private AggregateFunction buildForEach(String nestedName, List arguments) { @@ -96,7 +97,7 @@ private AggregateFunction buildForEach(String nestedName, List DataType itemType = ((ArrayType) arrayType).getItemType(); return new SlotReference("mocked", itemType, (((ArrayType) arrayType).containsNull())); }).collect(Collectors.toList()); - return (AggregateFunction) nestedBuilder.build(nestedName, forEachargs); + return (AggregateFunction) nestedBuilder.build(nestedName, forEachargs).first; } private AggregateFunction buildMergeOrUnion(String nestedName, List arguments) { @@ -118,24 +119,24 @@ private AggregateFunction buildMergeOrUnion(String nestedName, List arguments) { + public Pair build(String name, List arguments) { String nestedName = getNestedName(name); if (combinatorSuffix.equals(STATE)) { AggregateFunction nestedFunction = buildState(nestedName, arguments); - return new StateCombinator((List) arguments, nestedFunction); + return Pair.of(new StateCombinator((List) arguments, nestedFunction), nestedFunction); } else if (combinatorSuffix.equals(MERGE)) { AggregateFunction nestedFunction = buildMergeOrUnion(nestedName, arguments); - return new MergeCombinator((List) arguments, nestedFunction); + return Pair.of(new MergeCombinator((List) arguments, nestedFunction), nestedFunction); } else if (combinatorSuffix.equals(UNION)) { AggregateFunction nestedFunction = buildMergeOrUnion(nestedName, arguments); - return new UnionCombinator((List) arguments, nestedFunction); + return Pair.of(new UnionCombinator((List) arguments, nestedFunction), nestedFunction); } else if (combinatorSuffix.equals(FOREACH)) { AggregateFunction nestedFunction = buildForEach(nestedName, arguments); - return new ForEachCombinator((List) arguments, nestedFunction); + return Pair.of(new ForEachCombinator((List) arguments, nestedFunction), nestedFunction); } return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/BuiltinFunctionBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/BuiltinFunctionBuilder.java index e2dab713332fd6..a071328ec52635 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/BuiltinFunctionBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/BuiltinFunctionBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.ReflectionUtils; import org.apache.doris.nereids.trees.expressions.Expression; @@ -86,12 +87,12 @@ private Class getConstructorArgumentType(int index) { } @Override - public BoundFunction build(String name, List arguments) { + public Pair build(String name, List arguments) { try { if (isVariableLength) { - return builderMethod.newInstance(toVariableLengthArguments(arguments)); + return Pair.ofSame(builderMethod.newInstance(toVariableLengthArguments(arguments))); } else { - return builderMethod.newInstance(arguments.toArray()); + return Pair.ofSame(builderMethod.newInstance(arguments.toArray())); } } catch (Throwable t) { String argString = arguments.stream() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/FunctionBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/FunctionBuilder.java index d1e69d3e307d6f..760edacaeabaf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/FunctionBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/FunctionBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.trees.expressions.Expression; import com.google.common.collect.ImmutableList; @@ -32,7 +33,7 @@ public abstract class FunctionBuilder { /** check whether arguments can apply to the constructor */ public abstract boolean canApply(List arguments); - public final Expression build(String name, Object argument) { + public final Pair build(String name, Object argument) { return build(name, ImmutableList.of(argument)); } @@ -40,7 +41,10 @@ public final Expression build(String name, Object argument) { * build a BoundFunction by function name and arguments. * @param name function name which in the sql expression * @param arguments the function's argument expressions - * @return the concrete bound function instance + * @return the concrete bound function instance, + * key: the final result expression that should return, e.g. the function wrapped some cast function, + * value: the real BoundFunction */ - public abstract Expression build(String name, List arguments); + public abstract Pair build( + String name, List arguments); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConnectionId.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConnectionId.java index 7f16fa1a4acb69..6f17e247c44042 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConnectionId.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ConnectionId.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; @@ -32,7 +33,7 @@ * ScalarFunction 'ConnectionId'. */ public class ConnectionId extends ScalarFunction - implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable, Nondeterministic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CurrentUser.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CurrentUser.java index f36e4548ebc5c4..5f00e374716f32 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CurrentUser.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/CurrentUser.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.StringType; @@ -32,7 +33,7 @@ * ScalarFunction 'CurrentUser'. */ public class CurrentUser extends ScalarFunction - implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable, Nondeterministic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(StringType.INSTANCE).args() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Database.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Database.java index acb31f5ae4db07..c213fc6bc5c29d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Database.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.VarcharType; @@ -32,7 +33,7 @@ * ScalarFunction 'database'. */ public class Database extends ScalarFunction - implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable, Nondeterministic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/User.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/User.java index 3a53c291f9db74..6cf547f8ab5ac1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/User.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/User.java @@ -20,6 +20,7 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.VarcharType; @@ -32,7 +33,7 @@ * ScalarFunction 'User'. */ public class User extends ScalarFunction - implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable { + implements LeafExpression, ExplicitlyCastableSignature, AlwaysNotNullable, Nondeterministic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT).args() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java index e3e25481691c02..26602435651fab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; import org.apache.doris.nereids.trees.expressions.functions.CustomSignature; +import org.apache.doris.nereids.trees.expressions.functions.Nondeterministic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DataType; @@ -44,7 +45,8 @@ import java.util.stream.Collectors; /** TableValuedFunction */ -public abstract class TableValuedFunction extends BoundFunction implements UnaryExpression, CustomSignature { +public abstract class TableValuedFunction extends BoundFunction + implements UnaryExpression, CustomSignature, Nondeterministic { protected final Supplier catalogFunctionCache = Suppliers.memoize(this::toCatalogFunction); protected final Supplier tableCache = Suppliers.memoize(() -> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/AliasUdfBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/AliasUdfBuilder.java index 733bd5fcae1164..1f15b7e6049a25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/AliasUdfBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/AliasUdfBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions.udf; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.ReflectionUtils; import org.apache.doris.nereids.rules.expression.rules.FunctionBinder; import org.apache.doris.nereids.trees.expressions.Expression; @@ -72,7 +73,7 @@ public boolean canApply(List arguments) { } @Override - public Expression build(String name, List arguments) { + public Pair build(String name, List arguments) { // use AliasFunction to process TypeCoercion BoundFunction boundAliasFunction = ((BoundFunction) aliasUdf.withChildren(arguments.stream() .map(Expression.class::cast).collect(Collectors.toList()))); @@ -95,7 +96,7 @@ public Expression build(String name, List arguments) { replaceMap.put(slots.get(parameter), inputs.get(i)); } - return SlotReplacer.INSTANCE.replace(boundFunction, replaceMap); + return Pair.of(SlotReplacer.INSTANCE.replace(boundFunction, replaceMap), boundAliasFunction); } private static class SlotReplacer extends DefaultExpressionRewriter> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdafBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdafBuilder.java index 89073b4eb77e4d..0d8ad443f533e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdafBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdafBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions.udf; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.ReflectionUtils; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; @@ -32,19 +33,19 @@ * function builder for java udaf */ public class JavaUdafBuilder extends UdfBuilder { - private final JavaUdaf udf; + private final JavaUdaf udaf; private final int arity; private final boolean isVarArgs; - public JavaUdafBuilder(JavaUdaf udf) { - this.udf = udf; - this.isVarArgs = udf.hasVarArguments(); - this.arity = udf.arity(); + public JavaUdafBuilder(JavaUdaf udaf) { + this.udaf = udaf; + this.isVarArgs = udaf.hasVarArguments(); + this.arity = udaf.arity(); } @Override public List getArgTypes() { - return Suppliers.memoize(() -> udf.getSignatures().get(0).argumentsTypes.stream() + return Suppliers.memoize(() -> udaf.getSignatures().get(0).argumentsTypes.stream() .map(DataType.class::cast) .collect(Collectors.toList())).get(); } @@ -71,7 +72,11 @@ public boolean canApply(List arguments) { } @Override - public BoundFunction build(String name, List arguments) { - return udf.withChildren(arguments.stream().map(Expression.class::cast).collect(Collectors.toList())); + public Pair build(String name, List arguments) { + return Pair.ofSame((JavaUdaf) udaf.withChildren( + arguments.stream() + .map(Expression.class::cast) + .collect(Collectors.toList())) + ); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdfBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdfBuilder.java index a78f8cfe4b46ae..04d4741286a4ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdfBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/udf/JavaUdfBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.expressions.functions.udf; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.ReflectionUtils; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; @@ -73,14 +74,14 @@ public boolean canApply(List arguments) { } @Override - public BoundFunction build(String name, List arguments) { + public Pair build(String name, List arguments) { List exprs = arguments.stream().map(Expression.class::cast).collect(Collectors.toList()); List argTypes = udf.getSignatures().get(0).argumentsTypes; List processedExprs = Lists.newArrayList(); for (int i = 0; i < exprs.size(); ++i) { - processedExprs.add(TypeCoercionUtils.castIfNotSameType(exprs.get(i), ((DataType) argTypes.get(i)))); + processedExprs.add(TypeCoercionUtils.castIfNotSameType(exprs.get(i), argTypes.get(i))); } - return udf.withChildren(processedExprs); + return Pair.ofSame(udf.withChildren(processedExprs)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java index 38951ea9e453b4..89910b74374f38 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java @@ -27,7 +27,6 @@ import org.apache.doris.nereids.types.coercion.DateLikeType; import org.apache.doris.nereids.util.DateTimeFormatterUtils; import org.apache.doris.nereids.util.DateUtils; -import org.apache.doris.nereids.util.StandardDateFormat; import com.google.common.collect.ImmutableSet; @@ -365,12 +364,36 @@ public double getDouble() { @Override public String getStringValue() { + if (0 <= year && year <= 9999 && 0 <= month && month <= 99 && 0 <= day && day <= 99) { + char[] format = new char[] {'0', '0', '0', '0', '-', '0', '0', '-', '0', '0'}; + int offset = 3; + long year = this.year; + while (year > 0) { + format[offset--] = (char) ('0' + (year % 10)); + year /= 10; + } + + offset = 6; + long month = this.month; + while (month > 0) { + format[offset--] = (char) ('0' + (month % 10)); + month /= 10; + } + + offset = 9; + long day = this.day; + while (day > 0) { + format[offset--] = (char) ('0' + (day % 10)); + day /= 10; + } + return String.valueOf(format); + } return String.format("%04d-%02d-%02d", year, month, day); } @Override public String toSql() { - return String.format("'%s'", toString()); + return "'" + getStringValue() + "'"; } @Override @@ -380,7 +403,7 @@ public R accept(ExpressionVisitor visitor, C context) { @Override public String toString() { - return String.format("%04d-%02d-%02d", year, month, day); + return getStringValue(); } @Override @@ -401,22 +424,19 @@ public long getDay() { } public Expression plusDays(long days) { - return fromJavaDateType(DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusDays(days)); + return fromJavaDateType(toJavaDateType().plusDays(days)); } public Expression plusMonths(long months) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusMonths(months)); + return fromJavaDateType(toJavaDateType().plusMonths(months)); } public Expression plusWeeks(long weeks) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusWeeks(weeks)); + return fromJavaDateType(toJavaDateType().plusWeeks(weeks)); } public Expression plusYears(long years) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusYears(years)); + return fromJavaDateType(toJavaDateType().plusYears(years)); } public LocalDateTime toJavaDateType() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java index 3d5d049f13c2e9..0a0f095fe5fd15 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteral.java @@ -25,7 +25,6 @@ import org.apache.doris.nereids.types.DateTimeType; import org.apache.doris.nereids.types.coercion.DateLikeType; import org.apache.doris.nereids.util.DateUtils; -import org.apache.doris.nereids.util.StandardDateFormat; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -195,16 +194,63 @@ public double getDouble() { @Override public String toSql() { - return String.format("'%s'", toString()); + return "'" + getStringValue() + "'"; } @Override public String toString() { - return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second); + return getStringValue(); } @Override public String getStringValue() { + if (0 <= year && year <= 9999 && 0 <= month && month <= 99 && 0 <= day && day <= 99 + && 0 <= hour && hour <= 99 && 0 <= minute && minute <= 99 && 0 <= second && second <= 99) { + char[] format = new char[] { + '0', '0', '0', '0', '-', '0', '0', '-', '0', '0', ' ', '0', '0', ':', '0', '0', ':', '0', '0'}; + int offset = 3; + long year = this.year; + while (year > 0) { + format[offset--] = (char) ('0' + (year % 10)); + year /= 10; + } + + offset = 6; + long month = this.month; + while (month > 0) { + format[offset--] = (char) ('0' + (month % 10)); + month /= 10; + } + + offset = 9; + long day = this.day; + while (day > 0) { + format[offset--] = (char) ('0' + (day % 10)); + day /= 10; + } + + offset = 12; + long hour = this.hour; + while (hour > 0) { + format[offset--] = (char) ('0' + (hour % 10)); + hour /= 10; + } + + offset = 15; + long minute = this.minute; + while (minute > 0) { + format[offset--] = (char) ('0' + (minute % 10)); + minute /= 10; + } + + offset = 18; + long second = this.second; + while (second > 0) { + format[offset--] = (char) ('0' + (second % 10)); + second /= 10; + } + return String.valueOf(format); + } return String.format("%04d-%02d-%02d %02d:%02d:%02d", year, month, day, hour, minute, second); } @@ -213,39 +259,32 @@ public LiteralExpr toLegacyLiteral() { return new org.apache.doris.analysis.DateLiteral(year, month, day, hour, minute, second, Type.DATETIME); } - public Expression plusYears(long years) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusYears(years)); + public Expression plusDays(long days) { + return fromJavaDateType(toJavaDateType().plusDays(days)); } public Expression plusMonths(long months) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusMonths(months)); + return fromJavaDateType(toJavaDateType().plusMonths(months)); } public Expression plusWeeks(long weeks) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusWeeks(weeks)); + return fromJavaDateType(toJavaDateType().plusWeeks(weeks)); } - public Expression plusDays(long days) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusDays(days)); + public Expression plusYears(long years) { + return fromJavaDateType(toJavaDateType().plusYears(years)); } public Expression plusHours(long hours) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusHours(hours)); + return fromJavaDateType(toJavaDateType().plusHours(hours)); } public Expression plusMinutes(long minutes) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusMinutes(minutes)); + return fromJavaDateType(toJavaDateType().plusMinutes(minutes)); } public Expression plusSeconds(long seconds) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER, getStringValue()).plusSeconds(seconds)); + return fromJavaDateType(toJavaDateType().plusSeconds(seconds)); } public long getHour() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java index 53cfe1e1835bb8..2417d92e89addf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeV2Literal.java @@ -110,10 +110,72 @@ public String toString() { @Override public String getStringValue() { + int scale = getDataType().getScale(); + if (scale <= 0) { + return super.getStringValue(); + } + + if (0 <= year && year <= 9999 && 0 <= month && month <= 99 && 0 <= day && day <= 99 + && 0 <= hour && hour <= 99 && 0 <= minute && minute <= 99 && 0 <= second && second <= 99 + && 0 <= microSecond && microSecond <= MAX_MICROSECOND) { + char[] format = new char[] { + '0', '0', '0', '0', '-', '0', '0', '-', '0', '0', ' ', '0', '0', ':', '0', '0', ':', '0', '0', + '.', '0', '0', '0', '0', '0', '0'}; + int offset = 3; + long year = this.year; + while (year > 0) { + format[offset--] = (char) ('0' + (year % 10)); + year /= 10; + } + + offset = 6; + long month = this.month; + while (month > 0) { + format[offset--] = (char) ('0' + (month % 10)); + month /= 10; + } + + offset = 9; + long day = this.day; + while (day > 0) { + format[offset--] = (char) ('0' + (day % 10)); + day /= 10; + } + + offset = 12; + long hour = this.hour; + while (hour > 0) { + format[offset--] = (char) ('0' + (hour % 10)); + hour /= 10; + } + + offset = 15; + long minute = this.minute; + while (minute > 0) { + format[offset--] = (char) ('0' + (minute % 10)); + minute /= 10; + } + + offset = 18; + long second = this.second; + while (second > 0) { + format[offset--] = (char) ('0' + (second % 10)); + second /= 10; + } + + offset = 19 + scale; + long microSecond = (int) (this.microSecond / Math.pow(10, DateTimeV2Type.MAX_SCALE - scale)); + while (microSecond > 0) { + format[offset--] = (char) ('0' + (microSecond % 10)); + microSecond /= 10; + } + return String.valueOf(format, 0, 20 + scale); + } + return String.format("%04d-%02d-%02d %02d:%02d:%02d" - + (getDataType().getScale() > 0 ? ".%0" + getDataType().getScale() + "d" : ""), + + (scale > 0 ? ".%0" + scale + "d" : ""), year, month, day, hour, minute, second, - (int) (microSecond / Math.pow(10, DateTimeV2Type.MAX_SCALE - getDataType().getScale()))); + (int) (microSecond / Math.pow(10, DateTimeV2Type.MAX_SCALE - scale))); } public String getMicrosecondString() { @@ -124,59 +186,36 @@ public String getMicrosecondString() { (int) (microSecond / Math.pow(10, DateTimeV2Type.MAX_SCALE - getDataType().getScale()))); } - @Override - public Expression plusYears(long years) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusYears(years), getDataType().getScale()); + public Expression plusDays(long days) { + return fromJavaDateType(toJavaDateType().plusDays(days), getDataType().getScale()); } - @Override public Expression plusMonths(long months) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusMonths(months), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusMonths(months), getDataType().getScale()); } - @Override public Expression plusWeeks(long weeks) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusWeeks(weeks), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusWeeks(weeks), getDataType().getScale()); } - @Override - public Expression plusDays(long days) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusDays(days), getDataType().getScale()); + public Expression plusYears(long years) { + return fromJavaDateType(toJavaDateType().plusYears(years), getDataType().getScale()); } - @Override public Expression plusHours(long hours) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusHours(hours), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusHours(hours), getDataType().getScale()); } - @Override public Expression plusMinutes(long minutes) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusMinutes(minutes), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusMinutes(minutes), getDataType().getScale()); } - @Override public Expression plusSeconds(long seconds) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getStringValue()) - .plusSeconds(seconds), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusSeconds(seconds), getDataType().getScale()); } public Expression plusMicroSeconds(long microSeconds) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_TIME_FORMATTER_TO_MICRO_SECOND, getFullMicroSecondValue()) - .plusNanos(microSeconds * 1000L), getDataType().getScale()); + return fromJavaDateType(toJavaDateType().plusNanos(microSeconds * 1000L), getDataType().getScale()); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java index f522265f1617fb..d51a0eccc82cb7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateV2Literal.java @@ -24,8 +24,6 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DateTimeV2Type; import org.apache.doris.nereids.types.DateV2Type; -import org.apache.doris.nereids.util.DateUtils; -import org.apache.doris.nereids.util.StandardDateFormat; import java.time.LocalDateTime; @@ -53,22 +51,19 @@ public R accept(ExpressionVisitor visitor, C context) { } public Expression plusDays(long days) { - return fromJavaDateType(DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusDays(days)); + return fromJavaDateType(toJavaDateType().plusDays(days)); } public Expression plusMonths(long months) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusMonths(months)); + return fromJavaDateType(toJavaDateType().plusMonths(months)); } public Expression plusWeeks(long weeks) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusWeeks(weeks)); + return fromJavaDateType(toJavaDateType().plusWeeks(weeks)); } public Expression plusYears(long years) { - return fromJavaDateType( - DateUtils.getTime(StandardDateFormat.DATE_FORMATTER, getStringValue()).plusYears(years)); + return fromJavaDateType(toJavaDateType().plusYears(years)); } public static Expression fromJavaDateType(LocalDateTime dateTime) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java index 4747aa84898a8e..ecb04ac7e49748 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/AbstractPlan.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.plans.TreeStringPlan.TreeStringNode; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.util.MutableState; import org.apache.doris.nereids.util.TreeStringUtils; @@ -34,6 +35,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.json.JSONArray; import org.json.JSONObject; @@ -120,9 +122,31 @@ public boolean canBind() { public String treeString() { return TreeStringUtils.treeString(this, plan -> plan.toString(), - plan -> (List) ((Plan) plan).children(), - plan -> (List) ((Plan) plan).extraPlans(), - plan -> ((Plan) plan).displayExtraPlanFirst()); + plan -> { + if (plan instanceof TreeStringPlan) { + Optional treeStringNode = ((TreeStringPlan) plan).parseTreeStringNode(); + return treeStringNode.isPresent() ? ImmutableList.of(treeStringNode.get()) : ImmutableList.of(); + } + if (plan instanceof TreeStringNode) { + return (List) ((TreeStringNode) plan).children; + } + if (!(plan instanceof Plan)) { + return ImmutableList.of(); + } + return (List) ((Plan) plan).children(); + }, + plan -> { + if (!(plan instanceof Plan)) { + return ImmutableList.of(); + } + return (List) ((Plan) plan).extraPlans(); + }, + plan -> { + if (!(plan instanceof Plan)) { + return false; + } + return ((Plan) plan).displayExtraPlanFirst(); + }); } /** top toJson method, can be override by specific operator */ 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 daab18438b4774..165dcd5e3cab7f 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 @@ -27,6 +27,7 @@ public enum PlanType { // logical plans // logical relations + LOGICAL_SQL_CACHE, LOGICAL_BOUND_RELATION, LOGICAL_CTE_CONSUMER, LOGICAL_FILE_SCAN, @@ -83,6 +84,7 @@ public enum PlanType { // physical plans // physical relations + PHYSICAL_SQL_CACHE, PHYSICAL_CTE_CONSUMER, PHYSICAL_EMPTY_RELATION, PHYSICAL_ES_SCAN, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/TreeStringPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/TreeStringPlan.java new file mode 100644 index 00000000000000..c392b9ed9dd4b9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/TreeStringPlan.java @@ -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. + +package org.apache.doris.nereids.trees.plans; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.commons.io.LineIterator; + +import java.io.StringReader; +import java.util.List; +import java.util.Optional; +import java.util.Stack; + +/** TreeStringPlan */ +public interface TreeStringPlan { + String getChildrenTreeString(); + + /** parseTreeStringNode */ + default Optional parseTreeStringNode() { + String treeString = getChildrenTreeString(); + LineIterator lineIt = new LineIterator(new StringReader(treeString)); + + if (!lineIt.hasNext()) { + return Optional.empty(); + } + + Stack parseStack = new Stack<>(); + parseStack.push(new ParseTreeStringNodeContext(lineIt.next())); + + while (lineIt.hasNext()) { + String line = lineIt.next(); + int level = getLevel(line); + while (parseStack.size() >= level) { + ParseTreeStringNodeContext child = parseStack.pop(); + parseStack.peek().children.add( + new TreeStringNode(child.currentString, ImmutableList.copyOf(child.children)) + ); + } + parseStack.push(new ParseTreeStringNodeContext(line.substring((level - 1) * 3))); + } + + while (parseStack.size() > 1) { + ParseTreeStringNodeContext child = parseStack.pop(); + parseStack.peek().children.add( + new TreeStringNode(child.currentString, ImmutableList.copyOf(child.children)) + ); + } + + ParseTreeStringNodeContext top = parseStack.pop(); + return Optional.of(new TreeStringNode(top.currentString, ImmutableList.copyOf(top.children))); + } + + /** TreeStringNode */ + class TreeStringNode { + /** currentString */ + public final String currentString; + /** children */ + public final List children; + + public TreeStringNode(String currentString, List children) { + this.currentString = currentString; + this.children = children; + } + + @Override + public String toString() { + return currentString.trim(); + } + } + + /** ParseTreeStringNodeContext */ + class ParseTreeStringNodeContext { + /** currentString */ + public final String currentString; + /** children */ + public final List children; + + public ParseTreeStringNodeContext(String currentString) { + this.currentString = currentString; + this.children = Lists.newArrayList(); + } + } + + /** getLevel */ + static int getLevel(String currentString) { + int prefix = 0; + for (int i = 0; i < currentString.length(); i++) { + char c = currentString.charAt(i); + if (c == ' ' || c == '|' || c == '-' || c == '+') { + prefix++; + } else { + break; + } + } + return prefix / 3 + 1; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/SqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/SqlCache.java new file mode 100644 index 00000000000000..bd71fa1fd75ebe --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/SqlCache.java @@ -0,0 +1,36 @@ +// 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.algebra; + +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSqlCache; +import org.apache.doris.qe.cache.CacheAnalyzer; + +/** SqlCache */ +public interface SqlCache { + /** computeChild */ + static

P computeChild(CacheAnalyzer cacheAnalyzer) { + NereidsPlanner nereidsPlanner = (NereidsPlanner) cacheAnalyzer.getPlanner(); + PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); + while (physicalPlan instanceof PhysicalSqlCache) { + physicalPlan = (PhysicalPlan) physicalPlan.child(0); + } + return (P) physicalPlan; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCheckPolicy.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCheckPolicy.java index bda3b1f49d570d..69b669b25f7677 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCheckPolicy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCheckPolicy.java @@ -22,6 +22,8 @@ import org.apache.doris.mysql.privilege.DataMaskPolicy; import org.apache.doris.mysql.privilege.RowFilterPolicy; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.SqlCacheContext; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundAlias; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.memo.GroupExpression; @@ -147,6 +149,8 @@ public RelatedPolicy findPolicy(LogicalRelation logicalRelation, CascadesContext ImmutableList.Builder dataMasks = ImmutableList.builderWithExpectedSize(logicalRelation.getOutput().size()); + StatementContext statementContext = cascadesContext.getStatementContext(); + Optional sqlCacheContext = statementContext.getSqlCacheContext(); boolean hasDataMask = false; for (Slot slot : logicalRelation.getOutput()) { Optional dataMaskPolicy = accessManager.evalDataMaskPolicy( @@ -165,12 +169,19 @@ public RelatedPolicy findPolicy(LogicalRelation logicalRelation, CascadesContext } else { dataMasks.add(slot); } + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().addDataMaskPolicy(ctlName, dbName, tableName, slot.getName(), dataMaskPolicy); + } } - List policies = accessManager.evalRowFilterPolicies( + List rowPolicies = accessManager.evalRowFilterPolicies( currentUserIdentity, ctlName, dbName, tableName); + if (sqlCacheContext.isPresent()) { + sqlCacheContext.get().setRowFilterPolicy(ctlName, dbName, tableName, rowPolicies); + } + return new RelatedPolicy( - Optional.ofNullable(CollectionUtils.isEmpty(policies) ? null : mergeRowPolicy(policies)), + Optional.ofNullable(CollectionUtils.isEmpty(rowPolicies) ? null : mergeRowPolicy(rowPolicies)), hasDataMask ? Optional.of(dataMasks.build()) : Optional.empty() ); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java new file mode 100644 index 00000000000000..c0c728d8fc9a86 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSqlCache.java @@ -0,0 +1,147 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.logical; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.FdItem; +import org.apache.doris.nereids.properties.FunctionalDependencies.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.TreeStringPlan; +import org.apache.doris.nereids.trees.plans.algebra.SqlCache; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.proto.InternalService; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** LogicalSqlCache */ +public class LogicalSqlCache extends LogicalLeaf implements SqlCache, TreeStringPlan { + private final TUniqueId queryId; + private final List columnLabels; + private final List resultExprs; + private final List cacheValues; + private final String backendAddress; + private final String planBody; + + /** LogicalSqlCache */ + public LogicalSqlCache(TUniqueId queryId, + List columnLabels, List resultExprs, + List cacheValues, String backendAddress, String planBody) { + super(PlanType.LOGICAL_SQL_CACHE, Optional.empty(), Optional.empty()); + this.queryId = Objects.requireNonNull(queryId, "queryId can not be null"); + this.columnLabels = Objects.requireNonNull(columnLabels, "columnLabels can not be null"); + this.resultExprs = Objects.requireNonNull(resultExprs, "resultExprs can not be null"); + this.cacheValues = Objects.requireNonNull(cacheValues, "cacheValues can not be null"); + this.backendAddress = Objects.requireNonNull(backendAddress, "backendAddress can not be null"); + this.planBody = Objects.requireNonNull(planBody, "planBody can not be null"); + } + + public TUniqueId getQueryId() { + return queryId; + } + + public List getCacheValues() { + return cacheValues; + } + + public String getBackendAddress() { + return backendAddress; + } + + public List getColumnLabels() { + return columnLabels; + } + + public List getResultExprs() { + return resultExprs; + } + + public String getPlanBody() { + return planBody; + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalSqlCache[" + id.asInt() + "]", + "queryId", DebugUtil.printId(queryId) + ); + } + + @Override + public Plan withChildren(List children) { + throw new UnsupportedOperationException(); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalSqlCache(this, context); + } + + @Override + public List getExpressions() { + return ImmutableList.of(); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + throw new UnsupportedOperationException(); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + throw new UnsupportedOperationException(); + } + + @Override + public List computeOutput() { + throw new UnsupportedOperationException(); + } + + @Override + public String getChildrenTreeString() { + return planBody; + } + + @Override + public ImmutableSet computeFdItems() { + return ImmutableSet.of(); + } + + @Override + public void computeUnique(Builder fdBuilder) { + + } + + @Override + public void computeUniform(Builder fdBuilder) { + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java new file mode 100644 index 00000000000000..6e223a49cde915 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSqlCache.java @@ -0,0 +1,139 @@ +// 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.physical; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.FunctionalDependencies; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; +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.TreeStringPlan; +import org.apache.doris.nereids.trees.plans.algebra.SqlCache; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.proto.InternalService; +import org.apache.doris.statistics.Statistics; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** PhysicalSqlCache */ +public class PhysicalSqlCache extends PhysicalLeaf implements SqlCache, TreeStringPlan { + private final TUniqueId queryId; + private final List columnLabels; + private final List resultExprs; + private final List cacheValues; + private final String backendAddress; + private final String planBody; + + /** PhysicalSqlCache */ + public PhysicalSqlCache(TUniqueId queryId, + List columnLabels, List resultExprs, + List cacheValues, String backendAddress, String planBody) { + super(PlanType.PHYSICAL_SQL_CACHE, Optional.empty(), + new LogicalProperties(() -> ImmutableList.of(), () -> FunctionalDependencies.EMPTY_FUNC_DEPS)); + this.queryId = Objects.requireNonNull(queryId, "queryId can not be null"); + this.columnLabels = Objects.requireNonNull(columnLabels, "colNames can not be null"); + this.resultExprs = Objects.requireNonNull(resultExprs, "resultExprs can not be null"); + this.cacheValues = Objects.requireNonNull(cacheValues, "cacheValues can not be null"); + this.backendAddress = Objects.requireNonNull(backendAddress, "backendAddress can not be null"); + this.planBody = Objects.requireNonNull(planBody, "planBody can not be null"); + } + + public TUniqueId getQueryId() { + return queryId; + } + + public List getCacheValues() { + return cacheValues; + } + + public String getBackendAddress() { + return backendAddress; + } + + public List getColumnLabels() { + return columnLabels; + } + + public List getResultExprs() { + return resultExprs; + } + + public String getPlanBody() { + return planBody; + } + + @Override + public String toString() { + return Utils.toSqlString("PhysicalSqlCache[" + id.asInt() + "]", + "queryId", DebugUtil.printId(queryId), + "backend", backendAddress + ); + } + + @Override + public Plan withChildren(List children) { + throw new UnsupportedOperationException(); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPhysicalSqlCache(this, context); + } + + @Override + public List getExpressions() { + return ImmutableList.of(); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + throw new UnsupportedOperationException(); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + throw new UnsupportedOperationException(); + } + + @Override + public List computeOutput() { + throw new UnsupportedOperationException(); + } + + @Override + public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { + throw new UnsupportedOperationException(); + } + + @Override + public String getChildrenTreeString() { + return planBody; + } +} 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 82c99abb808642..9a21e5b14e0195 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 @@ -45,6 +45,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; @@ -73,6 +74,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalRepeat; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSqlCache; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; @@ -128,6 +130,9 @@ public R visitPhysicalSink(PhysicalSink physicalSink, C context) // ******************************* // Logical plans // ******************************* + public R visitLogicalSqlCache(LogicalSqlCache sqlCache, C context) { + return visit(sqlCache, context); + } public R visitLogicalAggregate(LogicalAggregate aggregate, C context) { return visit(aggregate, context); @@ -248,6 +253,9 @@ public R visitLogicalWindow(LogicalWindow window, C context) { // ******************************* // Physical plans // ******************************* + public R visitPhysicalSqlCache(PhysicalSqlCache sqlCache, C context) { + return visit(sqlCache, context); + } public R visitPhysicalHashAggregate(PhysicalHashAggregate agg, C context) { return visit(agg, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 16c6b6afb82983..2824466059028b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -17,6 +17,7 @@ package org.apache.doris.qe; +import org.apache.doris.analysis.ExplainOptions; import org.apache.doris.analysis.InsertStmt; import org.apache.doris.analysis.KillStmt; import org.apache.doris.analysis.LiteralExpr; @@ -37,6 +38,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.NotImplementedException; +import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.SqlParserUtils; @@ -49,6 +51,7 @@ import org.apache.doris.mysql.MysqlPacket; import org.apache.doris.mysql.MysqlSerializer; import org.apache.doris.mysql.MysqlServerStatusFlag; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.nereids.exceptions.ParseException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; @@ -56,10 +59,14 @@ import org.apache.doris.nereids.parser.Dialect; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.stats.StatsErrorEstimator; +import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; import org.apache.doris.plugin.DialectConverterPlugin; import org.apache.doris.plugin.PluginMgr; import org.apache.doris.proto.Data; import org.apache.doris.qe.QueryState.MysqlStateType; +import org.apache.doris.qe.cache.CacheAnalyzer; import org.apache.doris.thrift.TExprNode; import org.apache.doris.thrift.TMasterOpRequest; import org.apache.doris.thrift.TMasterOpResult; @@ -67,6 +74,8 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.StringUtils; @@ -80,6 +89,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import javax.annotation.Nullable; @@ -231,33 +241,46 @@ public void executeQuery(MysqlCommand mysqlCommand, String originStmt) throws Ex String sqlHash = DigestUtils.md5Hex(convertedStmt); ctx.setSqlHash(sqlHash); + SessionVariable sessionVariable = ctx.getSessionVariable(); + boolean wantToParseSqlFromSqlCache = sessionVariable.isEnableNereidsPlanner() + && CacheAnalyzer.canUseSqlCache(sessionVariable); List stmts = null; Exception nereidsParseException = null; long parseSqlStartTime = System.currentTimeMillis(); + List cachedStmts = null; // Nereids do not support prepare and execute now, so forbid prepare command, only process query command - if (mysqlCommand == MysqlCommand.COM_QUERY && ctx.getSessionVariable().isEnableNereidsPlanner()) { - try { - stmts = new NereidsParser().parseSQL(convertedStmt, ctx.getSessionVariable()); - } catch (NotSupportedException e) { - // Parse sql failed, audit it and return - handleQueryException(e, convertedStmt, null, null); - return; - } catch (ParseException e) { - if (LOG.isDebugEnabled()) { - LOG.debug("Nereids parse sql failed. Reason: {}. Statement: \"{}\".", - e.getMessage(), convertedStmt); + if (mysqlCommand == MysqlCommand.COM_QUERY && sessionVariable.isEnableNereidsPlanner()) { + if (wantToParseSqlFromSqlCache) { + cachedStmts = parseFromSqlCache(originStmt); + if (cachedStmts != null) { + stmts = cachedStmts; } - // ATTN: Do not set nereidsParseException in this case. - // Because ParseException means the sql is not supported by Nereids. - // It should be parsed by old parser, so not setting nereidsParseException to avoid - // suppressing the exception thrown by old parser. - } catch (Exception e) { - // TODO: We should catch all exception here until we support all query syntax. - if (LOG.isDebugEnabled()) { - LOG.debug("Nereids parse sql failed with other exception. Reason: {}. Statement: \"{}\".", - e.getMessage(), convertedStmt); + } + + if (cachedStmts == null) { + try { + stmts = new NereidsParser().parseSQL(convertedStmt, sessionVariable); + } catch (NotSupportedException e) { + // Parse sql failed, audit it and return + handleQueryException(e, convertedStmt, null, null); + return; + } catch (ParseException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Nereids parse sql failed. Reason: {}. Statement: \"{}\".", + e.getMessage(), convertedStmt); + } + // ATTN: Do not set nereidsParseException in this case. + // Because ParseException means the sql is not supported by Nereids. + // It should be parsed by old parser, so not setting nereidsParseException to avoid + // suppressing the exception thrown by old parser. + } catch (Exception e) { + // TODO: We should catch all exception here until we support all query syntax. + if (LOG.isDebugEnabled()) { + LOG.debug("Nereids parse sql failed with other exception. Reason: {}. Statement: \"{}\".", + e.getMessage(), convertedStmt); + } + nereidsParseException = e; } - nereidsParseException = e; } } @@ -292,58 +315,104 @@ public void executeQuery(MysqlCommand mysqlCommand, String originStmt) throws Ex boolean usingOrigSingleStmt = origSingleStmtList != null && origSingleStmtList.size() == stmts.size(); for (int i = 0; i < stmts.size(); ++i) { String auditStmt = usingOrigSingleStmt ? origSingleStmtList.get(i) : convertedStmt; + try { + ctx.getState().reset(); + if (i > 0) { + ctx.resetReturnRows(); + } - ctx.getState().reset(); - if (i > 0) { - ctx.resetReturnRows(); - } - - StatementBase parsedStmt = stmts.get(i); - parsedStmt.setOrigStmt(new OriginStatement(convertedStmt, i)); - parsedStmt.setUserInfo(ctx.getCurrentUserIdentity()); - executor = new StmtExecutor(ctx, parsedStmt); - executor.getProfile().getSummaryProfile().setParseSqlStartTime(parseSqlStartTime); - executor.getProfile().getSummaryProfile().setParseSqlFinishTime(parseSqlFinishTime); - ctx.setExecutor(executor); + StatementBase parsedStmt = stmts.get(i); + parsedStmt.setOrigStmt(new OriginStatement(convertedStmt, i)); + parsedStmt.setUserInfo(ctx.getCurrentUserIdentity()); + executor = new StmtExecutor(ctx, parsedStmt); + executor.getProfile().getSummaryProfile().setParseSqlStartTime(parseSqlStartTime); + executor.getProfile().getSummaryProfile().setParseSqlFinishTime(parseSqlFinishTime); + ctx.setExecutor(executor); - try { - executor.execute(); - if (connectType.equals(ConnectType.MYSQL)) { - if (i != stmts.size() - 1) { - ctx.getState().serverStatus |= MysqlServerStatusFlag.SERVER_MORE_RESULTS_EXISTS; - if (ctx.getState().getStateType() != MysqlStateType.ERR) { - finalizeCommand(); + try { + executor.execute(); + if (connectType.equals(ConnectType.MYSQL)) { + if (i != stmts.size() - 1) { + ctx.getState().serverStatus |= MysqlServerStatusFlag.SERVER_MORE_RESULTS_EXISTS; + if (ctx.getState().getStateType() != MysqlStateType.ERR) { + finalizeCommand(); + } + } + } else if (connectType.equals(ConnectType.ARROW_FLIGHT_SQL)) { + if (!ctx.isReturnResultFromLocal()) { + returnResultFromRemoteExecutor.add(executor); + } + Preconditions.checkState(ctx.getFlightSqlChannel().resultNum() <= 1); + if (ctx.getFlightSqlChannel().resultNum() == 1 && i != stmts.size() - 1) { + String errMsg = "Only be one stmt that returns the result and it is at the end. " + + "stmts.size(): " + stmts.size(); + LOG.warn(errMsg); + ctx.getState().setError(ErrorCode.ERR_ARROW_FLIGHT_SQL_MUST_ONLY_RESULT_STMT, errMsg); + ctx.getState().setErrType(QueryState.ErrType.OTHER_ERR); + break; } } - } else if (connectType.equals(ConnectType.ARROW_FLIGHT_SQL)) { - if (!ctx.isReturnResultFromLocal()) { - returnResultFromRemoteExecutor.add(executor); - } - Preconditions.checkState(ctx.getFlightSqlChannel().resultNum() <= 1); - if (ctx.getFlightSqlChannel().resultNum() == 1 && i != stmts.size() - 1) { - String errMsg = "Only be one stmt that returns the result and it is at the end. stmts.size(): " - + stmts.size(); - LOG.warn(errMsg); - ctx.getState().setError(ErrorCode.ERR_ARROW_FLIGHT_SQL_MUST_ONLY_RESULT_STMT, errMsg); - ctx.getState().setErrType(QueryState.ErrType.OTHER_ERR); + auditAfterExec(auditStmt, executor.getParsedStmt(), executor.getQueryStatisticsForAuditLog(), + true); + // execute failed, skip remaining stmts + if (ctx.getState().getStateType() == MysqlStateType.ERR) { break; } + } catch (Throwable throwable) { + handleQueryException(throwable, auditStmt, executor.getParsedStmt(), + executor.getQueryStatisticsForAuditLog()); + // execute failed, skip remaining stmts + throw throwable; } - auditAfterExec(auditStmt, executor.getParsedStmt(), executor.getQueryStatisticsForAuditLog(), - true); - // execute failed, skip remaining stmts - if (ctx.getState().getStateType() == MysqlStateType.ERR) { - break; + } finally { + StatementContext statementContext = ctx.getStatementContext(); + if (statementContext != null) { + statementContext.close(); } - } catch (Throwable throwable) { - handleQueryException(throwable, auditStmt, executor.getParsedStmt(), - executor.getQueryStatisticsForAuditLog()); - // execute failed, skip remaining stmts - throw throwable; } } } + private List parseFromSqlCache(String originStmt) { + StatementContext statementContext = new StatementContext(ctx, new OriginStatement(originStmt, 0)); + ctx.setStatementContext(statementContext); + try { + Optional> explainPlan = NereidsParser.tryParseExplainPlan(originStmt); + String cacheSqlKey = originStmt; + if (explainPlan.isPresent()) { + cacheSqlKey = explainPlan.get().second; + } + Env env = ctx.getEnv(); + Optional sqlCachePlanOpt = env.getSqlCacheManager().tryParseSql(ctx, cacheSqlKey); + if (sqlCachePlanOpt.isPresent()) { + LogicalSqlCache logicalSqlCache = sqlCachePlanOpt.get(); + LogicalPlan parsedPlan = logicalSqlCache; + if (explainPlan.isPresent()) { + ExplainOptions explainOptions = explainPlan.get().first; + parsedPlan = new ExplainCommand( + explainOptions.getExplainLevel(), + parsedPlan, + explainOptions.showPlanProcess() + ); + } + + LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(parsedPlan, statementContext); + logicalPlanAdapter.setColLabels( + Lists.newArrayList(logicalSqlCache.getColumnLabels()) + ); + logicalPlanAdapter.setResultExprs(logicalSqlCache.getResultExprs()); + logicalPlanAdapter.setOrigStmt(statementContext.getOriginStatement()); + logicalPlanAdapter.setUserInfo(ctx.getCurrentUserIdentity()); + return ImmutableList.of(logicalPlanAdapter); + } + } catch (Throwable t) { + LOG.warn("Parse from sql cache failed: " + t.getMessage(), t); + } finally { + statementContext.releasePlannerResources(); + } + return null; + } + private String convertOriginStmt(String originStmt) { String convertedStmt = originStmt; @Nullable Dialect sqlDialect = Dialect.getByName(ctx.getSessionVariable().getSqlDialect()); 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 6c84dbbaee6a23..19faa555999139 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 @@ -106,6 +106,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.NereidsException; +import org.apache.doris.common.NereidsSqlCacheManager; import org.apache.doris.common.UserException; import org.apache.doris.common.Version; import org.apache.doris.common.profile.Profile; @@ -152,6 +153,7 @@ import org.apache.doris.nereids.trees.plans.commands.insert.InsertOverwriteTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.OlapInsertExecutor; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSqlCache; import org.apache.doris.planner.DataSink; import org.apache.doris.planner.GroupCommitPlanner; import org.apache.doris.planner.GroupCommitScanNode; @@ -177,6 +179,7 @@ import org.apache.doris.qe.cache.Cache; import org.apache.doris.qe.cache.CacheAnalyzer; import org.apache.doris.qe.cache.CacheAnalyzer.CacheMode; +import org.apache.doris.qe.cache.SqlCache; import org.apache.doris.rewrite.ExprRewriter; import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException; import org.apache.doris.rpc.BackendServiceProxy; @@ -1666,15 +1669,30 @@ private boolean sendCachedValues(MysqlChannel channel, List allViewStmtSet; + private final Set allViewStmtSet; + private String allViewExpandStmtListStr; + private Planner planner; + private List scanTables = Lists.newArrayList(); public Cache getCache() { return cache; @@ -106,9 +117,10 @@ public CacheAnalyzer(ConnectContext context, StatementBase parsedStmt, Planner p this.context = context; this.queryId = context.queryId(); this.parsedStmt = parsedStmt; - scanNodes = planner.getScanNodes(); - latestTable = new CacheTable(); - allViewStmtSet = new HashSet<>(); + this.scanNodes = planner.getScanNodes(); + this.latestTable = new CacheTable(); + this.allViewStmtSet = new HashSet<>(); + this.planner = planner; checkCacheConfig(); } @@ -117,7 +129,7 @@ public CacheAnalyzer(ConnectContext context, StatementBase parsedStmt, List(); + this.allViewStmtSet = new HashSet<>(); checkCacheConfig(); } @@ -134,36 +146,46 @@ private void checkCacheConfig() { } } + public TUniqueId getQueryId() { + return queryId; + } + public CacheMode getCacheMode() { return cacheMode; } + public Planner getPlanner() { + return planner; + } + public class CacheTable implements Comparable { public TableIf table; public long latestPartitionId; - public long latestVersion; - public long latestTime; + public long latestPartitionVersion; + public long latestPartitionTime; public long partitionNum; public long sumOfPartitionNum; public CacheTable() { table = null; latestPartitionId = 0; - latestVersion = 0; - latestTime = 0; + latestPartitionVersion = 0; + latestPartitionTime = 0; partitionNum = 0; sumOfPartitionNum = 0; } @Override public int compareTo(CacheTable table) { - return Long.compare(table.latestTime, this.latestTime); + return Long.compare(table.latestPartitionTime, this.latestPartitionTime); } public void debug() { if (LOG.isDebugEnabled()) { - LOG.debug("table {}, partition id {}, ver {}, time {}, partition num {}, sumOfPartitionNum: {}", - table.getName(), latestPartitionId, latestVersion, latestTime, partitionNum, sumOfPartitionNum); + LOG.debug("table {}, partition id {}, ver {}, time {}," + + "partition num {}, sumOfPartitionNum: {}", + table.getName(), latestPartitionId, latestPartitionVersion, latestPartitionTime, + partitionNum, sumOfPartitionNum); } } } @@ -180,6 +202,19 @@ public boolean enablePartitionCache() { return enablePartitionCache; } + public static boolean canUseCache(SessionVariable sessionVariable) { + return (sessionVariable.isEnableSqlCache() || sessionVariable.isEnablePartitionCache()) + && commonCacheCondition(sessionVariable); + } + + public static boolean canUseSqlCache(SessionVariable sessionVariable) { + return sessionVariable.isEnableSqlCache() && commonCacheCondition(sessionVariable); + } + + public static boolean commonCacheCondition(SessionVariable sessionVariable) { + return sessionVariable.getSqlSelectLimit() < 0 && sessionVariable.getDefaultOrderByLimit() < 0; + } + /** * Check cache mode with SQL and table * 1、Only Olap table @@ -226,15 +261,17 @@ private CacheMode innerCheckCacheMode(long now) { latestTable.debug(); addAllViewStmt(selectStmt); - String allViewExpandStmtListStr = StringUtils.join(allViewStmtSet, "|"); + if (allViewExpandStmtListStr == null) { + allViewExpandStmtListStr = StringUtils.join(allViewStmtSet, "|"); + } if (now == 0) { now = nowtime(); } if (enableSqlCache() - && (now - latestTable.latestTime) >= Config.cache_last_version_interval_second * 1000L) { + && (now - latestTable.latestPartitionTime) >= Config.cache_last_version_interval_second * 1000L) { if (LOG.isDebugEnabled()) { - LOG.debug("Query cache time:{},{},{}", now, latestTable.latestTime, + LOG.debug("Query cache time:{},{},{}", now, latestTable.latestPartitionTime, Config.cache_last_version_interval_second * 1000); } cache = new SqlCache(this.queryId, this.selectStmt); @@ -260,7 +297,7 @@ private CacheMode innerCheckCacheMode(long now) { //Check if selectStmt matches partition key //Only one table can be updated in Config.cache_last_version_interval_second range for (int i = 1; i < tblTimeList.size(); i++) { - if ((now - tblTimeList.get(i).latestTime) < Config.cache_last_version_interval_second * 1000L) { + if ((now - tblTimeList.get(i).latestPartitionTime) < Config.cache_last_version_interval_second * 1000L) { if (LOG.isDebugEnabled()) { LOG.debug("the time of other tables is newer than {} s, queryid {}", Config.cache_last_version_interval_second, DebugUtil.printId(queryId)); @@ -343,9 +380,9 @@ private CacheMode innerCheckCacheModeSetOperation(long now) { now = nowtime(); } if (enableSqlCache() - && (now - latestTable.latestTime) >= Config.cache_last_version_interval_second * 1000L) { + && (now - latestTable.latestPartitionTime) >= Config.cache_last_version_interval_second * 1000L) { if (LOG.isDebugEnabled()) { - LOG.debug("Query cache time:{},{},{}", now, latestTable.latestTime, + LOG.debug("Query cache time:{},{},{}", now, latestTable.latestPartitionTime, Config.cache_last_version_interval_second * 1000); } cache = new SqlCache(this.queryId, parsedStmt.toSql()); @@ -377,29 +414,43 @@ private CacheMode innerCheckCacheModeForNereids(long now) { return CacheMode.None; } latestTable = tblTimeList.get(0); - latestTable.sumOfPartitionNum = tblTimeList.stream().mapToLong(item -> item.partitionNum).sum(); + long sumOfPartitionNum = 0; + for (CacheTable cacheTable : tblTimeList) { + sumOfPartitionNum += cacheTable.partitionNum; + } + latestTable.sumOfPartitionNum = sumOfPartitionNum; latestTable.debug(); if (((LogicalPlanAdapter) parsedStmt).getStatementContext().getParsedStatement().isExplain()) { return CacheMode.NoNeed; } - allViewStmtSet.addAll(((LogicalPlanAdapter) parsedStmt).getViewDdlSqls()); - String allViewExpandStmtListStr = StringUtils.join(allViewStmtSet, "|"); + boolean isNewAllViewExpandStmtListStr = allViewExpandStmtListStr == null; + if (isNewAllViewExpandStmtListStr) { + allViewStmtSet.addAll(((LogicalPlanAdapter) parsedStmt).getViewDdlSqls()); + allViewExpandStmtListStr = StringUtils.join(allViewStmtSet, "|"); + } if (now == 0) { now = nowtime(); } if (enableSqlCache() - && (now - latestTable.latestTime) >= Config.cache_last_version_interval_second * 1000L) { + && (now - latestTable.latestPartitionTime) >= Config.cache_last_version_interval_second * 1000L) { if (LOG.isDebugEnabled()) { - LOG.debug("Query cache time :{},{},{}", now, latestTable.latestTime, + LOG.debug("Query cache time :{},{},{}", now, latestTable.latestPartitionTime, Config.cache_last_version_interval_second * 1000); } + + PUniqueId existsMd5 = null; + if (cache instanceof SqlCache) { + existsMd5 = ((SqlCache) cache).getOrComputeCacheMd5(); + } cache = new SqlCache(this.queryId, ((LogicalPlanAdapter) parsedStmt).getStatementContext() .getOriginStatement().originStmt); - ((SqlCache) cache).setCacheInfo(this.latestTable, allViewExpandStmtListStr); + SqlCache sqlCache = (SqlCache) cache; + sqlCache.setCacheInfo(this.latestTable, allViewExpandStmtListStr); + sqlCache.setCacheMd5(existsMd5); MetricRepo.COUNTER_CACHE_ADDED_SQL.increase(1L); return CacheMode.Sql; } @@ -409,8 +460,15 @@ private CacheMode innerCheckCacheModeForNereids(long now) { private List buildCacheTableList() { //Check the last version time of the table MetricRepo.COUNTER_QUERY_TABLE.increase(1L); - long olapScanNodeSize = scanNodes.stream().filter(node -> node instanceof OlapScanNode).count(); - long hiveScanNodeSize = scanNodes.stream().filter(node -> node instanceof HiveScanNode).count(); + long olapScanNodeSize = 0; + long hiveScanNodeSize = 0; + for (ScanNode scanNode : scanNodes) { + if (scanNode instanceof OlapScanNode) { + olapScanNodeSize++; + } else if (scanNode instanceof HiveScanNode) { + hiveScanNodeSize++; + } + } if (olapScanNodeSize > 0) { MetricRepo.COUNTER_QUERY_OLAP_TABLE.increase(1L); } @@ -621,12 +679,20 @@ private CacheTable buildCacheTableForOlapScanNode(OlapScanNode node) { OlapTable olapTable = node.getOlapTable(); cacheTable.partitionNum = node.getSelectedPartitionIds().size(); cacheTable.table = olapTable; + + DatabaseIf database = olapTable.getDatabase(); + CatalogIf catalog = database.getCatalog(); + ScanTable scanTable = new ScanTable( + new FullTableName(catalog.getName(), database.getFullName(), olapTable.getName()), + olapTable.getVisibleVersionTime(), olapTable.getVisibleVersion()); + scanTables.add(scanTable); for (Long partitionId : node.getSelectedPartitionIds()) { Partition partition = olapTable.getPartition(partitionId); - if (partition.getVisibleVersionTime() >= cacheTable.latestTime) { + if (partition.getVisibleVersionTime() >= cacheTable.latestPartitionTime) { cacheTable.latestPartitionId = partition.getId(); - cacheTable.latestTime = partition.getVisibleVersionTime(); - cacheTable.latestVersion = partition.getVisibleVersion(); + cacheTable.latestPartitionTime = partition.getVisibleVersionTime(); + cacheTable.latestPartitionVersion = partition.getVisibleVersion(); + scanTable.addScanPartition(partitionId); } } return cacheTable; @@ -636,7 +702,14 @@ private CacheTable buildCacheTableForHiveScanNode(HiveScanNode node) { CacheTable cacheTable = new CacheTable(); cacheTable.table = node.getTargetTable(); cacheTable.partitionNum = node.getReadPartitionNum(); - cacheTable.latestTime = cacheTable.table.getUpdateTime(); + cacheTable.latestPartitionTime = cacheTable.table.getUpdateTime(); + TableIf tableIf = cacheTable.table; + DatabaseIf database = tableIf.getDatabase(); + CatalogIf catalog = database.getCatalog(); + ScanTable scanTable = new ScanTable(new FullTableName( + catalog.getName(), database.getFullName(), tableIf.getName() + ), cacheTable.latestPartitionTime, 0); + scanTables.add(scanTable); return cacheTable; } @@ -695,4 +768,27 @@ public void updateCache() { } cache.updateCache(); } + + public List getScanTables() { + return scanTables; + } + + public CacheTable getLatestTable() { + return latestTable; + } + + public boolean isEqualViewString(List views) { + Set viewSet = Sets.newHashSet(); + for (TableIf view : views) { + if (view instanceof View) { + viewSet.add(((View) view).getInlineViewDef()); + } else if (view instanceof HMSExternalTable) { + viewSet.add(((HMSExternalTable) view).getViewText()); + } else { + return false; + } + } + + return StringUtils.equals(allViewExpandStmtListStr, StringUtils.join(viewSet, "|")); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheCoordinator.java index 232a9400e6c42f..529454b9fa3048 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheCoordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/CacheCoordinator.java @@ -146,12 +146,11 @@ private void clearBackend(ImmutableMap idToBackend) { } public void addBackend(Backend backend) { - if (realNodes.contains(backend.getId())) { + if (realNodes.putIfAbsent(backend.getId(), backend) != null) { return; } - realNodes.put(backend.getId(), backend); for (int i = 0; i < VIRTUAL_NODES; i++) { - String nodeName = String.valueOf(backend.getId()) + "::" + String.valueOf(i); + String nodeName = backend.getId() + "::" + i; Types.PUniqueId nodeId = CacheBeProxy.getMd5(nodeName); virtualNodes.put(nodeId.getHi(), backend); if (LOG.isDebugEnabled()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java index 52736ab2210d3d..a7351a306d2e2d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/SqlCache.java @@ -22,7 +22,9 @@ import org.apache.doris.common.util.DebugUtil; import org.apache.doris.metric.MetricRepo; import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.Types.PUniqueId; import org.apache.doris.qe.RowBatch; +import org.apache.doris.system.Backend; import org.apache.doris.thrift.TUniqueId; import org.apache.logging.log4j.LogManager; @@ -32,6 +34,7 @@ public class SqlCache extends Cache { private static final Logger LOG = LogManager.getLogger(SqlCache.class); private String originSql; + private PUniqueId cacheMd5; public SqlCache(TUniqueId queryId, SelectStmt selectStmt) { super(queryId, selectStmt); @@ -46,6 +49,18 @@ public SqlCache(TUniqueId queryId, String originSql) { public void setCacheInfo(CacheAnalyzer.CacheTable latestTable, String allViewExpandStmtListStr) { this.latestTable = latestTable; this.allViewExpandStmtListStr = allViewExpandStmtListStr; + this.cacheMd5 = null; + } + + public PUniqueId getOrComputeCacheMd5() { + if (cacheMd5 == null) { + cacheMd5 = CacheProxy.getMd5(getSqlWithViewStmt()); + } + return cacheMd5; + } + + public void setCacheMd5(PUniqueId cacheMd5) { + this.cacheMd5 = cacheMd5; } public String getSqlWithViewStmt() { @@ -57,27 +72,50 @@ public String getSqlWithViewStmt() { return cacheKey; } + public long getLatestId() { + return latestTable.latestPartitionId; + } + public long getLatestTime() { - return latestTable.latestTime; + return latestTable.latestPartitionTime; + } + + public long getLatestVersion() { + return latestTable.latestPartitionVersion; } public long getSumOfPartitionNum() { return latestTable.sumOfPartitionNum; } - public InternalService.PFetchCacheResult getCacheData(Status status) { + public static Backend findCacheBe(PUniqueId cacheMd5) { + return CacheCoordinator.getInstance().findBackend(cacheMd5); + } + + public static InternalService.PFetchCacheResult getCacheData(CacheProxy proxy, + PUniqueId cacheKeyMd5, long latestPartitionId, long latestPartitionVersion, + long latestPartitionTime, long sumOfPartitionNum, Status status) { InternalService.PFetchCacheRequest request = InternalService.PFetchCacheRequest.newBuilder() - .setSqlKey(CacheProxy.getMd5(getSqlWithViewStmt())) + .setSqlKey(cacheKeyMd5) .addParams(InternalService.PCacheParam.newBuilder() - .setPartitionKey(latestTable.latestPartitionId) - .setLastVersion(latestTable.latestVersion) - .setLastVersionTime(latestTable.latestTime) - .setPartitionNum(latestTable.sumOfPartitionNum)) + .setPartitionKey(latestPartitionId) + .setLastVersion(latestPartitionVersion) + .setLastVersionTime(latestPartitionTime) + .setPartitionNum(sumOfPartitionNum)) .build(); InternalService.PFetchCacheResult cacheResult = proxy.fetchCache(request, 10000, status); if (status.ok() && cacheResult != null && cacheResult.getStatus() == InternalService.PCacheStatus.CACHE_OK) { cacheResult = cacheResult.toBuilder().setAllCount(1).build(); + } + return cacheResult; + } + + public InternalService.PFetchCacheResult getCacheData(Status status) { + InternalService.PFetchCacheResult cacheResult = getCacheData(proxy, getOrComputeCacheMd5(), + latestTable.latestPartitionId, latestTable.latestPartitionVersion, + latestTable.latestPartitionTime, latestTable.sumOfPartitionNum, status); + if (status.ok() && cacheResult != null && cacheResult.getStatus() == InternalService.PCacheStatus.CACHE_OK) { MetricRepo.COUNTER_CACHE_HIT_SQL.increase(1L); hitRange = HitRange.Full; } @@ -105,7 +143,9 @@ public void updateCache() { InternalService.PUpdateCacheRequest updateRequest = rowBatchBuilder.buildSqlUpdateRequest(getSqlWithViewStmt(), latestTable.latestPartitionId, - latestTable.latestVersion, latestTable.latestTime, latestTable.sumOfPartitionNum); + latestTable.latestPartitionVersion, latestTable.latestPartitionTime, + latestTable.sumOfPartitionNum + ); if (updateRequest.getValuesCount() > 0) { CacheBeProxy proxy = new CacheBeProxy(); Status status = new Status(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java index 851189cf0aa95f..e4ec929f2847c1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java @@ -121,7 +121,7 @@ protected void afterRegisterBuiltinFunctions(Map> ImmutableList arguments = ImmutableList.of(Literal.of(1)); FunctionBuilder functionBuilder = functionRegistry.findFunctionBuilder("foo", arguments); - Expression function = functionBuilder.build("foo", arguments); + Expression function = functionBuilder.build("foo", arguments).first; Assertions.assertEquals(function.getClass(), ExtendFunction.class); Assertions.assertEquals(arguments, function.getArguments()); } diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ExplainAction.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ExplainAction.groovy index cf0c03fc3bd73b..63708ce7ac6f1b 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ExplainAction.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/ExplainAction.groovy @@ -102,7 +102,7 @@ class ExplainAction implements SuiteAction { for (String string : containsStrings) { if (!explainString.contains(string)) { String msg = ("Explain and check failed, expect contains '${string}'," - + "but actual explain string is:\n${explainString}").toString() + + " but actual explain string is:\n${explainString}").toString() log.info(msg) def t = new IllegalStateException(msg) throw t @@ -111,7 +111,7 @@ class ExplainAction implements SuiteAction { for (String string : notContainsStrings) { if (explainString.contains(string)) { String msg = ("Explain and check failed, expect not contains '${string}'," - + "but actual explain string is:\n${explainString}").toString() + + " but actual explain string is:\n${explainString}").toString() log.info(msg) def t = new IllegalStateException(msg) throw t diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index 924bb0938e26e2..f780cf30615e37 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -43,6 +43,7 @@ import org.apache.doris.regression.util.JdbcUtils import org.apache.doris.regression.util.Hdfs import org.apache.doris.regression.util.SuiteUtils import org.apache.doris.regression.util.DebugPoint +import org.jetbrains.annotations.NotNull import org.junit.jupiter.api.Assertions import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -55,7 +56,10 @@ import java.sql.PreparedStatement import java.sql.ResultSetMetaData import java.util.Map; import java.util.concurrent.Callable +import java.util.concurrent.ExecutorService +import java.util.concurrent.Executors import java.util.concurrent.Future +import java.util.concurrent.ThreadFactory import java.util.concurrent.atomic.AtomicBoolean import java.util.stream.Collectors import java.util.stream.LongStream @@ -170,33 +174,63 @@ class Suite implements GroovyInterceptable { return SuiteUtils.timer(actionSupplier) } + public ListenableFuture extraThread( + String threadName = null, boolean daemon = false, Closure actionSupplier) { + def executorService = Executors.newFixedThreadPool(1, new ThreadFactory() { + @Override + Thread newThread(@NotNull Runnable r) { + def thread = new Thread(r, name) + thread.setDaemon(daemon) + return thread + } + }) + + try { + def connInfo = context.threadLocalConn.get() + return MoreExecutors.listeningDecorator(executorService).submit( + buildThreadCallable(threadName, connInfo, actionSupplier) + ) + } finally { + executorService.shutdown() + } + } + public ListenableFuture thread(String threadName = null, Closure actionSupplier) { def connInfo = context.threadLocalConn.get() - return MoreExecutors.listeningDecorator(context.actionExecutors).submit((Callable) { - long startTime = System.currentTimeMillis() - def originThreadName = Thread.currentThread().name - try { - Thread.currentThread().setName(threadName == null ? originThreadName : threadName) - if (connInfo != null) { - context.connectTo(connInfo.conn.getMetaData().getURL(), connInfo.username, connInfo.password); - } - context.scriptContext.eventListeners.each { it.onThreadStarted(context) } + return MoreExecutors.listeningDecorator(context.actionExecutors).submit( + buildThreadCallable(threadName, connInfo, actionSupplier) + ) + } - return actionSupplier.call() - } catch (Throwable t) { - context.scriptContext.eventListeners.each { it.onThreadFailed(context, t) } - throw t - } finally { + private Callable buildThreadCallable(String threadName, ConnectionInfo connInfo, Closure actionSupplier) { + return new Callable() { + @Override + T call() throws Exception { + long startTime = System.currentTimeMillis() + def originThreadName = Thread.currentThread().name try { - context.closeThreadLocal() + Thread.currentThread().setName(threadName == null ? originThreadName : threadName) + if (connInfo != null) { + context.connectTo(connInfo.conn.getMetaData().getURL(), connInfo.username, connInfo.password); + } + context.scriptContext.eventListeners.each { it.onThreadStarted(context) } + + return actionSupplier.call() } catch (Throwable t) { - logger.warn("Close thread local context failed", t) + context.scriptContext.eventListeners.each { it.onThreadFailed(context, t) } + throw t + } finally { + try { + context.closeThreadLocal() + } catch (Throwable t) { + logger.warn("Close thread local context failed", t) + } + long finishTime = System.currentTimeMillis() + context.scriptContext.eventListeners.each { it.onThreadFinished(context, finishTime - startTime) } + Thread.currentThread().setName(originThreadName) } - long finishTime = System.currentTimeMillis() - context.scriptContext.eventListeners.each { it.onThreadFinished(context, finishTime - startTime) } - Thread.currentThread().setName(originThreadName) } - }) + }; } public ListenableFuture lazyCheckThread(String threadName = null, Closure actionSupplier) { diff --git a/regression-test/plugins/test_helper.groovy b/regression-test/plugins/test_helper.groovy new file mode 100644 index 00000000000000..4f7eeb3c09b380 --- /dev/null +++ b/regression-test/plugins/test_helper.groovy @@ -0,0 +1,63 @@ +// 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.apache.doris.regression.suite.Suite + +Suite.metaClass.createTestTable = { String tableName, boolean uniqueTable = false -> + Suite suite = delegate as Suite + def sql = { String sqlStr -> + suite.sql sqlStr + } + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + + sql "drop table if exists ${tableName}" + + sql """ + create table ${tableName} + ( + id int, + value int + ) + ${uniqueTable ? "unique key(id)" : ""} + partition by range(id) + ( + partition p1 values[('1'), ('2')), + partition p2 values[('2'), ('3')), + partition p3 values[('3'), ('4')), + partition p4 values[('4'), ('5')), + partition p5 values[('5'), ('6')) + ) + distributed by hash(id) + properties( + 'replication_num'='1' + ) + """ + + sql """ + insert into ${tableName} + values (1, 1), (1, 2), + (2, 1), (2, 2), + (3, 1), (3, 2), + (4, 1), (4, 2), + (5, 1), (5, 2) + """ +} + + +logger.info("Added 'createTestTable' function to Suite") diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy new file mode 100644 index 00000000000000..7523df43f0cbb8 --- /dev/null +++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy @@ -0,0 +1,550 @@ +// 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("parse_sql_from_sql_cache") { + def assertHasCache = { String sqlStr -> + explain { + sql ("physical plan ${sqlStr}") + contains("PhysicalSqlCache") + } + } + + def assertNoCache = { String sqlStr -> + explain { + sql ("physical plan ${sqlStr}") + notContains("PhysicalSqlCache") + } + } + + + sql "ADMIN SET FRONTEND CONFIG ('cache_last_version_interval_second' = '10')" + + combineFutures( + extraThread("testUsePlanCache", { + createTestTable "test_use_plan_cache" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache" + + // create sql cache + sql "select * from test_use_plan_cache" + + // use sql cache + assertHasCache "select * from test_use_plan_cache" + }), + extraThread("testAddPartitionAndInsert", { + createTestTable "test_use_plan_cache2" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache2" + sql "select * from test_use_plan_cache2" + assertHasCache "select * from test_use_plan_cache2" + + // add empty partition can use cache + sql "alter table test_use_plan_cache2 add partition p6 values[('6'),('7'))" + assertHasCache "select * from test_use_plan_cache2" + + // insert data can not use cache + sql "insert into test_use_plan_cache2 values(6, 1)" + assertNoCache "select * from test_use_plan_cache2" + }), + extraThread("testDropPartition", { + createTestTable "test_use_plan_cache3" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache3" + sql "select * from test_use_plan_cache3" + assertHasCache "select * from test_use_plan_cache3" + + // drop partition can not use cache + sql "alter table test_use_plan_cache3 drop partition p5" + assertNoCache "select * from test_use_plan_cache3" + }), + extraThread("testReplacePartition", { + createTestTable "test_use_plan_cache4" + + sql "alter table test_use_plan_cache4 add temporary partition tp1 values [('1'), ('2'))" + + streamLoad { + table "test_use_plan_cache4" + set "temporaryPartitions", "tp1" + inputIterator([[1, 3], [1, 4]].iterator()) + } + sql "sync" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache4" + sql "select * from test_use_plan_cache4" + assertHasCache "select * from test_use_plan_cache4" + + // replace partition can not use cache + sql "alter table test_use_plan_cache4 replace partition (p1) with temporary partition(tp1)" + assertNoCache "select * from test_use_plan_cache4" + }), + extraThread("testStreamLoad", { + createTestTable "test_use_plan_cache5" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache5" + sql "select * from test_use_plan_cache5" + assertHasCache "select * from test_use_plan_cache5" + + streamLoad { + table "test_use_plan_cache5" + set "partitions", "p1" + inputIterator([[1, 3], [1, 4]].iterator()) + } + + // stream load can not use cache + sql "select * from test_use_plan_cache5" + assertNoCache "select * from test_use_plan_cache5" + }), + extraThread("testUpdate",{ + createTestTable "test_use_plan_cache6", uniqueTable=true + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache6" + sql "select * from test_use_plan_cache6" + assertHasCache "select * from test_use_plan_cache6" + + sql "update test_use_plan_cache6 set value=3 where id=1" + + // update can not use cache + sql "select * from test_use_plan_cache6" + assertNoCache "select * from test_use_plan_cache6" + }), + extraThread("testDelete", { + createTestTable "test_use_plan_cache7" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache7" + sql "select * from test_use_plan_cache7" + assertHasCache "select * from test_use_plan_cache7" + + sql "delete from test_use_plan_cache7 where id = 1" + + // delete can not use cache + sql "select * from test_use_plan_cache7" + assertNoCache "select * from test_use_plan_cache7" + }), + extraThread("testDropTable", { + createTestTable "test_use_plan_cache8" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache8" + sql "select * from test_use_plan_cache8" + assertHasCache "select * from test_use_plan_cache8" + + sql "drop table test_use_plan_cache8" + + // should visible the table has bean deleted + test { + sql "select * from test_use_plan_cache8" + exception "does not exist in database" + } + }), + extraThread("testCreateAndAlterView", { + createTestTable "test_use_plan_cache9" + + sql "drop view if exists test_use_plan_cache9_view" + sql "create view test_use_plan_cache9_view as select * from test_use_plan_cache9" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache9_view" + sql "select * from test_use_plan_cache9_view" + assertHasCache "select * from test_use_plan_cache9_view" + + // alter view should not use cache + sql "alter view test_use_plan_cache9_view as select id from test_use_plan_cache9" + assertNoCache "select * from test_use_plan_cache9_view" + }), + extraThread( "testDropView", { + createTestTable "test_use_plan_cache10" + + sql "drop view if exists test_use_plan_cache10_view" + sql "create view test_use_plan_cache10_view as select * from test_use_plan_cache10" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache10_view" + sql "select * from test_use_plan_cache10_view" + assertHasCache "select * from test_use_plan_cache10_view" + + sql "drop view test_use_plan_cache10_view" + // should visible the view has bean deleted + test { + sql "select * from test_use_plan_cache10_view" + exception "does not exist in database" + } + }), + extraThread("testBaseTableChanged", { + createTestTable "test_use_plan_cache11" + + sql "drop view if exists test_use_plan_cache11_view" + sql "create view test_use_plan_cache11_view as select * from test_use_plan_cache11" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache11_view" + sql "select * from test_use_plan_cache11_view" + assertHasCache "select * from test_use_plan_cache11_view" + + sql "insert into test_use_plan_cache11 values(1, 3)" + + // base table already changed, can not use cache + assertNoCache "select * from test_use_plan_cache11_view" + }), + extraThread("testNotShareCacheBetweenUsers", { + sql "drop user if exists test_cache_user1" + sql "create user test_cache_user1 identified by 'DORIS@2024'" + def dbName = context.config.getDbNameByFile(context.file) + sql """GRANT SELECT_PRIV ON *.* TO test_cache_user1""" + + createTestTable "test_use_plan_cache12" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache12" + sql "select * from test_use_plan_cache12" + assertHasCache "select * from test_use_plan_cache12" + + + extraThread("test_cache_user1_thread", { + connect(user = "test_cache_user1", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache12" + } + }).get() + }), + extraThread("testAddRowPolicy", { + def dbName = context.config.getDbNameByFile(context.file) + sql "set enable_nereids_planner=false" + try_sql """ + DROP ROW POLICY if exists test_cache_row_policy_2 + ON ${dbName}.test_use_plan_cache13 + FOR test_cache_user2""" + sql "set enable_nereids_planner=true" + + sql "drop user if exists test_cache_user2" + sql "create user test_cache_user2 identified by 'DORIS@2024'" + sql """GRANT SELECT_PRIV ON *.* TO test_cache_user2""" + + createTestTable "test_use_plan_cache13" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + extraThread("test_cache_user2_thread", { + connect(user = "test_cache_user2", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache13" + sql "select * from test_use_plan_cache13" + assertHasCache "select * from test_use_plan_cache13" + } + }).get() + + sql "set enable_nereids_planner=false" + sql """ + CREATE ROW POLICY test_cache_row_policy_2 + ON ${dbName}.test_use_plan_cache13 + AS RESTRICTIVE TO test_cache_user2 + USING (id = 'concat(id, "**")')""" + sql "set enable_nereids_planner=true" + + // after row policy changed, the cache is invalidate + extraThread("test_cache_user2_thread2", { + connect(user = "test_cache_user2", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache13" + } + }).get() + }), + extraThread("testDropRowPolicy", { + def dbName = context.config.getDbNameByFile(context.file) + sql "set enable_nereids_planner=false" + try_sql """ + DROP ROW POLICY if exists test_cache_row_policy_3 + ON ${dbName}.test_use_plan_cache14 + FOR test_cache_user3""" + sql "set enable_nereids_planner=true" + + sql "drop user if exists test_cache_user3" + sql "create user test_cache_user3 identified by 'DORIS@2024'" + sql """GRANT SELECT_PRIV ON *.* TO test_cache_user3""" + + createTestTable "test_use_plan_cache14" + + sql "set enable_nereids_planner=false" + sql """ + CREATE ROW POLICY test_cache_row_policy_3 + ON ${dbName}.test_use_plan_cache14 + AS RESTRICTIVE TO test_cache_user3 + USING (id = 'concat(id, "**")')""" + sql "set enable_nereids_planner=true" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + extraThread("test_cache_user3_thread", { + connect(user = "test_cache_user3", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache14" + sql "select * from test_use_plan_cache14" + assertHasCache "select * from test_use_plan_cache14" + } + }).get() + + sql "set enable_nereids_planner=false" + try_sql """ + DROP ROW POLICY if exists test_cache_row_policy_3 + ON ${dbName}.test_use_plan_cache14 + FOR test_cache_user3""" + sql "set enable_nereids_planner=true" + + // after row policy changed, the cache is invalidate + extraThread("test_cache_user3_thread2", { + connect(user = "test_cache_user3", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache14" + } + }).get() + }), + extraThread("testRemovePrivilege", { + def dbName = context.config.getDbNameByFile(context.file) + + createTestTable "test_use_plan_cache15" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "drop user if exists test_cache_user4" + sql "create user test_cache_user4 identified by 'DORIS@2024'" + sql "GRANT SELECT_PRIV ON regression_test.* TO test_cache_user4" + sql "GRANT SELECT_PRIV ON ${dbName}.test_use_plan_cache15 TO test_cache_user4" + + extraThread("test_cache_user4_thread", { + connect(user = "test_cache_user4", password="DORIS@2024") { + sql "use ${dbName}" + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select * from test_use_plan_cache15" + sql "select * from test_use_plan_cache15" + assertHasCache "select * from test_use_plan_cache15" + } + }).get() + + sql "REVOKE SELECT_PRIV ON ${dbName}.test_use_plan_cache15 FROM test_cache_user4" + + // after privileges changed, the cache is invalidate + extraThread("test_cache_user4_thread2", { + connect(user = "test_cache_user4", password="DORIS@2024") { + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + test { + sql ("select * from ${dbName}.test_use_plan_cache15") + exception "Permission denied" + } + } + }).get() + }), + extraThread("testNondeterministic", { + createTestTable "test_use_plan_cache16" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + assertNoCache "select random() from test_use_plan_cache16" + // create sql cache + sql "select random() from test_use_plan_cache16" + // can not use sql cache + assertNoCache "select random() from test_use_plan_cache16" + + + assertNoCache "select year(now()) from test_use_plan_cache16" + sql "select year(now()) from test_use_plan_cache16" + assertHasCache "select year(now()) from test_use_plan_cache16" + + + assertNoCache "select second(now()) from test_use_plan_cache16" + sql "select second(now()) from test_use_plan_cache16" + sleep(1000) + assertNoCache "select second(now()) from test_use_plan_cache16" + }), + extraThread("testUserVariable", { + // make sure if the table has been dropped, the cache should invalidate, + // so we should retry twice to check + for (i in 0..2) { + createTestTable "test_use_plan_cache17" + + // after partition changed 10s, the sql cache can be used + sleep(10000) + + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql "set enable_sql_cache=true" + + sql "set @custom_variable=10" + assertNoCache "select @custom_variable from test_use_plan_cache17" + // create sql cache + sql "select @custom_variable from test_use_plan_cache17" + // can use sql cache + assertHasCache "select @custom_variable from test_use_plan_cache17" + + sql "set @custom_variable=20" + assertNoCache "select @custom_variable from test_use_plan_cache17" + } + }), + extraThread("test_udf", { + def jarPath = """${context.config.suitePath}/javaudf_p0/jars/java-udf-case-jar-with-dependencies.jar""" + try_sql("DROP FUNCTION IF EXISTS java_udf_string_test(string, int, int);") + try_sql("DROP TABLE IF EXISTS test_javaudf_string") + + sql """ DROP TABLE IF EXISTS test_javaudf_string """ + sql """ + CREATE TABLE IF NOT EXISTS test_javaudf_string ( + `user_id` INT NOT NULL COMMENT "用户id", + `char_col` CHAR NOT NULL COMMENT "", + `varchar_col` VARCHAR(10) NOT NULL COMMENT "", + `string_col` STRING NOT NULL COMMENT "" + ) + DISTRIBUTED BY HASH(user_id) PROPERTIES("replication_num" = "1"); + """ + + StringBuilder values = new StringBuilder() + int i = 1 + for (; i < 9; i ++) { + values.append(" (${i}, '${i}','abcdefg${i}','poiuytre${i}abcdefg'),\n") + } + values.append("(${i}, '${i}','abcdefg${i}','poiuytre${i}abcdefg')") + + sql "INSERT INTO test_javaudf_string VALUES ${values}" + sql "sync" + + File path = new File(jarPath) + if (!path.exists()) { + throw new IllegalStateException("""${jarPath} doesn't exist! """) + } + + sql """ CREATE FUNCTION java_udf_string_test(string, int, int) RETURNS string PROPERTIES ( + "file"="file://${jarPath}", + "symbol"="org.apache.doris.udf.StringTest", + "type"="JAVA_UDF" + ); """ + + assertNoCache "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" + sql "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" + assertNoCache "SELECT java_udf_string_test(varchar_col, 2, 3) result FROM test_javaudf_string ORDER BY result;" + }) + ).get() +} diff --git a/regression-test/suites/query_p0/cache/sql_cache.groovy b/regression-test/suites/query_p0/cache/sql_cache.groovy index cccfca4d9650b1..d4b16cb5438fd2 100644 --- a/regression-test/suites/query_p0/cache/sql_cache.groovy +++ b/regression-test/suites/query_p0/cache/sql_cache.groovy @@ -211,5 +211,5 @@ suite("sql_cache") { k1; """ - sql "ADMIN SET FRONTEND CONFIG ('cache_last_version_interval_second' = '900')" + sql "ADMIN SET FRONTEND CONFIG ('cache_last_version_interval_second' = '10')" } diff --git a/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy b/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy index 19e78ac05ba8ec..007bb196244eff 100644 --- a/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy +++ b/regression-test/suites/schema_change_p0/test_schema_change_duplicate.groovy @@ -53,7 +53,6 @@ suite("test_schema_change_duplicate", "p0") { set 'column_separator', ',' file 'all_types.csv' - time 10000 // limit inflight 10s check { result, exception, startTime, endTime -> if (exception != null) { From d4d5ad032436be2a8fda6ebd494ea547ed04366c Mon Sep 17 00:00:00 2001 From: Xin Liao Date: Fri, 12 Apr 2024 20:03:18 +0800 Subject: [PATCH 37/71] [enhancement](merge-iterator) catch exception to avoid coredump when copy_rows (#33567) --- be/src/vec/olap/vgeneric_iterators.cpp | 34 +++++++++++++++----------- be/src/vec/olap/vgeneric_iterators.h | 10 ++++---- 2 files changed, 25 insertions(+), 19 deletions(-) diff --git a/be/src/vec/olap/vgeneric_iterators.cpp b/be/src/vec/olap/vgeneric_iterators.cpp index c973a7131f9ff7..26fe590dd6038f 100644 --- a/be/src/vec/olap/vgeneric_iterators.cpp +++ b/be/src/vec/olap/vgeneric_iterators.cpp @@ -136,42 +136,48 @@ bool VMergeIteratorContext::compare(const VMergeIteratorContext& rhs) const { } // `advanced = false` when current block finished -void VMergeIteratorContext::copy_rows(Block* block, bool advanced) { +Status VMergeIteratorContext::copy_rows(Block* block, bool advanced) { Block& src = *_block; Block& dst = *block; if (_cur_batch_num == 0) { - return; + return Status::OK(); } // copy a row to dst block column by column size_t start = _index_in_block - _cur_batch_num + 1 - advanced; - for (size_t i = 0; i < _num_columns; ++i) { - auto& s_col = src.get_by_position(i); - auto& d_col = dst.get_by_position(i); + RETURN_IF_CATCH_EXCEPTION({ + for (size_t i = 0; i < _num_columns; ++i) { + auto& s_col = src.get_by_position(i); + auto& d_col = dst.get_by_position(i); - ColumnPtr& s_cp = s_col.column; - ColumnPtr& d_cp = d_col.column; + ColumnPtr& s_cp = s_col.column; + ColumnPtr& d_cp = d_col.column; - d_cp->assume_mutable()->insert_range_from(*s_cp, start, _cur_batch_num); - } + d_cp->assume_mutable()->insert_range_from(*s_cp, start, _cur_batch_num); + } + }); const auto& tmp_pre_ctx_same_bit = get_pre_ctx_same(); dst.set_same_bit(tmp_pre_ctx_same_bit.begin(), tmp_pre_ctx_same_bit.begin() + _cur_batch_num); _cur_batch_num = 0; + return Status::OK(); } -void VMergeIteratorContext::copy_rows(BlockView* view, bool advanced) { +Status VMergeIteratorContext::copy_rows(BlockView* view, bool advanced) { if (_cur_batch_num == 0) { - return; + return Status::OK(); } size_t start = _index_in_block - _cur_batch_num + 1 - advanced; const auto& tmp_pre_ctx_same_bit = get_pre_ctx_same(); - for (size_t i = 0; i < _cur_batch_num; ++i) { - view->push_back({_block, static_cast(start + i), tmp_pre_ctx_same_bit[i]}); - } + RETURN_IF_CATCH_EXCEPTION({ + for (size_t i = 0; i < _cur_batch_num; ++i) { + view->push_back({_block, static_cast(start + i), tmp_pre_ctx_same_bit[i]}); + } + }); _cur_batch_num = 0; + return Status::OK(); } // This iterator will generate ordered data. For example for schema diff --git a/be/src/vec/olap/vgeneric_iterators.h b/be/src/vec/olap/vgeneric_iterators.h index f2e5e04bcc503d..d67bb68fefa542 100644 --- a/be/src/vec/olap/vgeneric_iterators.h +++ b/be/src/vec/olap/vgeneric_iterators.h @@ -108,9 +108,9 @@ class VMergeIteratorContext { bool compare(const VMergeIteratorContext& rhs) const; // `advanced = false` when current block finished - void copy_rows(Block* block, bool advanced = true); + Status copy_rows(Block* block, bool advanced = true); - void copy_rows(BlockView* view, bool advanced = true); + Status copy_rows(BlockView* view, bool advanced = true); RowLocation current_row_location() { DCHECK(_record_rowids); @@ -245,7 +245,7 @@ class VMergeIterator : public RowwiseIterator { ctx->add_cur_batch(); if (pre_ctx != ctx) { if (pre_ctx) { - pre_ctx->copy_rows(block); + RETURN_IF_ERROR(pre_ctx->copy_rows(block)); } pre_ctx = ctx; } @@ -257,14 +257,14 @@ class VMergeIterator : public RowwiseIterator { if (ctx->is_cur_block_finished() || row_idx >= _block_row_max) { // current block finished, ctx not advance // so copy start_idx = (_index_in_block - _cur_batch_num + 1) - ctx->copy_rows(block, false); + RETURN_IF_ERROR(ctx->copy_rows(block, false)); pre_ctx = nullptr; } } else if (_merged_rows != nullptr) { (*_merged_rows)++; // need skip cur row, so flush rows in pre_ctx if (pre_ctx) { - pre_ctx->copy_rows(block); + RETURN_IF_ERROR(pre_ctx->copy_rows(block)); pre_ctx = nullptr; } } From 09a318c55de30d4aedfacc405fb83cb3976e0d45 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Fri, 12 Apr 2024 22:17:27 +0800 Subject: [PATCH 38/71] [opt](Nereids) support set operation minus (#33582) --- .../src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 | 2 +- .../org/apache/doris/nereids/parser/LogicalPlanBuilder.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 754c68ee4aee47..74870648786d1a 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 @@ -301,7 +301,7 @@ query queryTerm : queryPrimary #queryTermDefault - | left=queryTerm operator=(UNION | EXCEPT | INTERSECT) + | left=queryTerm operator=(UNION | EXCEPT | MINUS | INTERSECT) setQuantifier? right=queryTerm #setOperation ; 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 8dc75102923957..5e3a5b70db80f8 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 @@ -1198,7 +1198,7 @@ && getQualifier((SetOperationContext) current) == qualifier LogicalPlan plan; if (ctx.UNION() != null) { plan = new LogicalUnion(qualifier, newChildren); - } else if (ctx.EXCEPT() != null) { + } else if (ctx.EXCEPT() != null || ctx.MINUS() != null) { plan = new LogicalExcept(qualifier, newChildren); } else if (ctx.INTERSECT() != null) { plan = new LogicalIntersect(qualifier, newChildren); From 1d59436801483e06fdd537064aeaedc29a423549 Mon Sep 17 00:00:00 2001 From: meiyi Date: Fri, 12 Apr 2024 23:16:33 +0800 Subject: [PATCH 39/71] [fix](feservice) remove connect context (#33583) --- fe/fe-core/src/main/java/org/apache/doris/service/FeServer.java | 2 ++ .../main/java/org/apache/doris/service/FrontendServiceImpl.java | 2 -- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FeServer.java b/fe/fe-core/src/main/java/org/apache/doris/service/FeServer.java index 32d055069a24d2..6c61c9cdd488ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FeServer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FeServer.java @@ -19,6 +19,7 @@ import org.apache.doris.common.ThriftServer; import org.apache.doris.metric.MetricRepo; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.FrontendService; import org.apache.logging.log4j.LogManager; @@ -62,6 +63,7 @@ public void start() throws IOException { // If exception occurs, do not deal it, just keep as the previous throw t; } finally { + ConnectContext.remove(); feServiceLogger.debug("finish process request for {}", name); if (MetricRepo.isInit) { long end = System.currentTimeMillis(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 683ad847872727..fb659f8d7125e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -1911,8 +1911,6 @@ public TStreamLoadPutResult streamLoadPut(TStreamLoadPutRequest request) { StreamLoadHandler streamLoadHandler = new StreamLoadHandler(request, null, result, clientAddr); try { - // Some requests forget to remove the ConnectContext, so remove it firstly - ConnectContext.remove(); streamLoadHandler.setCloudCluster(); List tWorkloadGroupList = null; From 182202ed29b9ef2a4064d6cb3dc69d0bf6372196 Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Fri, 12 Apr 2024 23:18:33 +0800 Subject: [PATCH 40/71] [testcases](auto-partition) Add and fix testcases in P0 #33588 --- .../test_auto_range_partition.out | 3 ++ .../test_auto_range_partition.groovy | 29 ++++++++++++++++++- 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out b/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out index 93c6d86bf9c734..dadc18076a4eb2 100644 --- a/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out +++ b/regression-test/data/partition_p0/auto_partition/test_auto_range_partition.out @@ -108,3 +108,6 @@ 9999-12-31T23:59:59 9999-12-31T23:59:59.999999 +-- !sql -- +2020-12-12 + diff --git a/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy b/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy index e0f8db3ac8f7e7..e17f4f8ef8a5fc 100644 --- a/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy +++ b/regression-test/suites/partition_p0/auto_partition/test_auto_range_partition.groovy @@ -117,7 +117,7 @@ suite("test_auto_range_partition") { ); """ sql " insert into week_range values (20240408), (20240409); " - result2 = sql "show partitions from week_range" + def result2 = sql "show partitions from week_range" logger.info("${result2}") assertEquals(result2.size(), 1) @@ -139,4 +139,31 @@ suite("test_auto_range_partition") { result2 = sql "show partitions from quarter_range" logger.info("${result2}") assertEquals(result2.size(), 2) + + // partition expr extraction + + sql " drop table if exists isit " + sql " drop table if exists isit_src " + sql """ + CREATE TABLE isit ( + k DATE NOT NULL + ) + AUTO PARTITION BY RANGE (date_trunc(k, 'day'))() + DISTRIBUTED BY HASH(k) BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ + CREATE TABLE isit_src ( + k DATE NOT NULL + ) + DISTRIBUTED BY HASH(k) BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql " insert into isit_src values (20201212); " + sql " insert into isit select * from isit_src " + qt_sql " select * from isit order by k " } From 9d11a81b3f0f7716a7ffaa0ebb5ba886d7f4effc Mon Sep 17 00:00:00 2001 From: zclllyybb Date: Fri, 12 Apr 2024 23:22:44 +0800 Subject: [PATCH 41/71] [Fix](timezone) fix miss of expected rounding of Date type with timezone #33553 --- be/src/http/action/stream_load.cpp | 3 +++ be/src/http/http_common.h | 1 + be/src/vec/runtime/vdatetime_value.cpp | 20 ++++++++++++++++ be/src/vec/runtime/vdatetime_value.h | 13 ++++++++++ .../data/datatype_p0/datetimev2/only_date.csv | 2 ++ .../datetimev2/test_tz_streamload.out | 4 ++++ .../datetimev2/test_tz_streamload.groovy | 24 +++++++++++++++++++ 7 files changed, 67 insertions(+) create mode 100644 regression-test/data/datatype_p0/datetimev2/only_date.csv diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp index 62c1f3ddf0ecaf..e5b426ade59fbf 100644 --- a/be/src/http/action/stream_load.cpp +++ b/be/src/http/action/stream_load.cpp @@ -475,8 +475,11 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, return Status::InvalidArgument("Invalid strict mode format. Must be bool type"); } } + // timezone first. if not, try time_zone if (!http_req->header(HTTP_TIMEZONE).empty()) { request.__set_timezone(http_req->header(HTTP_TIMEZONE)); + } else if (!http_req->header(HTTP_TIME_ZONE).empty()) { + request.__set_timezone(http_req->header(HTTP_TIME_ZONE)); } if (!http_req->header(HTTP_EXEC_MEM_LIMIT).empty()) { try { diff --git a/be/src/http/http_common.h b/be/src/http/http_common.h index 57bbd0642e6e96..517ee87c68511c 100644 --- a/be/src/http/http_common.h +++ b/be/src/http/http_common.h @@ -38,6 +38,7 @@ static const std::string HTTP_TEMP_PARTITIONS = "temporary_partitions"; static const std::string HTTP_NEGATIVE = "negative"; static const std::string HTTP_STRICT_MODE = "strict_mode"; static const std::string HTTP_TIMEZONE = "timezone"; +static const std::string HTTP_TIME_ZONE = "time_zone"; static const std::string HTTP_EXEC_MEM_LIMIT = "exec_mem_limit"; static const std::string HTTP_JSONPATHS = "jsonpaths"; static const std::string HTTP_JSONROOT = "json_root"; diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp index ee384286455b40..6b769c68f9dadf 100644 --- a/be/src/vec/runtime/vdatetime_value.cpp +++ b/be/src/vec/runtime/vdatetime_value.cpp @@ -2192,6 +2192,26 @@ bool DateV2Value::from_date_str_base(const char* date_str, int len, int scale return false; } + // In check_range_and_set_time, for Date type the time part will be truncated. So if the timezone offset should make + // rounding to date part, it would be lost. To avoid this, we use a Datetime type to do these calc. It will save the + // time part and apply the offset. Then convert to Date type back. + // see https://github.com/apache/doris/pull/33553 for more details. + if constexpr (!is_datetime) { + if (sec_offset) { + DateV2Value tmp; + if (!tmp.check_range_and_set_time(date_val[0], date_val[1], date_val[2], date_val[3], + date_val[4], date_val[5], date_val[6])) { + return false; + } + if (!tmp.date_add_interval( + TimeInterval {TimeUnit::SECOND, sec_offset, false})) { + return false; + } + this->assign_from(tmp); + return true; + } + } + if (!check_range_and_set_time(date_val[0], date_val[1], date_val[2], date_val[3], date_val[4], date_val[5], date_val[6])) { return false; diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h index 54dc368f620f22..2031c782267175 100644 --- a/be/src/vec/runtime/vdatetime_value.h +++ b/be/src/vec/runtime/vdatetime_value.h @@ -824,6 +824,19 @@ class DateV2Value { return from_date_format_str(format, format_len, value, value_len, nullptr); } + template + void assign_from(DateV2Value src) { + date_v2_value_.year_ = src.year(); + date_v2_value_.month_ = src.month(); + date_v2_value_.day_ = src.day(); + if constexpr (is_datetime && std::is_same_v) { + date_v2_value_.hour_ = src.hour(); + date_v2_value_.minute_ = src.minute(); + date_v2_value_.second_ = src.second(); + date_v2_value_.microsecond_ = src.microsecond(); + } + } + // Construct Date/Datetime type value from string. // At least the following formats are recognised (based on number of digits) // 'YYMMDD', 'YYYYMMDD', 'YYMMDDHHMMSS', 'YYYYMMDDHHMMSS' diff --git a/regression-test/data/datatype_p0/datetimev2/only_date.csv b/regression-test/data/datatype_p0/datetimev2/only_date.csv new file mode 100644 index 00000000000000..63b49fef9d9a54 --- /dev/null +++ b/regression-test/data/datatype_p0/datetimev2/only_date.csv @@ -0,0 +1,2 @@ +1,2024-04-11T16:00:13+08:00,2024-04-11T16:00:13+08:00 +1,2024-04-11T06:00:13+08:00,2024-04-11T06:00:13+08:00 diff --git a/regression-test/data/datatype_p0/datetimev2/test_tz_streamload.out b/regression-test/data/datatype_p0/datetimev2/test_tz_streamload.out index c7fd50e3bb4d40..ab103c3a306f46 100644 --- a/regression-test/data/datatype_p0/datetimev2/test_tz_streamload.out +++ b/regression-test/data/datatype_p0/datetimev2/test_tz_streamload.out @@ -19,3 +19,7 @@ 7 2023-08-17T17:41:18 8 2023-08-17T19:41:18 +-- !table1 -- +1 2024-04-10 2024-04-10T22:00:13 +1 2024-04-11 2024-04-11T08:00:13 + diff --git a/regression-test/suites/datatype_p0/datetimev2/test_tz_streamload.groovy b/regression-test/suites/datatype_p0/datetimev2/test_tz_streamload.groovy index 6e32facc83e8e9..9ccd48477e1c87 100644 --- a/regression-test/suites/datatype_p0/datetimev2/test_tz_streamload.groovy +++ b/regression-test/suites/datatype_p0/datetimev2/test_tz_streamload.groovy @@ -68,4 +68,28 @@ suite("test_tz_streamload") { } sql "sync" qt_table2 "select * from ${table2} order by id" + + // test rounding for date type. from hour to date. + sql "drop table if exists d" + sql """ + CREATE TABLE d ( + `k1` int, + `k2` date, + `k3` datetime + ) + DISTRIBUTED BY HASH(k1) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + streamLoad { + table "d" + set 'column_separator', ',' + set 'timezone', 'UTC' + file "only_date.csv" + time 20000 + } + sql "sync" + qt_table1 "select * from d order by k1, k2, k3" } From 41bde7ce81837054c64853145d5889992fe39fac Mon Sep 17 00:00:00 2001 From: plat1ko Date: Sat, 13 Apr 2024 10:36:35 +0800 Subject: [PATCH 42/71] [fix](hdfs accessor) Fix hdfs accessor build error (#33605) --- be/src/cloud/cloud_storage_engine.cpp | 6 +++++- cloud/src/recycler/hdfs_accessor.cpp | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/be/src/cloud/cloud_storage_engine.cpp b/be/src/cloud/cloud_storage_engine.cpp index d3f4c05180c140..c95998ecba1446 100644 --- a/be/src/cloud/cloud_storage_engine.cpp +++ b/be/src/cloud/cloud_storage_engine.cpp @@ -275,7 +275,11 @@ void CloudStorageEngine::sync_storage_vault() { return; } - CHECK(!vault_infos.empty()) << "no s3 infos"; + if (vault_infos.empty()) { + LOG(WARNING) << "no storage vault info"; + return; + } + for (auto& [id, vault_info] : vault_infos) { auto fs = get_filesystem(id); auto st = (fs == nullptr) diff --git a/cloud/src/recycler/hdfs_accessor.cpp b/cloud/src/recycler/hdfs_accessor.cpp index 96461d7bb6820e..ece196baa4f31e 100644 --- a/cloud/src/recycler/hdfs_accessor.cpp +++ b/cloud/src/recycler/hdfs_accessor.cpp @@ -91,7 +91,7 @@ class HDFSBuilder { if (conf.has_hdfs_kerberos_principal()) { kerberos_login = true; - hdfsBuilderSetPrincipal(hdfs_builder_, conf.hdfs_kerberos_keytab().c_str()); + hdfsBuilderSetPrincipal(hdfs_builder_, conf.hdfs_kerberos_principal().c_str()); } else if (conf.has_user()) { hdfsBuilderSetUserName(hdfs_builder_, conf.user().c_str()); #ifdef USE_HADOOP_HDFS From 8a94b011349d209a8ca810108b11cdfcab8feb7d Mon Sep 17 00:00:00 2001 From: zhannngchen <48427519+zhannngchen@users.noreply.github.com> Date: Sat, 13 Apr 2024 10:41:30 +0800 Subject: [PATCH 43/71] [fix](merge-on-write) schema change may cause mow duplicate key (#33536) --- be/src/olap/schema_change.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index fb9137d4c84eac..13182467591a5e 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -969,6 +969,8 @@ Status SchemaChangeJob::_do_process_alter_tablet(const TAlterTabletReqV2& reques break; } + DCHECK_GE(real_alter_version, request.alter_version); + if (_new_tablet->keys_type() == UNIQUE_KEYS && _new_tablet->enable_unique_key_merge_on_write()) { res = _calc_delete_bitmap_for_mow_table(real_alter_version); @@ -1086,6 +1088,7 @@ Status SchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParams& sc auto sc_procedure = _get_sc_procedure(changer, sc_sorting, sc_directly); // c.Convert historical data + bool have_failure_rowset = false; for (const auto& rs_reader : sc_params.ref_rowset_readers) { // set status for monitor // As long as there is a new_table as running, ref table is set as running @@ -1131,6 +1134,7 @@ Status SchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParams& sc << "tablet=" << _new_tablet->tablet_id() << ", version='" << rs_reader->version().first << "-" << rs_reader->version().second; _local_storage_engine.add_unused_rowset(new_rowset); + have_failure_rowset = true; res = Status::OK(); } else if (!res) { LOG(WARNING) << "failed to register new version. " @@ -1144,7 +1148,9 @@ Status SchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParams& sc << ", version=" << rs_reader->version().first << "-" << rs_reader->version().second; } - *real_alter_version = rs_reader->version().second; + if (!have_failure_rowset) { + *real_alter_version = rs_reader->version().second; + } VLOG_TRACE << "succeed to convert a history version." << " version=" << rs_reader->version().first << "-" From 11d2f14b29a20de89295f170ad33bfda01256c60 Mon Sep 17 00:00:00 2001 From: AlexYue Date: Sat, 13 Apr 2024 10:50:17 +0800 Subject: [PATCH 44/71] [feature](Cloud) Only cloud mode and instance with enable_storage_vault can use storage vault stmt (#33561) --- .../analysis/CreateStorageVaultStmt.java | 12 +++++++++++ .../analysis/SetDefaultStorageVaultStmt.java | 21 +++++++++++++++++++ .../analysis/CreateStorageVaultStmtTest.java | 7 +++++++ 3 files changed, 40 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateStorageVaultStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateStorageVaultStmt.java index a606ed483366a8..652c15d4b8ea05 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateStorageVaultStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateStorageVaultStmt.java @@ -19,9 +19,12 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.StorageVault; +import org.apache.doris.cloud.catalog.CloudEnv; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.FeNameFormat; import org.apache.doris.common.UserException; import org.apache.doris.common.util.PrintableMap; @@ -72,6 +75,15 @@ public void setStorageVaultType(StorageVault.StorageVaultType type) throws UserE @Override public void analyze(Analyzer analyzer) throws UserException { + if (Config.isNotCloudMode()) { + throw new AnalysisException("Storage Vault is only supported for cloud mode"); + } + if (!FeConstants.runningUnitTest) { + // In legacy cloud mode, some s3 back-ended storage does need to use storage vault. + if (!((CloudEnv) Env.getCurrentEnv()).getEnableStorageVault()) { + throw new AnalysisException("Your cloud instance doesn't support storage vault"); + } + } super.analyze(analyzer); // check auth diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetDefaultStorageVaultStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetDefaultStorageVaultStmt.java index eec693bb40db9b..b9facec56a52de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetDefaultStorageVaultStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetDefaultStorageVaultStmt.java @@ -17,6 +17,13 @@ package org.apache.doris.analysis; +import org.apache.doris.catalog.Env; +import org.apache.doris.cloud.catalog.CloudEnv; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.FeConstants; +import org.apache.doris.common.UserException; + // SET vault_name DEFAULT STORAGE VAULT public class SetDefaultStorageVaultStmt extends DdlStmt { public final String vaultName; @@ -29,6 +36,20 @@ public String getStorageVaultName() { return vaultName; } + @Override + public void analyze(Analyzer analyzer) throws UserException { + if (Config.isNotCloudMode()) { + throw new AnalysisException("Storage Vault is only supported for cloud mode"); + } + if (!FeConstants.runningUnitTest) { + // In legacy cloud mode, some s3 back-ended storage does need to use storage vault. + if (!((CloudEnv) Env.getCurrentEnv()).getEnableStorageVault()) { + throw new AnalysisException("Your cloud instance doesn't support storage vault"); + } + } + super.analyze(analyzer); + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateStorageVaultStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateStorageVaultStmtTest.java index 7064ef7b66113a..0479c82d25d590 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateStorageVaultStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateStorageVaultStmtTest.java @@ -20,6 +20,8 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.StorageVault.StorageVaultType; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.UserException; import org.apache.doris.mysql.privilege.AccessControllerManager; import org.apache.doris.mysql.privilege.PrivPredicate; @@ -43,6 +45,7 @@ public class CreateStorageVaultStmtTest { public void setUp() { analyzer = AccessTestUtil.fetchAdminAnalyzer(true); vaultName = "hdfs"; + FeConstants.runningUnitTest = true; } @Test @@ -57,6 +60,7 @@ public void testNormal(@Mocked Env env, @Injectable AccessControllerManager acce } }; + Config.cloud_unique_id = "not_empty"; Map properties = Maps.newHashMap(); properties.put("type", "hdfs"); CreateStorageVaultStmt stmt = new CreateStorageVaultStmt(true, vaultName, properties); @@ -64,6 +68,7 @@ public void testNormal(@Mocked Env env, @Injectable AccessControllerManager acce Assert.assertEquals(vaultName, stmt.getStorageVaultName()); Assert.assertEquals(StorageVaultType.HDFS, stmt.getStorageVaultType()); Assert.assertEquals("CREATE STORAGE VAULT 'hdfs' PROPERTIES(\"type\" = \"hdfs\")", stmt.toSql()); + Config.cloud_unique_id = ""; } @@ -79,9 +84,11 @@ public void testUnsupportedResourceType(@Mocked Env env, @Injectable AccessContr } }; + Config.cloud_unique_id = "not_empty"; Map properties = Maps.newHashMap(); properties.put("type", "hadoop"); CreateStorageVaultStmt stmt = new CreateStorageVaultStmt(true, vaultName, properties); stmt.analyze(analyzer); + Config.cloud_unique_id = ""; } } From b4fbf27903a16a19cd70d378f857735ce306560a Mon Sep 17 00:00:00 2001 From: Vallish Pai Date: Sat, 13 Apr 2024 10:50:23 +0530 Subject: [PATCH 45/71] [security] Dont print password in logs (#33606) --- cloud/src/recycler/recycler_service.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/cloud/src/recycler/recycler_service.cpp b/cloud/src/recycler/recycler_service.cpp index 0dc523312df228..5b0b3e8078ca82 100644 --- a/cloud/src/recycler/recycler_service.cpp +++ b/cloud/src/recycler/recycler_service.cpp @@ -384,7 +384,6 @@ void RecyclerServiceImpl::http(::google::protobuf::RpcController* controller, LOG(INFO) << " host " << *host; LOG(INFO) << " port " << *port; LOG(INFO) << " user " << *user; - LOG(INFO) << " passwd " << *password; LOG(INFO) << " instance " << *instance_id; if (instance_id == nullptr || instance_id->empty() || host == nullptr || host->empty() || port == nullptr || port->empty() || password == nullptr || user == nullptr || From 6ae033c47e819e41879b849bf312bfa9c478dc90 Mon Sep 17 00:00:00 2001 From: xueweizhang Date: Sat, 13 Apr 2024 13:25:00 +0800 Subject: [PATCH 46/71] =?UTF-8?q?[feature](backup)=20ignore=20table=20that?= =?UTF-8?q?=20not=20support=20type=20when=20backup,=20and=E2=80=A6=20(#331?= =?UTF-8?q?58)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * [feature](backup) ignore table that not support type when backup, and not report exception Signed-off-by: nextdreamblue * fix Signed-off-by: nextdreamblue --------- Signed-off-by: nextdreamblue --- .../main/java/org/apache/doris/common/Config.java | 6 ++++++ .../java/org/apache/doris/backup/BackupHandler.java | 13 ++++++++++++- 2 files changed, 18 insertions(+), 1 deletion(-) 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 f5d60377c62a18..d318b77aa416dd 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 @@ -1458,6 +1458,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = true) public static int max_backup_restore_job_num_per_db = 10; + /** + * whether to ignore table that not support type when backup, and not report exception. + */ + @ConfField(mutable = true, masterOnly = true) + public static boolean ignore_backup_not_support_table_type = false; + /** * Control the default max num of the instance for a user. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java index 03865433f6b319..fc2b0259f02b9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java @@ -384,6 +384,7 @@ private void backup(Repository repository, Database db, BackupStmt stmt) throws // Check if backup objects are valid // This is just a pre-check to avoid most of invalid backup requests. // Also calculate the signature for incremental backup check. + List tblRefsNotSupport = Lists.newArrayList(); for (TableRef tblRef : tblRefs) { String tblName = tblRef.getName().getTbl(); Table tbl = db.getTableOrDdlException(tblName); @@ -391,7 +392,15 @@ private void backup(Repository repository, Database db, BackupStmt stmt) throws continue; } if (tbl.getType() != TableType.OLAP) { - ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tblName); + if (Config.ignore_backup_not_support_table_type) { + LOG.warn("Table '{}' is a {} table, can not backup and ignore it." + + "Only OLAP(Doris)/ODBC/VIEW table can be backed up", + tblName, tbl.getType().toString()); + tblRefsNotSupport.add(tblRef); + continue; + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_NOT_OLAP_TABLE, tblName); + } } OlapTable olapTbl = (OlapTable) tbl; @@ -422,6 +431,8 @@ private void backup(Repository repository, Database db, BackupStmt stmt) throws } } + tblRefs.removeAll(tblRefsNotSupport); + // Check if label already be used long repoId = -1; if (repository != null) { From e0cea180737949153418b71762b5ee252c72a070 Mon Sep 17 00:00:00 2001 From: wangbo Date: Sat, 13 Apr 2024 14:24:49 +0800 Subject: [PATCH 47/71] [Fix](executor)reset remote scan thread num #33579 --- be/src/runtime/workload_group/workload_group.cpp | 15 +++++++++++++-- be/src/vec/exec/scan/scanner_scheduler.h | 7 ++++--- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/be/src/runtime/workload_group/workload_group.cpp b/be/src/runtime/workload_group/workload_group.cpp index 19ed39a6ab7864..1b15e89b08e554 100644 --- a/be/src/runtime/workload_group/workload_group.cpp +++ b/be/src/runtime/workload_group/workload_group.cpp @@ -374,7 +374,9 @@ void WorkloadGroup::upsert_task_scheduler(WorkloadGroupInfo* tg_info, ExecEnv* e std::unique_ptr scan_scheduler = std::make_unique("Scan_" + tg_name, cg_cpu_ctl_ptr); - Status ret = scan_scheduler->start(); + Status ret = scan_scheduler->start(config::doris_scanner_thread_pool_thread_num, + config::doris_scanner_thread_pool_thread_num, + config::doris_scanner_thread_pool_queue_size); if (ret.ok()) { _scan_task_sched = std::move(scan_scheduler); } else { @@ -386,10 +388,19 @@ void WorkloadGroup::upsert_task_scheduler(WorkloadGroupInfo* tg_info, ExecEnv* e } if (_remote_scan_task_sched == nullptr) { + int remote_max_thread_num = + config::doris_max_remote_scanner_thread_pool_thread_num != -1 + ? config::doris_max_remote_scanner_thread_pool_thread_num + : std::max(512, CpuInfo::num_cores() * 10); + remote_max_thread_num = + std::max(remote_max_thread_num, config::doris_scanner_thread_pool_thread_num); + std::unique_ptr remote_scan_scheduler = std::make_unique("RScan_" + tg_name, cg_cpu_ctl_ptr); - Status ret = remote_scan_scheduler->start(); + Status ret = + remote_scan_scheduler->start(remote_max_thread_num, remote_max_thread_num, + config::doris_remote_scanner_thread_pool_queue_size); if (ret.ok()) { _remote_scan_task_sched = std::move(remote_scan_scheduler); } else { diff --git a/be/src/vec/exec/scan/scanner_scheduler.h b/be/src/vec/exec/scan/scanner_scheduler.h index f3f9caaa4d3dc9..01e0859643409a 100644 --- a/be/src/vec/exec/scan/scanner_scheduler.h +++ b/be/src/vec/exec/scan/scanner_scheduler.h @@ -118,10 +118,11 @@ class SimplifiedScanScheduler { _scan_thread_pool->wait(); } - Status start() { + Status start(int max_thread_num, int min_thread_num, int queue_size) { RETURN_IF_ERROR(ThreadPoolBuilder(_sched_name) - .set_min_threads(config::doris_scanner_thread_pool_thread_num) - .set_max_threads(config::doris_scanner_thread_pool_thread_num) + .set_min_threads(min_thread_num) + .set_max_threads(max_thread_num) + .set_max_queue_size(queue_size) .set_cgroup_cpu_ctl(_cgroup_cpu_ctl) .build(&_scan_thread_pool)); return Status::OK(); From 5f27badf38aab18b6b3c0291b0c49e5a0e0a1250 Mon Sep 17 00:00:00 2001 From: walter Date: Sat, 13 Apr 2024 14:29:44 +0800 Subject: [PATCH 48/71] [case](regression) Add backup restore test for hdfs repo (#33581) --- .../test_backup_restore_with_hdfs.groovy | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) create mode 100644 regression-test/suites/backup_restore/test_backup_restore_with_hdfs.groovy diff --git a/regression-test/suites/backup_restore/test_backup_restore_with_hdfs.groovy b/regression-test/suites/backup_restore/test_backup_restore_with_hdfs.groovy new file mode 100644 index 00000000000000..17267a76cd3f55 --- /dev/null +++ b/regression-test/suites/backup_restore/test_backup_restore_with_hdfs.groovy @@ -0,0 +1,89 @@ +// 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_backup_restore_with_hdfs", "backup_restore") { + String suiteName = "test_backup_restore_with_hdfs" + + if (!enableHdfs()) { + logger.info("hdfs is not eabled, skip case ${suiteName}") + return + } + + String repoName = "${suiteName}_repo" + String dbName = "${suiteName}_db" + String tableName = "${suiteName}_table" + String snapshotName = "${suiteName}_snapshot" + + def syncer = getSyncer() + syncer.createHdfsRepository(repoName) + + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "DROP TABLE IF EXISTS ${dbName}.${tableName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0") + AGGREGATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + + List values = [] + for (int i = 1; i <= 10; ++i) { + values.add("(${i}, ${i})") + } + sql "INSERT INTO ${dbName}.${tableName} VALUES ${values.join(",")}" + def result = sql "SELECT * FROM ${dbName}.${tableName}" + assertEquals(result.size(), values.size()); + + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + ON (${tableName}) + """ + + syncer.waitSnapshotFinish(dbName) + + def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + sql "TRUNCATE TABLE ${dbName}.${tableName}" + + sql """ + RESTORE SNAPSHOT ${dbName}.${snapshotName} + FROM `${repoName}` + ON ( `${tableName}`) + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName) + + result = sql "SELECT * FROM ${dbName}.${tableName}" + assertEquals(result.size(), values.size()); + + sql "DROP TABLE ${dbName}.${tableName} FORCE" + sql "DROP DATABASE ${dbName} FORCE" + sql "DROP REPOSITORY `${repoName}`" +} + From c0968834f08e847e9d10e9ed5fe26ce80c511b8d Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Sat, 13 Apr 2024 17:50:22 +0800 Subject: [PATCH 49/71] [bugfix](feoom) add timeout to send be request to avoid too long to occupy memory (#33614) Co-authored-by: yiguolei --- .../main/java/org/apache/doris/rpc/BackendServiceClient.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java index 4027cf6d9510b4..8a74e45edc5f26 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java @@ -85,7 +85,8 @@ public Future execPlanFragmentStartAsyn public ListenableFuture cancelPlanFragmentAsync( InternalService.PCancelPlanFragmentRequest request) { - return stub.cancelPlanFragment(request); + return stub.withDeadlineAfter(execPlanTimeout, TimeUnit.MILLISECONDS) + .cancelPlanFragment(request); } public Future fetchDataAsync(InternalService.PFetchDataRequest request) { From 1562d0e644a13e50caae11b3d2f29cb169792da8 Mon Sep 17 00:00:00 2001 From: zfr95 <87513668+zfr9527@users.noreply.github.com> Date: Sat, 13 Apr 2024 19:08:20 +0800 Subject: [PATCH 50/71] [test](mtmv)Add column name case sensitive test (#33538) --- .../mv/dimension/dimension_1.groovy | 222 +++++++++--------- 1 file changed, 112 insertions(+), 110 deletions(-) diff --git a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy index 50689b2733d385..192f991647cb3d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/dimension/dimension_1.groovy @@ -135,7 +135,7 @@ suite("partition_mv_rewrite_dimension_1") { """ } - def create_mv_all = { mv_name, mv_sql -> + def create_mv = { mv_name, mv_sql -> sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name};""" sql """DROP TABLE IF EXISTS ${mv_name}""" sql""" @@ -168,9 +168,9 @@ suite("partition_mv_rewrite_dimension_1") { // join direction def mv_name_1 = "mv_join_1" def join_direction_mv_1 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey - from lineitem_1 - left join orders_1 + select l_Shipdate, o_Orderdate, l_partkey, l_suppkey + from lineitem_1 + left join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey """ @@ -179,16 +179,16 @@ suite("partition_mv_rewrite_dimension_1") { waitingMTMVTaskFinished(job_name_1) def join_direction_sql_1 = """ - select l_shipdate - from lineitem_1 - left join orders_1 + select L_SHIPDATE + from lineitem_1 + left join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey """ def join_direction_sql_2 = """ - select l_shipdate - from orders_1 - left join lineitem_1 - on orders_1.o_orderkey = lineitem_1.l_orderkey + select L_SHIPDATE + from orders_1 + left join lineitem_1 + on orders_1.o_orderkey = lineitem_1.L_ORDERKEY """ explain { sql("${join_direction_sql_1}") @@ -204,9 +204,9 @@ suite("partition_mv_rewrite_dimension_1") { def mv_name_2 = "mv_join_2" def join_direction_mv_2 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey - from lineitem_1 - inner join orders_1 + select L_SHIPDATE, O_orderdate, l_partkey, l_suppkey + from lineitem_1 + inner join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey """ @@ -215,15 +215,15 @@ suite("partition_mv_rewrite_dimension_1") { waitingMTMVTaskFinished(job_name_2) def join_direction_sql_3 = """ - select l_shipdate - from lineitem_1 - inner join orders_1 + select l_shipdaTe + from lineitem_1 + inner join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey """ def join_direction_sql_4 = """ - select l_shipdate - from orders_1 - inner join lineitem_1 + select L_shipdate + from orders_1 + inner join lineitem_1 on orders_1.o_orderkey = lineitem_1.l_orderkey """ explain { @@ -240,44 +240,44 @@ suite("partition_mv_rewrite_dimension_1") { // join filter position def join_filter_stmt_1 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 + select L_SHIPDATE, o_orderdate, l_partkey, l_suppkey, O_orderkey + from lineitem_1 + left join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey""" def join_filter_stmt_2 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from (select * from lineitem_1 where l_shipdate = '2023-10-17' ) t1 - left join orders_1 + select l_shipdate, o_orderdate, L_partkey, l_suppkey, O_ORDERKEY + from (select * from lineitem_1 where l_shipdate = '2023-10-17' ) t1 + left join orders_1 on t1.l_orderkey = orders_1.o_orderkey""" def join_filter_stmt_3 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join (select * from orders_1 where o_orderdate = '2023-10-17' ) t2 + select l_shipdate, o_orderdate, l_Partkey, l_suppkey, o_orderkey + from lineitem_1 + left join (select * from orders_1 where o_orderdate = '2023-10-17' ) t2 on lineitem_1.l_orderkey = t2.o_orderkey""" def join_filter_stmt_4 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey + select l_shipdate, o_orderdate, l_parTkey, l_suppkey, o_orderkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey where l_shipdate = '2023-10-17' and o_orderdate = '2023-10-17'""" def join_filter_stmt_5 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey + select l_shipdate, o_orderdate, l_partkeY, l_suppkey, o_orderkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey where l_shipdate = '2023-10-17'""" def join_filter_stmt_6 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey + select l_shipdatE, o_orderdate, l_partkey, l_suppkey, o_orderkey + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey where o_orderdate = '2023-10-17'""" def join_filter_stmt_7 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkey - from lineitem_1 - left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey - where orders_1.o_orderkey=1""" + select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderkeY + from lineitem_1 + left join orders_1 + on lineitem_1.l_orderkey = orders_1.o_orderkey + where orders_1.O_ORDERKEY=1""" def mv_list = [ join_filter_stmt_1, join_filter_stmt_2, join_filter_stmt_3, join_filter_stmt_4, @@ -351,7 +351,6 @@ suite("partition_mv_rewrite_dimension_1") { notContains "${join_filter_mv}(${join_filter_mv})" } } - } } else if (i == 4) { for (int j = 0; j < mv_list.size(); j++) { @@ -410,14 +409,17 @@ suite("partition_mv_rewrite_dimension_1") { select l_shipdate, o_orderdate, l_partkey, l_suppkey from lineitem_1 left join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey""" + on lineitem_1.L_ORDERKEY = orders_1.o_orderkey""" def join_type_stmt_2 = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey from lineitem_1 inner join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey""" + + // Todo: right/cross/full/semi/anti join + // Currently, only left join and inner join are supported. def join_type_stmt_3 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey + select l_shipdate, o_orderdatE, l_partkey, l_suppkey from lineitem_1 right join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey""" @@ -426,39 +428,39 @@ suite("partition_mv_rewrite_dimension_1") { // from lineitem_1 // cross join orders_1""" def join_type_stmt_5 = """ - select l_shipdate, o_orderdate, l_partkey, l_suppkey + select l_shipdate, o_orderdate, L_partkey, l_suppkey from lineitem_1 full join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey""" def join_type_stmt_6 = """ - select l_shipdate, l_partkey, l_suppkey, l_linenumber + select l_shipdate, l_partkey, l_suppkey, l_Shipmode, l_orderkey from lineitem_1 left semi join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey""" + on lineitem_1.L_ORDERKEY = orders_1.o_orderkey""" def join_type_stmt_7 = """ - select o_orderdate, o_orderkey, o_custkey, o_orderstatus + select o_orderkey, o_custkey, o_Orderdate, o_clerk, o_totalprice from lineitem_1 right semi join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey""" def join_type_stmt_8 = """ - select l_shipdate, l_partkey, l_suppkey, l_linenumber + select l_shipdate, l_partkey, l_suppkeY, l_shipmode, l_orderkey from lineitem_1 left anti join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey""" + on lineitem_1.l_orderkey = orders_1.o_orderkeY""" def join_type_stmt_9 = """ - select o_orderdate, o_orderkey, o_custkey, o_orderstatus + select o_orderkey, o_custkeY, o_orderdate, o_clerk, o_totalprice from lineitem_1 right anti join orders_1 - on lineitem_1.l_orderkey = orders_1.o_orderkey""" - def join_type_stmt_list = [join_type_stmt_1, join_type_stmt_2, join_type_stmt_3, join_type_stmt_5, - join_type_stmt_6, join_type_stmt_7, join_type_stmt_8, join_type_stmt_9] + on lineitem_1.L_ORDERKEY = orders_1.o_orderkey""" + def join_type_stmt_list = [join_type_stmt_1, join_type_stmt_2, join_type_stmt_3, + join_type_stmt_5, join_type_stmt_6, join_type_stmt_7, join_type_stmt_8, join_type_stmt_9] for (int i = 0; i < join_type_stmt_list.size(); i++) { logger.info("i:" + i) String join_type_mv = """join_type_mv_${i}""" - if (i == 2 || i == 5 || i == 7) { + if (i in [2, 5, 7]) { create_mv_orders(join_type_mv, join_type_stmt_list[i]) } else if (i == 3) { - create_mv_all(join_type_mv, join_type_stmt_list[i]) + create_mv(join_type_mv, join_type_stmt_list[i]) } else { create_mv_lineitem(join_type_mv, join_type_stmt_list[i]) } @@ -494,24 +496,24 @@ suite("partition_mv_rewrite_dimension_1") { PROPERTIES ('replication_num' = '1') AS select - 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 + 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 orders_1 """ def agg_job_name_1 = getJobName(db, agg_mv_name_1) waitingMTMVTaskFinished(agg_job_name_1) - def agg_sql_1 = """select - count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, - count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, - sum(o_totalprice), - max(o_totalprice), - min(o_totalprice), - count(*) + def agg_sql_1 = """select + count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, + count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, + sum(O_totalprice), + max(o_totalprice), + min(o_totalprice), + count(*) from orders_1 """ explain { @@ -524,23 +526,23 @@ suite("partition_mv_rewrite_dimension_1") { // agg + with group by + without agg function def agg_mv_name_2 = "agg_mv_name_2" def agg_mv_stmt_2 = """ - select o_orderdate, o_shippriority, o_comment - from orders_1 - group by - o_orderdate, - o_shippriority, - o_comment + select o_orderdatE, O_SHIPPRIORITY, o_comment + from orders_1 + group by + o_orderdate, + o_shippriority, + o_comment """ create_mv_orders(agg_mv_name_2, agg_mv_stmt_2) def agg_job_name_2 = getJobName(db, agg_mv_name_2) waitingMTMVTaskFinished(agg_job_name_2) sql """analyze table ${agg_mv_name_2} with sync;""" - def agg_sql_2 = """select o_shippriority, o_comment - from orders_1 - group by - o_shippriority, - o_comment + def agg_sql_2 = """select O_shippriority, o_commenT + from orders_1 + group by + o_shippriority, + o_comment """ def agg_sql_explain_2 = sql """explain ${agg_sql_2};""" def mv_index_1 = agg_sql_explain_2.toString().indexOf("MaterializedViewRewriteFail:") @@ -551,35 +553,35 @@ suite("partition_mv_rewrite_dimension_1") { // agg + with group by + with agg function def agg_mv_name_3 = "agg_mv_name_3" def agg_mv_stmt_3 = """ - select o_orderdate, o_shippriority, o_comment, - 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 orders_1 - group by - o_orderdate, - o_shippriority, - o_comment + select o_orderdatE, o_shippriority, o_comment, + 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 orders_1 + group by + o_orderdatE, + o_shippriority, + o_comment """ create_mv_orders(agg_mv_name_3, agg_mv_stmt_3) def agg_job_name_3 = getJobName(db, agg_mv_name_3) waitingMTMVTaskFinished(agg_job_name_3) sql """analyze table ${agg_mv_name_3} with sync;""" - def agg_sql_3 = """select o_shippriority, o_comment, + def agg_sql_3 = """select o_shipprioritY, o_comment, count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, - count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, - sum(o_totalprice), - max(o_totalprice), - min(o_totalprice), - count(*) - from orders_1 - group by - o_shippriority, - o_comment + count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, + sum(o_totalprice), + max(o_totalprice), + min(o_totalprice), + count(*) + from orders_1 + group by + o_shippriority, + o_commenT """ def agg_sql_explain_3 = sql """explain ${agg_sql_3};""" def mv_index_2 = agg_sql_explain_3.toString().indexOf("MaterializedViewRewriteFail:") @@ -616,7 +618,7 @@ suite("partition_mv_rewrite_dimension_1") { // view partital rewriting def view_partition_mv_name_1 = "view_partition_mv_name_1" def view_partition_mv_stmt_1 = """ - select l_shipdate, l_partkey, l_orderkey from lineitem_1 group by l_shipdate, l_partkey, l_orderkey""" + select l_shipdatE, l_partkey, l_orderkey from lineitem_1 group by l_shipdate, l_partkey, l_orderkeY""" create_mv_lineitem(view_partition_mv_name_1, view_partition_mv_stmt_1) def view_partition_job_name_1 = getJobName(db, view_partition_mv_name_1) waitingMTMVTaskFinished(view_partition_job_name_1) @@ -661,7 +663,7 @@ suite("partition_mv_rewrite_dimension_1") { // predicate compensate def predicate_mv_name_1 = "predicate_mv_name_1" def predicate_mv_stmt_1 = """ - select l_shipdate, o_orderdate, l_partkey + select l_shipdatE, o_orderdate, l_partkey from lineitem_1 left join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey @@ -672,7 +674,7 @@ suite("partition_mv_rewrite_dimension_1") { waitingMTMVTaskFinished(predicate_job_name_1) def predicate_sql_1 = """ - select l_shipdate, o_orderdate, l_partkey + select l_shipdate, o_orderdate, l_partkeY from lineitem_1 left join orders_1 on lineitem_1.l_orderkey = orders_1.o_orderkey From d1187c186d2837579946f4cf4991a51fef539fb9 Mon Sep 17 00:00:00 2001 From: Luwei <814383175@qq.com> Date: Sun, 14 Apr 2024 09:40:15 +0800 Subject: [PATCH 51/71] [fix](merge-cloud) fix mtmv data leak in cloud mode (#33619) --- .../main/java/org/apache/doris/catalog/CatalogRecycleBin.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java index 368f0921d127c7..aa081ebd6e2932 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java @@ -352,7 +352,7 @@ private synchronized void eraseTable(long currentTimeMs, int keepNum) { long tableId = table.getId(); if (isExpire(tableId, currentTimeMs)) { - if (table.getType() == TableType.OLAP) { + if (table.getType() == TableType.OLAP || table.getType() == TableType.MATERIALIZED_VIEW) { Env.getCurrentEnv().onEraseOlapTable((OlapTable) table, false); } From d5a3c9f2c16e909a06a2e9a38fbdad7459b8bbe3 Mon Sep 17 00:00:00 2001 From: AlexYue Date: Sun, 14 Apr 2024 09:40:38 +0800 Subject: [PATCH 52/71] [feature](Cloud) Add S3 storage vault implementation (#33377) --- be/src/cloud/cloud_meta_mgr.cpp | 14 +- .../meta-service/meta_service_resource.cpp | 208 +++--- cloud/test/meta_service_test.cpp | 58 +- .../doris/analysis/CreateResourceStmt.java | 35 +- .../apache/doris/catalog/S3StorageVault.java | 74 ++ .../apache/doris/catalog/StorageVault.java | 45 +- .../apache/doris/catalog/StorageVaultMgr.java | 38 +- .../property/constants/S3Properties.java | 17 + .../org/apache/doris/qe/ShowExecutor.java | 8 +- .../cloud/catalog/HdfsStorageVaultTest.java | 18 +- gensrc/proto/cloud.proto | 10 +- .../default_vault_p2}/sql/flat_q1.1.out | 0 .../default_vault_p2}/sql/flat_q1.2.out | 0 .../default_vault_p2}/sql/flat_q1.3.out | 0 .../default_vault_p2}/sql/flat_q2.1.out | 0 .../default_vault_p2}/sql/flat_q2.2.out | 0 .../default_vault_p2}/sql/flat_q2.3.out | 0 .../default_vault_p2}/sql/flat_q3.1.out | 0 .../default_vault_p2}/sql/flat_q3.2.out | 0 .../default_vault_p2}/sql/flat_q3.3.out | 0 .../default_vault_p2}/sql/flat_q3.4.out | 0 .../default_vault_p2}/sql/flat_q4.1.out | 0 .../default_vault_p2}/sql/flat_q4.2.out | 0 .../default_vault_p2}/sql/flat_q4.3.out | 0 .../default_vault_p2}/sql/q1.1.out | 0 .../default_vault_p2}/sql/q1.2.out | 0 .../default_vault_p2}/sql/q1.3.out | 0 .../default_vault_p2}/sql/q2.1.out | 0 .../default_vault_p2}/sql/q2.2.out | 0 .../default_vault_p2}/sql/q2.3.out | 0 .../default_vault_p2}/sql/q3.1.out | 0 .../default_vault_p2}/sql/q3.2.out | 0 .../default_vault_p2}/sql/q3.3.out | 0 .../default_vault_p2}/sql/q3.4.out | 0 .../default_vault_p2}/sql/q4.1.out | 0 .../default_vault_p2}/sql/q4.2.out | 0 .../default_vault_p2}/sql/q4.3.out | 0 .../s3_vault/multi_vault_p2/sql/flat_q1.1.out | 4 + .../s3_vault/multi_vault_p2/sql/flat_q1.2.out | 4 + .../s3_vault/multi_vault_p2/sql/flat_q1.3.out | 4 + .../s3_vault/multi_vault_p2/sql/flat_q2.1.out | 283 +++++++ .../s3_vault/multi_vault_p2/sql/flat_q2.2.out | 59 ++ .../s3_vault/multi_vault_p2/sql/flat_q2.3.out | 10 + .../s3_vault/multi_vault_p2/sql/flat_q3.1.out | 153 ++++ .../s3_vault/multi_vault_p2/sql/flat_q3.2.out | 602 +++++++++++++++ .../s3_vault/multi_vault_p2/sql/flat_q3.3.out | 27 + .../s3_vault/multi_vault_p2/sql/flat_q3.4.out | 6 + .../s3_vault/multi_vault_p2/sql/flat_q4.1.out | 38 + .../s3_vault/multi_vault_p2/sql/flat_q4.2.out | 103 +++ .../s3_vault/multi_vault_p2/sql/flat_q4.3.out | 697 ++++++++++++++++++ .../data/s3_vault/multi_vault_p2/sql/q1.1.out | 4 + .../data/s3_vault/multi_vault_p2/sql/q1.2.out | 4 + .../data/s3_vault/multi_vault_p2/sql/q1.3.out | 4 + .../data/s3_vault/multi_vault_p2/sql/q2.1.out | 283 +++++++ .../data/s3_vault/multi_vault_p2/sql/q2.2.out | 59 ++ .../data/s3_vault/multi_vault_p2/sql/q2.3.out | 10 + .../data/s3_vault/multi_vault_p2/sql/q3.1.out | 153 ++++ .../data/s3_vault/multi_vault_p2/sql/q3.2.out | 602 +++++++++++++++ .../data/s3_vault/multi_vault_p2/sql/q3.3.out | 27 + .../data/s3_vault/multi_vault_p2/sql/q3.4.out | 6 + .../data/s3_vault/multi_vault_p2/sql/q4.1.out | 38 + .../data/s3_vault/multi_vault_p2/sql/q4.2.out | 103 +++ .../data/s3_vault/multi_vault_p2/sql/q4.3.out | 697 ++++++++++++++++++ .../s3_vault/ssb_sf1_p2/sql/flat_q1.1.out | 4 + .../s3_vault/ssb_sf1_p2/sql/flat_q1.2.out | 4 + .../s3_vault/ssb_sf1_p2/sql/flat_q1.3.out | 4 + .../s3_vault/ssb_sf1_p2/sql/flat_q2.1.out | 283 +++++++ .../s3_vault/ssb_sf1_p2/sql/flat_q2.2.out | 59 ++ .../s3_vault/ssb_sf1_p2/sql/flat_q2.3.out | 10 + .../s3_vault/ssb_sf1_p2/sql/flat_q3.1.out | 153 ++++ .../s3_vault/ssb_sf1_p2/sql/flat_q3.2.out | 602 +++++++++++++++ .../s3_vault/ssb_sf1_p2/sql/flat_q3.3.out | 27 + .../s3_vault/ssb_sf1_p2/sql/flat_q3.4.out | 6 + .../s3_vault/ssb_sf1_p2/sql/flat_q4.1.out | 38 + .../s3_vault/ssb_sf1_p2/sql/flat_q4.2.out | 103 +++ .../s3_vault/ssb_sf1_p2/sql/flat_q4.3.out | 697 ++++++++++++++++++ .../data/s3_vault/ssb_sf1_p2/sql/q1.1.out | 4 + .../data/s3_vault/ssb_sf1_p2/sql/q1.2.out | 4 + .../data/s3_vault/ssb_sf1_p2/sql/q1.3.out | 4 + .../data/s3_vault/ssb_sf1_p2/sql/q2.1.out | 283 +++++++ .../data/s3_vault/ssb_sf1_p2/sql/q2.2.out | 59 ++ .../data/s3_vault/ssb_sf1_p2/sql/q2.3.out | 10 + .../data/s3_vault/ssb_sf1_p2/sql/q3.1.out | 153 ++++ .../data/s3_vault/ssb_sf1_p2/sql/q3.2.out | 602 +++++++++++++++ .../data/s3_vault/ssb_sf1_p2/sql/q3.3.out | 27 + .../data/s3_vault/ssb_sf1_p2/sql/q3.4.out | 6 + .../data/s3_vault/ssb_sf1_p2/sql/q4.1.out | 38 + .../data/s3_vault/ssb_sf1_p2/sql/q4.2.out | 103 +++ .../data/s3_vault/ssb_sf1_p2/sql/q4.3.out | 697 ++++++++++++++++++ .../doris/regression/suite/Suite.groovy | 5 + .../hdfs_vault/default_vault_p2/load.groovy | 4 +- .../hdfs_vault/multi_vault_p2/load.groovy | 4 +- .../suites/hdfs_vault/ssb_sf1_p2/load.groovy | 4 +- .../default_vault_p2/ddl/customer_create.sql | 15 + .../default_vault_p2/ddl/customer_delete.sql | 1 + .../default_vault_p2/ddl/date_create.sql | 24 + .../default_vault_p2/ddl/date_delete.sql | 1 + .../default_vault_p2/ddl/lineorder_create.sql | 24 + .../default_vault_p2/ddl/lineorder_delete.sql | 1 + .../ddl/lineorder_flat_create.sql | 45 ++ .../ddl/lineorder_flat_delete.sql | 1 + .../default_vault_p2/ddl/part_create.sql | 16 + .../default_vault_p2/ddl/part_delete.sql | 1 + .../default_vault_p2/ddl/supplier_create.sql | 14 + .../default_vault_p2/ddl/supplier_delete.sql | 1 + .../s3_vault/default_vault_p2/load.groovy | 147 ++++ .../default_vault_p2/sql/flat_q1.1.sql | 24 + .../default_vault_p2/sql/flat_q1.2.sql | 24 + .../default_vault_p2/sql/flat_q1.3.sql | 25 + .../default_vault_p2/sql/flat_q2.1.sql | 24 + .../default_vault_p2/sql/flat_q2.2.sql | 27 + .../default_vault_p2/sql/flat_q2.3.sql | 26 + .../default_vault_p2/sql/flat_q3.1.sql | 29 + .../default_vault_p2/sql/flat_q3.2.sql | 29 + .../default_vault_p2/sql/flat_q3.3.sql | 29 + .../default_vault_p2/sql/flat_q3.4.sql | 29 + .../default_vault_p2/sql/flat_q4.1.sql | 27 + .../default_vault_p2/sql/flat_q4.2.sql | 33 + .../default_vault_p2/sql/flat_q4.3.sql | 29 + .../s3_vault/default_vault_p2/sql/q1.1.sql | 24 + .../s3_vault/default_vault_p2/sql/q1.2.sql | 24 + .../s3_vault/default_vault_p2/sql/q1.3.sql | 25 + .../s3_vault/default_vault_p2/sql/q2.1.sql | 26 + .../s3_vault/default_vault_p2/sql/q2.2.sql | 27 + .../s3_vault/default_vault_p2/sql/q2.3.sql | 26 + .../s3_vault/default_vault_p2/sql/q3.1.sql | 28 + .../s3_vault/default_vault_p2/sql/q3.2.sql | 28 + .../s3_vault/default_vault_p2/sql/q3.3.sql | 30 + .../s3_vault/default_vault_p2/sql/q3.4.sql | 30 + .../s3_vault/default_vault_p2/sql/q4.1.sql | 30 + .../s3_vault/default_vault_p2/sql/q4.2.sql | 31 + .../s3_vault/default_vault_p2/sql/q4.3.sql | 29 + .../multi_vault_p2/ddl/customer_create.sql | 16 + .../multi_vault_p2/ddl/customer_delete.sql | 1 + .../multi_vault_p2/ddl/date_create.sql | 25 + .../multi_vault_p2/ddl/date_delete.sql | 1 + .../multi_vault_p2/ddl/lineorder_create.sql | 25 + .../multi_vault_p2/ddl/lineorder_delete.sql | 1 + .../ddl/lineorder_flat_create.sql | 46 ++ .../ddl/lineorder_flat_delete.sql | 1 + .../multi_vault_p2/ddl/part_create.sql | 17 + .../multi_vault_p2/ddl/part_delete.sql | 1 + .../multi_vault_p2/ddl/supplier_create.sql | 15 + .../multi_vault_p2/ddl/supplier_delete.sql | 1 + .../s3_vault/multi_vault_p2/load.groovy | 158 ++++ .../s3_vault/multi_vault_p2/sql/flat_q1.1.sql | 24 + .../s3_vault/multi_vault_p2/sql/flat_q1.2.sql | 24 + .../s3_vault/multi_vault_p2/sql/flat_q1.3.sql | 25 + .../s3_vault/multi_vault_p2/sql/flat_q2.1.sql | 24 + .../s3_vault/multi_vault_p2/sql/flat_q2.2.sql | 27 + .../s3_vault/multi_vault_p2/sql/flat_q2.3.sql | 26 + .../s3_vault/multi_vault_p2/sql/flat_q3.1.sql | 29 + .../s3_vault/multi_vault_p2/sql/flat_q3.2.sql | 29 + .../s3_vault/multi_vault_p2/sql/flat_q3.3.sql | 29 + .../s3_vault/multi_vault_p2/sql/flat_q3.4.sql | 29 + .../s3_vault/multi_vault_p2/sql/flat_q4.1.sql | 27 + .../s3_vault/multi_vault_p2/sql/flat_q4.2.sql | 33 + .../s3_vault/multi_vault_p2/sql/flat_q4.3.sql | 29 + .../s3_vault/multi_vault_p2/sql/q1.1.sql | 24 + .../s3_vault/multi_vault_p2/sql/q1.2.sql | 24 + .../s3_vault/multi_vault_p2/sql/q1.3.sql | 25 + .../s3_vault/multi_vault_p2/sql/q2.1.sql | 26 + .../s3_vault/multi_vault_p2/sql/q2.2.sql | 27 + .../s3_vault/multi_vault_p2/sql/q2.3.sql | 26 + .../s3_vault/multi_vault_p2/sql/q3.1.sql | 28 + .../s3_vault/multi_vault_p2/sql/q3.2.sql | 28 + .../s3_vault/multi_vault_p2/sql/q3.3.sql | 30 + .../s3_vault/multi_vault_p2/sql/q3.4.sql | 30 + .../s3_vault/multi_vault_p2/sql/q4.1.sql | 30 + .../s3_vault/multi_vault_p2/sql/q4.2.sql | 31 + .../s3_vault/multi_vault_p2/sql/q4.3.sql | 29 + .../ssb_sf1_p2/ddl/customer_create.sql | 16 + .../ssb_sf1_p2/ddl/customer_delete.sql | 1 + .../s3_vault/ssb_sf1_p2/ddl/date_create.sql | 25 + .../s3_vault/ssb_sf1_p2/ddl/date_delete.sql | 1 + .../ssb_sf1_p2/ddl/lineorder_create.sql | 25 + .../ssb_sf1_p2/ddl/lineorder_delete.sql | 1 + .../ssb_sf1_p2/ddl/lineorder_flat_create.sql | 46 ++ .../ssb_sf1_p2/ddl/lineorder_flat_delete.sql | 1 + .../s3_vault/ssb_sf1_p2/ddl/part_create.sql | 17 + .../s3_vault/ssb_sf1_p2/ddl/part_delete.sql | 1 + .../ssb_sf1_p2/ddl/supplier_create.sql | 15 + .../ssb_sf1_p2/ddl/supplier_delete.sql | 1 + .../suites/s3_vault/ssb_sf1_p2/load.groovy | 143 ++++ .../s3_vault/ssb_sf1_p2/sql/flat_q1.1.sql | 24 + .../s3_vault/ssb_sf1_p2/sql/flat_q1.2.sql | 24 + .../s3_vault/ssb_sf1_p2/sql/flat_q1.3.sql | 25 + .../s3_vault/ssb_sf1_p2/sql/flat_q2.1.sql | 24 + .../s3_vault/ssb_sf1_p2/sql/flat_q2.2.sql | 27 + .../s3_vault/ssb_sf1_p2/sql/flat_q2.3.sql | 26 + .../s3_vault/ssb_sf1_p2/sql/flat_q3.1.sql | 29 + .../s3_vault/ssb_sf1_p2/sql/flat_q3.2.sql | 29 + .../s3_vault/ssb_sf1_p2/sql/flat_q3.3.sql | 29 + .../s3_vault/ssb_sf1_p2/sql/flat_q3.4.sql | 29 + .../s3_vault/ssb_sf1_p2/sql/flat_q4.1.sql | 27 + .../s3_vault/ssb_sf1_p2/sql/flat_q4.2.sql | 33 + .../s3_vault/ssb_sf1_p2/sql/flat_q4.3.sql | 29 + .../suites/s3_vault/ssb_sf1_p2/sql/q1.1.sql | 24 + .../suites/s3_vault/ssb_sf1_p2/sql/q1.2.sql | 24 + .../suites/s3_vault/ssb_sf1_p2/sql/q1.3.sql | 25 + .../suites/s3_vault/ssb_sf1_p2/sql/q2.1.sql | 26 + .../suites/s3_vault/ssb_sf1_p2/sql/q2.2.sql | 27 + .../suites/s3_vault/ssb_sf1_p2/sql/q2.3.sql | 26 + .../suites/s3_vault/ssb_sf1_p2/sql/q3.1.sql | 28 + .../suites/s3_vault/ssb_sf1_p2/sql/q3.2.sql | 28 + .../suites/s3_vault/ssb_sf1_p2/sql/q3.3.sql | 30 + .../suites/s3_vault/ssb_sf1_p2/sql/q3.4.sql | 30 + .../suites/s3_vault/ssb_sf1_p2/sql/q4.1.sql | 30 + .../suites/s3_vault/ssb_sf1_p2/sql/q4.2.sql | 31 + .../suites/s3_vault/ssb_sf1_p2/sql/q4.3.sql | 29 + 210 files changed, 11349 insertions(+), 184 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/S3StorageVault.java rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q1.1.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q1.2.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q1.3.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q2.1.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q2.2.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q2.3.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q3.1.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q3.2.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q3.3.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q3.4.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q4.1.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q4.2.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/flat_q4.3.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q1.1.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q1.2.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q1.3.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q2.1.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q2.2.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q2.3.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q3.1.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q3.2.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q3.3.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q3.4.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q4.1.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q4.2.out (100%) rename regression-test/data/{hdfs_vault/default_vault => s3_vault/default_vault_p2}/sql/q4.3.out (100%) create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.1.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.2.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.3.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.1.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.2.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.3.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.1.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.2.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.3.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.4.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.1.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.2.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.3.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q1.1.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q1.2.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q1.3.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q2.1.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q2.2.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q2.3.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q3.1.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q3.2.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q3.3.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q3.4.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q4.1.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q4.2.out create mode 100644 regression-test/data/s3_vault/multi_vault_p2/sql/q4.3.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.1.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.2.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.3.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.1.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.2.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.3.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.1.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.2.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.3.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.4.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.1.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.2.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.3.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.1.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.2.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.3.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.1.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.2.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.3.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.1.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.2.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.3.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.4.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.1.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.2.out create mode 100644 regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.3.out create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/customer_create.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/customer_delete.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/date_create.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/date_delete.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_create.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_delete.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_flat_create.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_flat_delete.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/part_create.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/part_delete.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/supplier_create.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/ddl/supplier_delete.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/load.groovy create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.1.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.2.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.3.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.1.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.2.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.3.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.1.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.2.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.3.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.4.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.1.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.2.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.3.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q1.1.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q1.2.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q1.3.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q2.1.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q2.2.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q2.3.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q3.1.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q3.2.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q3.3.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q3.4.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q4.1.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q4.2.sql create mode 100644 regression-test/suites/s3_vault/default_vault_p2/sql/q4.3.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/customer_create.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/customer_delete.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/date_create.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/date_delete.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_create.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_delete.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_flat_create.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_flat_delete.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/part_create.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/part_delete.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/supplier_create.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/ddl/supplier_delete.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/load.groovy create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.1.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.2.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.3.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.1.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.2.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.3.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.1.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.2.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.3.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.4.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.1.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.2.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.3.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q1.1.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q1.2.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q1.3.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q2.1.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q2.2.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q2.3.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q3.1.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q3.2.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q3.3.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q3.4.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q4.1.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q4.2.sql create mode 100644 regression-test/suites/s3_vault/multi_vault_p2/sql/q4.3.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/customer_create.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/customer_delete.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/date_create.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/date_delete.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_create.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_delete.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_flat_create.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_flat_delete.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/part_create.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/part_delete.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/supplier_create.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/ddl/supplier_delete.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/load.groovy create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.1.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.2.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.3.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.1.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.2.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.3.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.1.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.2.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.3.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.4.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.1.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.2.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.3.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.1.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.2.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.3.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.1.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.2.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.3.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.1.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.2.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.3.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.4.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.1.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.2.sql create mode 100644 regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.3.sql diff --git a/be/src/cloud/cloud_meta_mgr.cpp b/be/src/cloud/cloud_meta_mgr.cpp index c636d92ed822c2..7ee91061136a3f 100644 --- a/be/src/cloud/cloud_meta_mgr.cpp +++ b/be/src/cloud/cloud_meta_mgr.cpp @@ -24,6 +24,7 @@ #include #include +#include #include #include #include @@ -811,7 +812,7 @@ Status CloudMetaMgr::get_storage_vault_info(StorageVaultInfos* vault_infos) { return s; } - for (const auto& obj_store : resp.obj_info()) { + auto add_obj_store = [&vault_infos](const auto& obj_store) { vault_infos->emplace_back(obj_store.id(), S3Conf { .bucket = obj_store.bucket(), @@ -823,12 +824,17 @@ Status CloudMetaMgr::get_storage_vault_info(StorageVaultInfos* vault_infos) { .sse_enabled = obj_store.sse_enabled(), .provider = obj_store.provider(), }); - } - for (const auto& vault : resp.storage_vault()) { + }; + + std::ranges::for_each(resp.obj_info(), add_obj_store); + std::ranges::for_each(resp.storage_vault(), [&](const auto& vault) { if (vault.has_hdfs_info()) { vault_infos->emplace_back(vault.id(), vault.hdfs_info()); } - } + if (vault.has_obj_info()) { + add_obj_store(vault.obj_info()); + } + }); return Status::OK(); } diff --git a/cloud/src/meta-service/meta_service_resource.cpp b/cloud/src/meta-service/meta_service_resource.cpp index f4f8c944c7b6ae..6afff87d76b323 100644 --- a/cloud/src/meta-service/meta_service_resource.cpp +++ b/cloud/src/meta-service/meta_service_resource.cpp @@ -362,7 +362,7 @@ bool normalize_hdfs_fs_name(std::string& fs_name) { } // namespace detail static int add_hdfs_storage_vault(InstanceInfoPB& instance, Transaction* txn, - StorageVaultPB hdfs_param, MetaServiceCode& code, + StorageVaultPB& hdfs_param, MetaServiceCode& code, std::string& msg) { if (!hdfs_param.has_hdfs_info()) { code = MetaServiceCode::INVALID_ARGUMENT; @@ -408,12 +408,83 @@ static int add_hdfs_storage_vault(InstanceInfoPB& instance, Transaction* txn, hdfs_param.set_id(vault_id); std::string val = hdfs_param.SerializeAsString(); txn->put(key, val); - LOG_INFO("try to put storage vault_id={}, vault_name={}", vault_id, hdfs_param.name()); + LOG_INFO("try to put storage vault_id={}, vault_name={}, vault_key={}", vault_id, + hdfs_param.name(), hex(key)); instance.mutable_resource_ids()->Add(std::move(vault_id)); *instance.mutable_storage_vault_names()->Add() = hdfs_param.name(); return 0; } +static void create_object_info_with_encrypt(const InstanceInfoPB& instance, ObjectStoreInfoPB* obj, + bool sse_enabled, MetaServiceCode& code, + std::string& msg) { + std::string plain_ak = obj->has_ak() ? obj->ak() : ""; + std::string plain_sk = obj->has_sk() ? obj->sk() : ""; + std::string bucket = obj->has_bucket() ? obj->bucket() : ""; + std::string prefix = obj->has_prefix() ? obj->prefix() : ""; + // format prefix, such as `/aa/bb/`, `aa/bb//`, `//aa/bb`, ` /aa/bb` -> `aa/bb` + prefix = trim(prefix); + std::string endpoint = obj->has_endpoint() ? obj->endpoint() : ""; + std::string external_endpoint = obj->has_external_endpoint() ? obj->external_endpoint() : ""; + std::string region = obj->has_region() ? obj->region() : ""; + + // ATTN: prefix may be empty + if (plain_ak.empty() || plain_sk.empty() || bucket.empty() || endpoint.empty() || + region.empty() || !obj->has_provider() || external_endpoint.empty()) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "s3 conf info err, please check it"; + return; + } + EncryptionInfoPB encryption_info; + AkSkPair cipher_ak_sk_pair; + auto ret = encrypt_ak_sk_helper(plain_ak, plain_sk, &encryption_info, &cipher_ak_sk_pair, code, + msg); + { + [[maybe_unused]] std::tuple ak_sk_ret {&ret, &code, &msg}; + TEST_SYNC_POINT_CALLBACK("create_object_info_with_encrypt", &ak_sk_ret); + } + if (ret != 0) { + return; + } + + obj->set_ak(std::move(cipher_ak_sk_pair.first)); + obj->set_sk(std::move(cipher_ak_sk_pair.second)); + obj->mutable_encryption_info()->CopyFrom(encryption_info); + obj->set_bucket(bucket); + obj->set_prefix(prefix); + obj->set_endpoint(endpoint); + obj->set_external_endpoint(external_endpoint); + obj->set_region(region); + obj->set_id(next_available_vault_id(instance)); + auto now_time = std::chrono::system_clock::now(); + uint64_t time = + std::chrono::duration_cast(now_time.time_since_epoch()).count(); + obj->set_ctime(time); + obj->set_mtime(time); + obj->set_sse_enabled(sse_enabled); +} + +static int add_vault_into_instance(InstanceInfoPB& instance, Transaction* txn, + StorageVaultPB& vault_param, MetaServiceCode& code, + std::string& msg) { + if (vault_param.has_hdfs_info()) { + return add_hdfs_storage_vault(instance, txn, vault_param, code, msg); + } + create_object_info_with_encrypt(instance, vault_param.mutable_obj_info(), true, code, msg); + if (code != MetaServiceCode::OK) { + return -1; + } + vault_param.mutable_obj_info()->CopyFrom(vault_param.obj_info()); + vault_param.set_id(vault_param.obj_info().id()); + auto vault_key = storage_vault_key({instance.instance_id(), vault_param.obj_info().id()}); + *instance.mutable_resource_ids()->Add() = vault_param.id(); + *instance.mutable_storage_vault_names()->Add() = vault_param.name(); + LOG_INFO("try to put storage vault_id={}, vault_name={}, vault_key={}", vault_param.id(), + vault_param.name(), hex(vault_key)); + txn->put(vault_key, vault_param.SerializeAsString()); + return 0; +} + static int remove_hdfs_storage_vault(InstanceInfoPB& instance, Transaction* txn, const StorageVaultPB& hdfs_info, MetaServiceCode& code, std::string& msg) { @@ -449,16 +520,22 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont RPC_PREPROCESS(alter_obj_store_info); switch (request->op()) { case AlterObjStoreInfoRequest::ADD_OBJ_INFO: + case AlterObjStoreInfoRequest::ADD_S3_VAULT: case AlterObjStoreInfoRequest::LEGACY_UPDATE_AK_SK: case AlterObjStoreInfoRequest::UPDATE_AK_SK: { + if (!request->has_obj() && (!request->has_vault() || !request->vault().has_obj_info())) { + code = MetaServiceCode::INVALID_ARGUMENT; + msg = "s3 obj info err " + proto_to_json(*request); + return; + } + auto& obj = request->has_obj() ? request->obj() : request->vault().obj_info(); // Prepare data - if (!request->has_obj() || !request->obj().has_ak() || !request->obj().has_sk()) { + if (!obj.has_ak() || !obj.has_sk()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "s3 obj info err " + proto_to_json(*request); return; } - auto& obj = request->obj(); std::string plain_ak = obj.has_ak() ? obj.ak() : ""; std::string plain_sk = obj.has_sk() ? obj.sk() : ""; @@ -488,14 +565,14 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont } break; case AlterObjStoreInfoRequest::ADD_HDFS_INFO: case AlterObjStoreInfoRequest::DROP_HDFS_INFO: { - if (!request->has_hdfs() || !request->hdfs().has_name()) { + if (!request->has_vault() || !request->vault().has_name()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "hdfs info is not found " + proto_to_json(*request); return; } } break; case AlterObjStoreInfoRequest::SET_DEFAULT_VAULT: { - if (!request->has_hdfs() || !request->hdfs().has_name()) { + if (!request->has_vault() || !request->vault().has_name()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "hdfs info is not found " + proto_to_json(*request); return; @@ -503,8 +580,8 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont break; } case AlterObjStoreInfoRequest::ADD_BUILT_IN_VAULT: { - // It should at least has one hdfs info or obj info - if ((!request->has_hdfs() && !request->has_obj())) { + // It should at least has one hdfs info or obj info inside storage vault + if ((!request->has_vault())) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "hdfs info is not found " + proto_to_json(*request); return; @@ -575,8 +652,6 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont uint64_t time = std::chrono::duration_cast(now_time.time_since_epoch()).count(); - // TODO(ByteYue): We need to handle different situations like the obj info lies in instance.obj - // or if the obj info lies in secondary indexs switch (request->op()) { case AlterObjStoreInfoRequest::LEGACY_UPDATE_AK_SK: { // get id @@ -605,13 +680,15 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont } } } break; - case AlterObjStoreInfoRequest::ADD_OBJ_INFO: { + case AlterObjStoreInfoRequest::ADD_OBJ_INFO: if (instance.enable_storage_vault()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "Storage vault doesn't support add obj info"; return; } - if (!request->obj().has_provider()) { + case AlterObjStoreInfoRequest::ADD_S3_VAULT: { + auto& obj = request->has_obj() ? request->obj() : request->vault().obj_info(); + if (!obj.has_provider()) { code = MetaServiceCode::INVALID_ARGUMENT; msg = "s3 conf lease provider info"; return; @@ -632,8 +709,7 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont for (auto& it : objs) { if (bucket == it.bucket() && prefix == it.prefix() && endpoint == it.endpoint() && region == it.region() && ak == it.ak() && sk == it.sk() && - request->obj().provider() == it.provider() && - external_endpoint == it.external_endpoint()) { + obj.provider() == it.provider() && external_endpoint == it.external_endpoint()) { // err, anything not changed code = MetaServiceCode::INVALID_ARGUMENT; msg = "original obj infos has a same conf, please check it"; @@ -645,8 +721,8 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont last_item.set_ctime(time); last_item.set_mtime(time); last_item.set_id(next_available_vault_id(instance)); - if (request->obj().has_user_id()) { - last_item.set_user_id(request->obj().user_id()); + if (obj.has_user_id()) { + last_item.set_user_id(obj.user_id()); } last_item.set_ak(std::move(cipher_ak_sk_pair.first)); last_item.set_sk(std::move(cipher_ak_sk_pair.second)); @@ -658,12 +734,24 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont last_item.set_endpoint(endpoint); last_item.set_external_endpoint(external_endpoint); last_item.set_region(region); - last_item.set_provider(request->obj().provider()); + last_item.set_provider(obj.provider()); last_item.set_sse_enabled(instance.sse_enabled()); - instance.add_obj_info()->CopyFrom(last_item); + if (request->op() == AlterObjStoreInfoRequest::ADD_OBJ_INFO) { + instance.add_obj_info()->CopyFrom(last_item); + } else if (request->op() == AlterObjStoreInfoRequest::ADD_S3_VAULT) { + StorageVaultPB vault; + vault.set_id(last_item.id()); + vault.set_name(request->vault().name()); + *instance.mutable_resource_ids()->Add() = vault.id(); + *instance.mutable_storage_vault_names()->Add() = vault.name(); + vault.mutable_obj_info()->MergeFrom(last_item); + auto vault_key = storage_vault_key({instance.instance_id(), last_item.id()}); + txn->put(vault_key, vault.SerializeAsString()); + } } break; case AlterObjStoreInfoRequest::ADD_HDFS_INFO: { - if (auto ret = add_hdfs_storage_vault(instance, txn.get(), request->hdfs(), code, msg); + if (auto ret = add_hdfs_storage_vault( + instance, txn.get(), const_cast(request->vault()), code, msg); ret != 0) { return; } @@ -678,22 +766,22 @@ void MetaServiceImpl::alter_obj_store_info(google::protobuf::RpcController* cont msg = ss.str(); return; } - // TODO(ByteYue): Also support create s3 obj info vault - if (auto ret = add_hdfs_storage_vault(instance, txn.get(), request->hdfs(), code, msg); + if (auto ret = add_hdfs_storage_vault( + instance, txn.get(), const_cast(request->vault()), code, msg); ret != 0) { return; } return; } case AlterObjStoreInfoRequest::DROP_HDFS_INFO: { - if (auto ret = remove_hdfs_storage_vault(instance, txn.get(), request->hdfs(), code, msg); + if (auto ret = remove_hdfs_storage_vault(instance, txn.get(), request->vault(), code, msg); ret != 0) { return; } break; } case AlterObjStoreInfoRequest::SET_DEFAULT_VAULT: { - const auto& name = request->hdfs().name(); + const auto& name = request->vault().name(); auto name_itr = std::find_if(instance.storage_vault_names().begin(), instance.storage_vault_names().end(), [&](const auto& vault_name) { return name == vault_name; }); @@ -935,64 +1023,6 @@ void MetaServiceImpl::update_ak_sk(google::protobuf::RpcController* controller, LOG(INFO) << update_record.str(); } -static int create_instance_with_object_info(InstanceInfoPB& instance, const ObjectStoreInfoPB& obj, - bool sse_enabled, MetaServiceCode& code, - std::string& msg) { - std::string plain_ak = obj.has_ak() ? obj.ak() : ""; - std::string plain_sk = obj.has_sk() ? obj.sk() : ""; - std::string bucket = obj.has_bucket() ? obj.bucket() : ""; - std::string prefix = obj.has_prefix() ? obj.prefix() : ""; - // format prefix, such as `/aa/bb/`, `aa/bb//`, `//aa/bb`, ` /aa/bb` -> `aa/bb` - prefix = trim(prefix); - std::string endpoint = obj.has_endpoint() ? obj.endpoint() : ""; - std::string external_endpoint = obj.has_external_endpoint() ? obj.external_endpoint() : ""; - std::string region = obj.has_region() ? obj.region() : ""; - - // ATTN: prefix may be empty - if (plain_ak.empty() || plain_sk.empty() || bucket.empty() || endpoint.empty() || - region.empty() || !obj.has_provider() || external_endpoint.empty()) { - code = MetaServiceCode::INVALID_ARGUMENT; - msg = "s3 conf info err, please check it"; - return -1; - } - EncryptionInfoPB encryption_info; - AkSkPair cipher_ak_sk_pair; - auto ret = encrypt_ak_sk_helper(plain_ak, plain_sk, &encryption_info, &cipher_ak_sk_pair, code, - msg); - { - [[maybe_unused]] std::tuple ak_sk_ret {&ret, &code, &msg}; - TEST_SYNC_POINT_CALLBACK("create_instance_with_object_info", &ak_sk_ret); - } - if (ret != 0) { - return -1; - } - - ObjectStoreInfoPB obj_info; - if (obj.has_user_id()) { - obj_info.set_user_id(obj.user_id()); - } - obj_info.set_ak(std::move(cipher_ak_sk_pair.first)); - obj_info.set_sk(std::move(cipher_ak_sk_pair.second)); - obj_info.mutable_encryption_info()->CopyFrom(encryption_info); - obj_info.set_bucket(bucket); - obj_info.set_prefix(prefix); - obj_info.set_endpoint(endpoint); - obj_info.set_external_endpoint(external_endpoint); - obj_info.set_region(region); - obj_info.set_provider(obj.provider()); - std::ostringstream oss; - // create instance's s3 conf, id = 1 - obj_info.set_id(next_available_vault_id(instance)); - auto now_time = std::chrono::system_clock::now(); - uint64_t time = - std::chrono::duration_cast(now_time.time_since_epoch()).count(); - obj_info.set_ctime(time); - obj_info.set_mtime(time); - obj_info.set_sse_enabled(sse_enabled); - instance.mutable_obj_info()->Add(std::move(obj_info)); - return 0; -} - void MetaServiceImpl::create_instance(google::protobuf::RpcController* controller, const CreateInstanceRequest* request, CreateInstanceResponse* response, @@ -1019,10 +1049,13 @@ void MetaServiceImpl::create_instance(google::protobuf::RpcController* controlle instance.set_sse_enabled(request->sse_enabled()); instance.set_enable_storage_vault(!request->has_obj_info()); if (request->has_obj_info()) { - if (0 != create_instance_with_object_info(instance, request->obj_info(), - request->sse_enabled(), code, msg)) { + create_object_info_with_encrypt(instance, + const_cast(&request->obj_info()), + request->sse_enabled(), code, msg); + if (code != MetaServiceCode::OK) { return; } + instance.mutable_obj_info()->Add()->MergeFrom(request->obj_info()); } if (request->has_ram_user()) { auto& ram_user = request->ram_user(); @@ -1054,11 +1087,10 @@ void MetaServiceImpl::create_instance(google::protobuf::RpcController* controlle LOG(WARNING) << msg << " err=" << err; return; } - if (request->has_hdfs_info()) { - StorageVaultPB hdfs_param; - hdfs_param.mutable_hdfs_info()->MergeFrom(request->hdfs_info()); - hdfs_param.set_name(BUILT_IN_STORAGE_VAULT_NAME.data()); - if (0 != add_hdfs_storage_vault(instance, txn.get(), std::move(hdfs_param), code, msg)) { + if (request->has_vault()) { + auto& param = const_cast(request->vault()); + param.set_name(BUILT_IN_STORAGE_VAULT_NAME.data()); + if (0 != add_vault_into_instance(instance, txn.get(), param, code, msg)) { return; } } diff --git a/cloud/test/meta_service_test.cpp b/cloud/test/meta_service_test.cpp index 1186c4bcd2c2fa..60cbf3bda64bfb 100644 --- a/cloud/test/meta_service_test.cpp +++ b/cloud/test/meta_service_test.cpp @@ -342,7 +342,9 @@ TEST(MetaServiceTest, CreateInstanceTest) { conf.set_fs_name("hdfs://127.0.0.1:8020"); conf.set_user("test_user"); hdfs.mutable_build_conf()->CopyFrom(conf); - req.mutable_hdfs_info()->CopyFrom(hdfs); + StorageVaultPB vault; + vault.mutable_hdfs_info()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(vault); auto sp = SyncPoint::get_instance(); sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", @@ -5211,9 +5213,9 @@ TEST(MetaServiceTest, AddObjInfoTest) { req.set_cloud_unique_id("test_cloud_unique_id"); req.set_op(AlterObjStoreInfoRequest::ADD_OBJ_INFO); auto sp = SyncPoint::get_instance(); - sp->set_call_back("create_instance_with_object_info", + sp->set_call_back("create_object_info_with_encrypt", [](void* p) { *reinterpret_cast(p) = 0; }); - sp->set_call_back("create_instance_with_object_info::pred", + sp->set_call_back("create_object_info_with_encrypt::pred", [](void* p) { *((bool*)p) = true; }); sp->enable_processing(); @@ -5312,7 +5314,7 @@ TEST(MetaServiceTest, AddHdfsInfoTest) { HdfsVaultInfo params; hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5320,7 +5322,7 @@ TEST(MetaServiceTest, AddHdfsInfoTest) { reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); // Invalid fs name ASSERT_EQ(res.status().code(), MetaServiceCode::INVALID_ARGUMENT) << res.status().msg(); - req.mutable_hdfs()->mutable_hdfs_info()->mutable_build_conf()->set_fs_name( + req.mutable_vault()->mutable_hdfs_info()->mutable_build_conf()->set_fs_name( "hdfs://ip:port"); meta_service->alter_obj_store_info( reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &req, &res, nullptr); @@ -5343,7 +5345,7 @@ TEST(MetaServiceTest, AddHdfsInfoTest) { params.mutable_build_conf()->set_fs_name("hdfs://ip:port"); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5363,7 +5365,7 @@ TEST(MetaServiceTest, AddHdfsInfoTest) { params.mutable_build_conf()->set_fs_name("hdfs://ip:port"); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5451,7 +5453,7 @@ TEST(MetaServiceTest, DropHdfsInfoTest) { HdfsVaultInfo params; hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5471,7 +5473,7 @@ TEST(MetaServiceTest, DropHdfsInfoTest) { HdfsVaultInfo params; hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5503,7 +5505,7 @@ TEST(MetaServiceTest, DropHdfsInfoTest) { params.mutable_build_conf()->MergeFrom(conf); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5530,7 +5532,7 @@ TEST(MetaServiceTest, DropHdfsInfoTest) { params.mutable_build_conf()->MergeFrom(conf); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5556,7 +5558,7 @@ TEST(MetaServiceTest, DropHdfsInfoTest) { params.mutable_build_conf()->MergeFrom(conf); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5579,7 +5581,7 @@ TEST(MetaServiceTest, DropHdfsInfoTest) { HdfsVaultInfo params; hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5617,7 +5619,7 @@ TEST(MetaServiceTest, DropHdfsInfoTest) { params.mutable_build_conf()->MergeFrom(conf); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5661,12 +5663,14 @@ TEST(MetaServiceTest, GetDefaultVaultTest) { conf.set_fs_name("hdfs://127.0.0.1:8020"); conf.set_user("test_user"); hdfs.mutable_build_conf()->CopyFrom(conf); - req.mutable_hdfs_info()->CopyFrom(hdfs); + StorageVaultPB vault; + vault.mutable_hdfs_info()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(vault); auto sp = SyncPoint::get_instance(); - sp->set_call_back("create_instance_with_object_info", + sp->set_call_back("create_object_info_with_encrypt", [](void* p) { *reinterpret_cast(p) = 0; }); - sp->set_call_back("create_instance_with_object_info::pred", + sp->set_call_back("create_object_info_with_encrypt::pred", [](void* p) { *((bool*)p) = true; }); sp->enable_processing(); CreateInstanceResponse res; @@ -5714,14 +5718,14 @@ TEST(MetaServiceTest, GetDefaultVaultTest) { req.mutable_obj_info()->CopyFrom(obj_info); auto sp = SyncPoint::get_instance(); - sp->set_call_back("create_instance_with_object_info", [](void* p) { + sp->set_call_back("create_object_info_with_encrypt", [](void* p) { std::tuple& ret_tuple = *reinterpret_cast*>(p); *std::get<0>(ret_tuple) = 0; *std::get<1>(ret_tuple) = MetaServiceCode::OK; *std::get<2>(ret_tuple) = ""; }); - sp->set_call_back("create_instance_with_object_info::pred", + sp->set_call_back("create_object_info_with_encrypt::pred", [](void* p) { *((bool*)p) = true; }); sp->enable_processing(); CreateInstanceResponse res; @@ -5761,7 +5765,9 @@ TEST(MetaServiceTest, SetDefaultVaultTest) { conf.set_fs_name("hdfs://127.0.0.1:8020"); conf.set_user("test_user"); hdfs.mutable_build_conf()->CopyFrom(conf); - req.mutable_hdfs_info()->CopyFrom(hdfs); + StorageVaultPB vault; + vault.mutable_hdfs_info()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(vault); auto sp = SyncPoint::get_instance(); sp->set_call_back("encrypt_ak_sk:get_encryption_key_ret", @@ -5795,7 +5801,7 @@ TEST(MetaServiceTest, SetDefaultVaultTest) { params.mutable_build_conf()->MergeFrom(conf); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -5806,7 +5812,7 @@ TEST(MetaServiceTest, SetDefaultVaultTest) { AlterObjStoreInfoRequest set_default_req; set_default_req.set_cloud_unique_id("test_cloud_unique_id"); set_default_req.set_op(AlterObjStoreInfoRequest::SET_DEFAULT_VAULT); - set_default_req.mutable_hdfs()->CopyFrom(hdfs); + set_default_req.mutable_vault()->CopyFrom(hdfs); AlterObjStoreInfoResponse set_default_res; meta_service->alter_obj_store_info( reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &set_default_req, @@ -5830,7 +5836,7 @@ TEST(MetaServiceTest, SetDefaultVaultTest) { AlterObjStoreInfoRequest set_default_req; set_default_req.set_cloud_unique_id("test_cloud_unique_id"); set_default_req.set_op(AlterObjStoreInfoRequest::SET_DEFAULT_VAULT); - set_default_req.mutable_hdfs()->CopyFrom(hdfs); + set_default_req.mutable_vault()->CopyFrom(hdfs); AlterObjStoreInfoResponse set_default_res; meta_service->alter_obj_store_info( reinterpret_cast<::google::protobuf::RpcController*>(&cntl), &set_default_req, @@ -5906,7 +5912,7 @@ TEST(MetaServiceTest, GetObjStoreInfoTest) { params.mutable_build_conf()->MergeFrom(conf); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -6026,7 +6032,7 @@ TEST(MetaServiceTest, CreateTabletsVaultsTest) { params.mutable_build_conf()->set_fs_name("hdfs://ip:port"); hdfs.mutable_hdfs_info()->CopyFrom(params); - req.mutable_hdfs()->CopyFrom(hdfs); + req.mutable_vault()->CopyFrom(hdfs); brpc::Controller cntl; AlterObjStoreInfoResponse res; @@ -6053,7 +6059,7 @@ TEST(MetaServiceTest, CreateTabletsVaultsTest) { AlterObjStoreInfoRequest set_default_req; set_default_req.set_cloud_unique_id("test_cloud_unique_id"); set_default_req.set_op(AlterObjStoreInfoRequest::SET_DEFAULT_VAULT); - set_default_req.mutable_hdfs()->CopyFrom(hdfs); + set_default_req.mutable_vault()->CopyFrom(hdfs); AlterObjStoreInfoResponse set_default_res; brpc::Controller cntl; meta_service->alter_obj_store_info( diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateResourceStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateResourceStmt.java index 58316aa9ec5619..9b4ae717ae51ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateResourceStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateResourceStmt.java @@ -67,6 +67,24 @@ public ResourceType getResourceType() { return resourceType; } + public void analyzeResourceType() throws UserException { + String type = properties.get(TYPE); + if (type == null) { + throw new AnalysisException("Resource type can't be null"); + } + resourceType = ResourceType.fromString(type); + if (resourceType == ResourceType.UNKNOWN) { + throw new AnalysisException("Unsupported resource type: " + type); + } + if (resourceType == ResourceType.SPARK && !isExternal) { + throw new AnalysisException("Spark is external resource"); + } + if (resourceType == ResourceType.ODBC_CATALOG && !Config.enable_odbc_mysql_broker_table) { + throw new AnalysisException("ODBC table is deprecated, use JDBC instead. Or you can set " + + "`enable_odbc_mysql_broker_table=true` in fe.conf to enable ODBC again."); + } + } + @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); @@ -83,21 +101,8 @@ public void analyze(Analyzer analyzer) throws UserException { if (properties == null || properties.isEmpty()) { throw new AnalysisException("Resource properties can't be null"); } - String type = properties.get(TYPE); - if (type == null) { - throw new AnalysisException("Resource type can't be null"); - } - resourceType = ResourceType.fromString(type); - if (resourceType == ResourceType.UNKNOWN) { - throw new AnalysisException("Unsupported resource type: " + type); - } - if (resourceType == ResourceType.SPARK && !isExternal) { - throw new AnalysisException("Spark is external resource"); - } - if (resourceType == ResourceType.ODBC_CATALOG && !Config.enable_odbc_mysql_broker_table) { - throw new AnalysisException("ODBC table is deprecated, use JDBC instead. Or you can set " - + "`enable_odbc_mysql_broker_table=true` in fe.conf to enable ODBC again."); - } + + analyzeResourceType(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/S3StorageVault.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/S3StorageVault.java new file mode 100644 index 00000000000000..25f7e60ce3d194 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/S3StorageVault.java @@ -0,0 +1,74 @@ +// 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.catalog; + +import org.apache.doris.analysis.CreateResourceStmt; +import org.apache.doris.common.DdlException; + +import com.google.gson.annotations.SerializedName; + +import java.util.Map; + +/** + * S3 storage vault + *

+ * Syntax: + * CREATE STORAGE VAULT "remote_s3" + * PROPERTIES + * ( + * "type" = "s3", + * "AWS_ENDPOINT" = "bj", + * "AWS_REGION" = "bj", + * "AWS_ROOT_PATH" = "/path/to/root", + * "AWS_ACCESS_KEY" = "bbb", + * "AWS_SECRET_KEY" = "aaaa", + * "AWS_MAX_CONNECTION" = "50", + * "AWS_REQUEST_TIMEOUT_MS" = "3000", + * "AWS_CONNECTION_TIMEOUT_MS" = "1000" + * ); + *

+ * For AWS S3, BE need following properties: + * 1. AWS_ACCESS_KEY: ak + * 2. AWS_SECRET_KEY: sk + * 3. AWS_ENDPOINT: s3.us-east-1.amazonaws.com + * 4. AWS_REGION: us-east-1 + * And file path: s3://bucket_name/csv/taxi.csv + */ +public class S3StorageVault extends StorageVault { + // Reuse all the code from S3Resource + private Resource resource; + + @SerializedName(value = "properties") + private Map properties; + + public S3StorageVault(String name, boolean ifNotExists, CreateResourceStmt stmt) throws DdlException { + super(name, StorageVault.StorageVaultType.S3, ifNotExists); + resource = Resource.fromStmt(stmt); + } + + @Override + public void modifyProperties(Map properties) throws DdlException { + resource.setProperties(properties); + } + + @Override + public Map getCopiedProperties() { + return resource.getCopiedProperties(); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVault.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVault.java index 481ca8d2a845f5..e9b205a81c7943 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVault.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVault.java @@ -17,10 +17,12 @@ package org.apache.doris.catalog; +import org.apache.doris.analysis.CreateResourceStmt; import org.apache.doris.analysis.CreateStorageVaultStmt; import org.apache.doris.cloud.proto.Cloud; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; +import org.apache.doris.common.UserException; import org.apache.doris.qe.ShowResultSetMetaData; import com.google.common.base.Strings; @@ -88,7 +90,7 @@ public StorageVault(String name, StorageVaultType type, boolean ifNotExists) { this.ifNotExists = ifNotExists; } - public static StorageVault fromStmt(CreateStorageVaultStmt stmt) throws DdlException { + public static StorageVault fromStmt(CreateStorageVaultStmt stmt) throws DdlException, UserException { return getStorageVaultInstance(stmt); } @@ -112,7 +114,8 @@ public void setId(String id) { * @return * @throws DdlException */ - private static StorageVault getStorageVaultInstance(CreateStorageVaultStmt stmt) throws DdlException { + private static StorageVault + getStorageVaultInstance(CreateStorageVaultStmt stmt) throws DdlException, UserException { StorageVaultType type = stmt.getStorageVaultType(); String name = stmt.getStorageVaultName(); boolean ifNotExists = stmt.isIfNotExists(); @@ -120,12 +123,17 @@ private static StorageVault getStorageVaultInstance(CreateStorageVaultStmt stmt) switch (type) { case HDFS: vault = new HdfsStorageVault(name, ifNotExists); + vault.modifyProperties(stmt.getProperties()); + break; + case S3: + CreateResourceStmt resourceStmt = + new CreateResourceStmt(false, ifNotExists, name, stmt.getProperties()); + resourceStmt.analyzeResourceType(); + vault = new S3StorageVault(name, ifNotExists, resourceStmt); break; default: throw new DdlException("Unknown StorageVault type: " + type); } - vault.modifyProperties(stmt.getProperties()); - return vault; } @@ -166,28 +174,23 @@ protected void replaceIfEffectiveValue(Map properties, String ke .addColumn(new Column("Propeties", ScalarType.createVarchar(65535))) .build(); - public static List convertToShowStorageVaultProperties(Cloud.ObjectStoreInfoPB info) { - Cloud.ObjectStoreInfoPB.Builder builder = Cloud.ObjectStoreInfoPB.newBuilder(); - builder.mergeFrom(info); - List row = new ArrayList<>(); - row.add(info.getName()); - row.add(info.getId()); - TextFormat.Printer printer = TextFormat.printer(); - builder.clearId(); - builder.clearName(); - builder.setSk("xxxxxxx"); - row.add(printer.shortDebugString(builder)); - return row; - } - public static List convertToShowStorageVaultProperties(Cloud.StorageVaultPB vault) { List row = new ArrayList<>(); row.add(vault.getName()); row.add(vault.getId()); - Cloud.HdfsVaultInfo.Builder builder = Cloud.HdfsVaultInfo.newBuilder(); - builder.mergeFrom(vault.getHdfsInfo()); TextFormat.Printer printer = TextFormat.printer(); - row.add(printer.shortDebugString(builder)); + if (vault.hasHdfsInfo()) { + Cloud.HdfsVaultInfo.Builder builder = Cloud.HdfsVaultInfo.newBuilder(); + builder.mergeFrom(vault.getHdfsInfo()); + row.add(printer.shortDebugString(builder)); + } + if (vault.hasObjInfo()) { + Cloud.ObjectStoreInfoPB.Builder builder = Cloud.ObjectStoreInfoPB.newBuilder(); + builder.mergeFrom(vault.getObjInfo()); + builder.clearId(); + builder.setSk("xxxxxxx"); + row.add(printer.shortDebugString(builder)); + } return row; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVaultMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVaultMgr.java index b473afd6840957..db213d4dc77d69 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVaultMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/StorageVaultMgr.java @@ -24,6 +24,7 @@ import org.apache.doris.cloud.rpc.MetaServiceProxy; import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; +import org.apache.doris.datasource.property.constants.S3Properties; import org.apache.doris.proto.InternalService.PAlterVaultSyncRequest; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.RpcException; @@ -64,11 +65,14 @@ public void createStorageVaultResource(CreateStorageVaultStmt stmt) throws Excep createHdfsVault(StorageVault.fromStmt(stmt)); break; case S3: - throw new DdlException("Currently S3 is not support."); + createS3Vault(StorageVault.fromStmt(stmt)); + break; case UNKNOWN: default: throw new DdlException("Only support S3, HDFS storage vault."); } + // Make BE eagerly fetch the storage vault info from Meta Service + ALTER_BE_SYNC_THREAD_POOL.execute(() -> alterSyncVaultTask()); } @VisibleForTesting @@ -76,7 +80,7 @@ public void setDefaultStorageVault(SetDefaultStorageVaultStmt stmt) throws DdlEx Cloud.AlterObjStoreInfoRequest.Builder builder = Cloud.AlterObjStoreInfoRequest.newBuilder(); Cloud.StorageVaultPB.Builder vaultBuilder = Cloud.StorageVaultPB.newBuilder(); vaultBuilder.setName(stmt.getStorageVaultName()); - builder.setHdfs(vaultBuilder.build()); + builder.setVault(vaultBuilder.build()); builder.setOp(Operation.SET_DEFAULT_VAULT); String vaultId; LOG.info("try to set vault {} as default vault", stmt.getStorageVaultName()); @@ -130,13 +134,12 @@ public void createHdfsVault(StorageVault vault) throws DdlException { Cloud.AlterObjStoreInfoRequest.Builder requestBuilder = Cloud.AlterObjStoreInfoRequest.newBuilder(); requestBuilder.setOp(Cloud.AlterObjStoreInfoRequest.Operation.ADD_HDFS_INFO); - requestBuilder.setHdfs(alterHdfsInfoBuilder.build()); + requestBuilder.setVault(alterHdfsInfoBuilder.build()); try { Cloud.AlterObjStoreInfoResponse response = MetaServiceProxy.getInstance().alterObjStoreInfo(requestBuilder.build()); if (response.getStatus().getCode() == Cloud.MetaServiceCode.ALREADY_EXISTED && hdfsStorageVault.ifNotExists()) { - ALTER_BE_SYNC_THREAD_POOL.execute(() -> alterSyncVaultTask()); return; } if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { @@ -159,4 +162,31 @@ private void alterSyncVaultTask() { } }); } + + public void createS3Vault(StorageVault vault) throws DdlException { + S3StorageVault s3StorageVault = (S3StorageVault) vault; + Cloud.AlterObjStoreInfoRequest.Builder requestBuilder + = Cloud.AlterObjStoreInfoRequest.newBuilder(); + requestBuilder.setOp(Cloud.AlterObjStoreInfoRequest.Operation.ADD_S3_VAULT); + Cloud.ObjectStoreInfoPB.Builder objBuilder = S3Properties.getObjStoreInfoPB(vault.getCopiedProperties()); + Cloud.StorageVaultPB.Builder alterObjVaultBuilder = Cloud.StorageVaultPB.newBuilder(); + alterObjVaultBuilder.setName(s3StorageVault.getName()); + alterObjVaultBuilder.setObjInfo(objBuilder.build()); + requestBuilder.setVault(alterObjVaultBuilder.build()); + try { + Cloud.AlterObjStoreInfoResponse response = + MetaServiceProxy.getInstance().alterObjStoreInfo(requestBuilder.build()); + if (response.getStatus().getCode() == Cloud.MetaServiceCode.ALREADY_EXISTED + && s3StorageVault.ifNotExists()) { + return; + } + if (response.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + LOG.warn("failed to alter storage vault response: {} ", response); + throw new DdlException(response.getStatus().getMsg()); + } + } catch (RpcException e) { + LOG.warn("failed to alter storage vault due to RpcException: {}", e); + throw new DdlException(e.getMessage()); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/S3Properties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/S3Properties.java index 3297a4bd550e72..947174f86ef600 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/S3Properties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/S3Properties.java @@ -17,6 +17,8 @@ package org.apache.doris.datasource.property.constants; +import org.apache.doris.cloud.proto.Cloud; +import org.apache.doris.cloud.proto.Cloud.ObjectStoreInfoPB.Provider; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; import org.apache.doris.datasource.credentials.CloudCredential; @@ -48,6 +50,7 @@ public class S3Properties extends BaseProperties { public static final String CREDENTIALS_PROVIDER = "s3.credentials.provider"; public static final String ENDPOINT = "s3.endpoint"; + public static final String EXTERNAL_ENDPOINT = "s3.external_endpoint"; public static final String REGION = "s3.region"; public static final String ACCESS_KEY = "s3.access_key"; public static final String SECRET_KEY = "s3.secret_key"; @@ -60,6 +63,7 @@ public class S3Properties extends BaseProperties { public static final String ROOT_PATH = "s3.root.path"; public static final String BUCKET = "s3.bucket"; public static final String VALIDITY_CHECK = "s3_validity_check"; + public static final String PROVIDER = "provider"; public static final List REQUIRED_FIELDS = Arrays.asList(ENDPOINT); public static final List TVF_REQUIRED_FIELDS = Arrays.asList(ACCESS_KEY, SECRET_KEY); public static final List FS_KEYS = Arrays.asList(ENDPOINT, REGION, ACCESS_KEY, SECRET_KEY, SESSION_TOKEN, @@ -277,4 +281,17 @@ public static TS3StorageParam getS3TStorageParam(Map properties) s3Info.setUsePathStyle(Boolean.parseBoolean(usePathStyle)); return s3Info; } + + public static Cloud.ObjectStoreInfoPB.Builder getObjStoreInfoPB(Map properties) { + Cloud.ObjectStoreInfoPB.Builder builder = Cloud.ObjectStoreInfoPB.newBuilder(); + builder.setEndpoint(properties.get(S3Properties.ENDPOINT)); + builder.setRegion(properties.get(S3Properties.REGION)); + builder.setAk(properties.get(S3Properties.ACCESS_KEY)); + builder.setSk(properties.get(S3Properties.SECRET_KEY)); + builder.setPrefix(properties.get(S3Properties.ROOT_PATH)); + builder.setBucket(properties.get(S3Properties.BUCKET)); + builder.setExternalEndpoint(properties.get(S3Properties.EXTERNAL_ENDPOINT)); + builder.setProvider(Provider.valueOf(properties.get(S3Properties.PROVIDER))); + return builder; + } } 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 6136a0be42ebd9..f7caec9a657724 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 @@ -268,7 +268,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import java.util.stream.Collectors; -import java.util.stream.Stream; // Execute one show statement. public class ShowExecutor { @@ -3084,11 +3083,8 @@ private void handleShowStorageVault() throws AnalysisException { try { Cloud.GetObjStoreInfoResponse resp = MetaServiceProxy.getInstance() .getObjStoreInfo(Cloud.GetObjStoreInfoRequest.newBuilder().build()); - rows = Stream.concat( - resp.getObjInfoList().stream() - .map(StorageVault::convertToShowStorageVaultProperties), - resp.getStorageVaultList().stream() - .map(StorageVault::convertToShowStorageVaultProperties)) + rows = resp.getStorageVaultList().stream() + .map(StorageVault::convertToShowStorageVaultProperties) .collect(Collectors.toList()); } catch (RpcException e) { throw new AnalysisException(e.getMessage()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/HdfsStorageVaultTest.java b/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/HdfsStorageVaultTest.java index e3458e57a36317..a78f7ad7e7339c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/HdfsStorageVaultTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/cloud/catalog/HdfsStorageVaultTest.java @@ -91,11 +91,11 @@ public void testAlterMetaServiceWithDuplicateName() throws Exception { alterObjStoreInfo(Cloud.AlterObjStoreInfoRequest request) throws RpcException { Cloud.AlterObjStoreInfoResponse.Builder resp = Cloud.AlterObjStoreInfoResponse.newBuilder(); MetaServiceResponseStatus.Builder status = MetaServiceResponseStatus.newBuilder(); - if (existed.contains(request.getHdfs().getName())) { + if (existed.contains(request.getVault().getName())) { status.setCode(MetaServiceCode.ALREADY_EXISTED); } else { status.setCode(MetaServiceCode.OK); - existed.add(request.getHdfs().getName()); + existed.add(request.getVault().getName()); } resp.setStatus(status.build()); resp.setStorageVaultId("1"); @@ -119,7 +119,7 @@ public void testAlterMetaServiceWithMissingFiels() throws Exception { public Cloud.AlterObjStoreInfoResponse alterObjStoreInfo(Cloud.AlterObjStoreInfoRequest request) throws RpcException { Cloud.AlterObjStoreInfoResponse.Builder resp = Cloud.AlterObjStoreInfoResponse.newBuilder(); - if (!request.getHdfs().hasName() || request.getHdfs().getName().isEmpty()) { + if (!request.getVault().hasName() || request.getVault().getName().isEmpty()) { resp.setStatus(MetaServiceResponseStatus.newBuilder() .setCode(MetaServiceCode.INVALID_ARGUMENT).build()); } else { @@ -147,11 +147,11 @@ public void testAlterMetaServiceIfNotExists() throws Exception { alterObjStoreInfo(Cloud.AlterObjStoreInfoRequest request) throws RpcException { Cloud.AlterObjStoreInfoResponse.Builder resp = Cloud.AlterObjStoreInfoResponse.newBuilder(); MetaServiceResponseStatus.Builder status = MetaServiceResponseStatus.newBuilder(); - if (existed.contains(request.getHdfs().getName())) { + if (existed.contains(request.getVault().getName())) { status.setCode(MetaServiceCode.ALREADY_EXISTED); } else { status.setCode(MetaServiceCode.OK); - existed.add(request.getHdfs().getName()); + existed.add(request.getVault().getName()); } resp.setStatus(status.build()); resp.setStorageVaultId("1"); @@ -182,17 +182,17 @@ public Pair getDefaultStorageVaultInfo() { Cloud.AlterObjStoreInfoResponse.Builder resp = Cloud.AlterObjStoreInfoResponse.newBuilder(); MetaServiceResponseStatus.Builder status = MetaServiceResponseStatus.newBuilder(); if (request.getOp() == Operation.ADD_HDFS_INFO) { - if (existed.contains(request.getHdfs().getName())) { + if (existed.contains(request.getVault().getName())) { status.setCode(MetaServiceCode.ALREADY_EXISTED); } else { status.setCode(MetaServiceCode.OK); - existed.add(request.getHdfs().getName()); + existed.add(request.getVault().getName()); } } else if (request.getOp() == Operation.SET_DEFAULT_VAULT) { - if (!existed.contains(request.getHdfs().getName())) { + if (!existed.contains(request.getVault().getName())) { status.setCode(MetaServiceCode.INVALID_ARGUMENT); } else { - this.defaultVaultInfo = Pair.of(request.getHdfs().getName(), "1"); + this.defaultVaultInfo = Pair.of(request.getVault().getName(), "1"); status.setCode(MetaServiceCode.OK); } } diff --git a/gensrc/proto/cloud.proto b/gensrc/proto/cloud.proto index b856d50206b921..37fa2e46715dde 100644 --- a/gensrc/proto/cloud.proto +++ b/gensrc/proto/cloud.proto @@ -191,15 +191,14 @@ message ObjectStoreInfoPB { optional string user_id = 13; optional EncryptionInfoPB encryption_info = 14; optional bool sse_enabled = 15; - optional string name = 16; } -// The legacy OObjectStoreInfoPBbjectStoreInfoPB is stored in instnaceinfopb +// The legacy ObjectStoreInfoPB is stored in InstanceInfoPB message StorageVaultPB { optional string id = 1; optional string name = 2; optional HdfsVaultInfo hdfs_info = 3; // HdfsResource - reserved 4; // reserved for S3. + optional ObjectStoreInfoPB obj_info = 4; } message HdfsBuildConf { @@ -762,13 +761,14 @@ message AlterObjStoreInfoRequest { ADD_HDFS_INFO = 100; DROP_HDFS_INFO = 101; ADD_BUILT_IN_VAULT = 102; + ADD_S3_VAULT = 103; SET_DEFAULT_VAULT = 200; } optional string cloud_unique_id = 1; // For auth optional ObjectStoreInfoPB obj = 2; optional Operation op = 3; - optional StorageVaultPB hdfs = 4; + optional StorageVaultPB vault = 4; } message AlterObjStoreInfoResponse { @@ -912,7 +912,7 @@ message CreateInstanceRequest { optional ObjectStoreInfoPB obj_info = 4; optional RamUserPB ram_user = 5; optional bool sse_enabled = 6; - optional HdfsVaultInfo hdfs_info = 7; + optional StorageVaultPB vault = 7; } message CreateInstanceResponse { diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q1.1.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q1.1.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q1.1.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q1.1.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q1.2.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q1.2.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q1.2.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q1.2.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q1.3.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q1.3.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q1.3.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q1.3.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q2.1.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q2.1.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q2.1.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q2.1.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q2.2.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q2.2.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q2.2.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q2.2.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q2.3.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q2.3.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q2.3.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q2.3.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q3.1.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q3.1.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q3.1.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q3.1.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q3.2.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q3.2.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q3.2.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q3.2.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q3.3.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q3.3.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q3.3.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q3.3.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q3.4.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q3.4.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q3.4.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q3.4.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q4.1.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q4.1.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q4.1.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q4.1.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q4.2.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q4.2.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q4.2.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q4.2.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/flat_q4.3.out b/regression-test/data/s3_vault/default_vault_p2/sql/flat_q4.3.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/flat_q4.3.out rename to regression-test/data/s3_vault/default_vault_p2/sql/flat_q4.3.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q1.1.out b/regression-test/data/s3_vault/default_vault_p2/sql/q1.1.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q1.1.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q1.1.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q1.2.out b/regression-test/data/s3_vault/default_vault_p2/sql/q1.2.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q1.2.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q1.2.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q1.3.out b/regression-test/data/s3_vault/default_vault_p2/sql/q1.3.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q1.3.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q1.3.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q2.1.out b/regression-test/data/s3_vault/default_vault_p2/sql/q2.1.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q2.1.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q2.1.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q2.2.out b/regression-test/data/s3_vault/default_vault_p2/sql/q2.2.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q2.2.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q2.2.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q2.3.out b/regression-test/data/s3_vault/default_vault_p2/sql/q2.3.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q2.3.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q2.3.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q3.1.out b/regression-test/data/s3_vault/default_vault_p2/sql/q3.1.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q3.1.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q3.1.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q3.2.out b/regression-test/data/s3_vault/default_vault_p2/sql/q3.2.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q3.2.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q3.2.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q3.3.out b/regression-test/data/s3_vault/default_vault_p2/sql/q3.3.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q3.3.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q3.3.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q3.4.out b/regression-test/data/s3_vault/default_vault_p2/sql/q3.4.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q3.4.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q3.4.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q4.1.out b/regression-test/data/s3_vault/default_vault_p2/sql/q4.1.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q4.1.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q4.1.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q4.2.out b/regression-test/data/s3_vault/default_vault_p2/sql/q4.2.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q4.2.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q4.2.out diff --git a/regression-test/data/hdfs_vault/default_vault/sql/q4.3.out b/regression-test/data/s3_vault/default_vault_p2/sql/q4.3.out similarity index 100% rename from regression-test/data/hdfs_vault/default_vault/sql/q4.3.out rename to regression-test/data/s3_vault/default_vault_p2/sql/q4.3.out diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.1.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.1.out new file mode 100644 index 00000000000000..58a86b03f0c288 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.1.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q1.1 -- +446031203850 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.2.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.2.out new file mode 100644 index 00000000000000..7ca21d7cbbe869 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.2.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q1.2 -- +98714004603 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.3.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.3.out new file mode 100644 index 00000000000000..b6051a28437af7 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q1.3.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q1.3 -- +26110729246 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.1.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.1.out new file mode 100644 index 00000000000000..9b4932a76b0b93 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.1.out @@ -0,0 +1,283 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q2.1 -- +567838207 1992 MFGR#121 +610663790 1992 MFGR#1210 +550769662 1992 MFGR#1211 +649205856 1992 MFGR#1212 +624031241 1992 MFGR#1213 +670488468 1992 MFGR#1214 +633152470 1992 MFGR#1215 +674846781 1992 MFGR#1216 +675093435 1992 MFGR#1217 +600202070 1992 MFGR#1218 +538043594 1992 MFGR#1219 +655326672 1992 MFGR#122 +540262882 1992 MFGR#1220 +556120633 1992 MFGR#1221 +590762777 1992 MFGR#1222 +535448651 1992 MFGR#1223 +703752611 1992 MFGR#1224 +570832868 1992 MFGR#1225 +614061593 1992 MFGR#1226 +581759388 1992 MFGR#1227 +644642592 1992 MFGR#1228 +640858430 1992 MFGR#1229 +789755835 1992 MFGR#123 +468535087 1992 MFGR#1230 +592436656 1992 MFGR#1231 +664275152 1992 MFGR#1232 +613885100 1992 MFGR#1233 +667399281 1992 MFGR#1234 +640290070 1992 MFGR#1235 +501892561 1992 MFGR#1236 +591481503 1992 MFGR#1237 +477423770 1992 MFGR#1238 +638259374 1992 MFGR#1239 +572354196 1992 MFGR#124 +740479248 1992 MFGR#1240 +478777095 1992 MFGR#125 +592174616 1992 MFGR#126 +706151632 1992 MFGR#127 +542306646 1992 MFGR#128 +581987352 1992 MFGR#129 +823087702 1993 MFGR#121 +648160706 1993 MFGR#1210 +634743898 1993 MFGR#1211 +785639283 1993 MFGR#1212 +638255029 1993 MFGR#1213 +616837237 1993 MFGR#1214 +634687975 1993 MFGR#1215 +638353900 1993 MFGR#1216 +663372951 1993 MFGR#1217 +683985855 1993 MFGR#1218 +646950033 1993 MFGR#1219 +622532984 1993 MFGR#122 +530830127 1993 MFGR#1220 +543346337 1993 MFGR#1221 +756921203 1993 MFGR#1222 +533544350 1993 MFGR#1223 +915916085 1993 MFGR#1224 +473007381 1993 MFGR#1225 +739036124 1993 MFGR#1226 +592178887 1993 MFGR#1227 +583507058 1993 MFGR#1228 +617453491 1993 MFGR#1229 +637863868 1993 MFGR#123 +625534310 1993 MFGR#1230 +580327635 1993 MFGR#1231 +697373098 1993 MFGR#1232 +515571416 1993 MFGR#1233 +651935758 1993 MFGR#1234 +575779480 1993 MFGR#1235 +591878667 1993 MFGR#1236 +609618576 1993 MFGR#1237 +444614010 1993 MFGR#1238 +595256327 1993 MFGR#1239 +660586237 1993 MFGR#124 +788730059 1993 MFGR#1240 +616224539 1993 MFGR#125 +617126754 1993 MFGR#126 +654438324 1993 MFGR#127 +731657001 1993 MFGR#128 +548048395 1993 MFGR#129 +564405648 1994 MFGR#121 +645404849 1994 MFGR#1210 +631620635 1994 MFGR#1211 +568332348 1994 MFGR#1212 +678785857 1994 MFGR#1213 +534002330 1994 MFGR#1214 +654400242 1994 MFGR#1215 +558646341 1994 MFGR#1216 +687845641 1994 MFGR#1217 +546674347 1994 MFGR#1218 +567272942 1994 MFGR#1219 +659884062 1994 MFGR#122 +562582172 1994 MFGR#1220 +598618997 1994 MFGR#1221 +601016441 1994 MFGR#1222 +555134404 1994 MFGR#1223 +737422302 1994 MFGR#1224 +570745955 1994 MFGR#1225 +746302245 1994 MFGR#1226 +651707481 1994 MFGR#1227 +573693547 1994 MFGR#1228 +647918373 1994 MFGR#1229 +580449592 1994 MFGR#123 +493270412 1994 MFGR#1230 +603546148 1994 MFGR#1231 +719865331 1994 MFGR#1232 +638982238 1994 MFGR#1233 +743247677 1994 MFGR#1234 +598680959 1994 MFGR#1235 +615726097 1994 MFGR#1236 +542569815 1994 MFGR#1237 +573510781 1994 MFGR#1238 +579855853 1994 MFGR#1239 +684573322 1994 MFGR#124 +873735737 1994 MFGR#1240 +560488304 1994 MFGR#125 +657036514 1994 MFGR#126 +622571183 1994 MFGR#127 +586845664 1994 MFGR#128 +534541525 1994 MFGR#129 +706469511 1995 MFGR#121 +602892803 1995 MFGR#1210 +645166092 1995 MFGR#1211 +613289283 1995 MFGR#1212 +599586479 1995 MFGR#1213 +562570804 1995 MFGR#1214 +672528755 1995 MFGR#1215 +669000972 1995 MFGR#1216 +725362449 1995 MFGR#1217 +657026635 1995 MFGR#1218 +519659003 1995 MFGR#1219 +724727741 1995 MFGR#122 +517956131 1995 MFGR#1220 +635741351 1995 MFGR#1221 +564368410 1995 MFGR#1222 +600665149 1995 MFGR#1223 +762700351 1995 MFGR#1224 +671669586 1995 MFGR#1225 +572568748 1995 MFGR#1226 +530361300 1995 MFGR#1227 +633357085 1995 MFGR#1228 +547960244 1995 MFGR#1229 +660711077 1995 MFGR#123 +602735858 1995 MFGR#1230 +499852146 1995 MFGR#1231 +715300753 1995 MFGR#1232 +557149571 1995 MFGR#1233 +710023059 1995 MFGR#1234 +622425239 1995 MFGR#1235 +634565501 1995 MFGR#1236 +572847270 1995 MFGR#1237 +549318912 1995 MFGR#1238 +593851712 1995 MFGR#1239 +585421815 1995 MFGR#124 +707207888 1995 MFGR#1240 +538246872 1995 MFGR#125 +605799021 1995 MFGR#126 +665978112 1995 MFGR#127 +646960956 1995 MFGR#128 +508749401 1995 MFGR#129 +523879145 1996 MFGR#121 +643645053 1996 MFGR#1210 +595065339 1996 MFGR#1211 +674626440 1996 MFGR#1212 +496297087 1996 MFGR#1213 +583249505 1996 MFGR#1214 +702184857 1996 MFGR#1215 +601809334 1996 MFGR#1216 +704898387 1996 MFGR#1217 +528843086 1996 MFGR#1218 +586246330 1996 MFGR#1219 +712110492 1996 MFGR#122 +518444215 1996 MFGR#1220 +499319414 1996 MFGR#1221 +679469356 1996 MFGR#1222 +628762754 1996 MFGR#1223 +724844856 1996 MFGR#1224 +660620587 1996 MFGR#1225 +667674729 1996 MFGR#1226 +483838085 1996 MFGR#1227 +609855391 1996 MFGR#1228 +658959557 1996 MFGR#1229 +566217852 1996 MFGR#123 +528879998 1996 MFGR#1230 +589481194 1996 MFGR#1231 +702805896 1996 MFGR#1232 +663679947 1996 MFGR#1233 +571149450 1996 MFGR#1234 +478648074 1996 MFGR#1235 +568249365 1996 MFGR#1236 +592616167 1996 MFGR#1237 +466676148 1996 MFGR#1238 +670693719 1996 MFGR#1239 +560667719 1996 MFGR#124 +821167950 1996 MFGR#1240 +476864333 1996 MFGR#125 +558030884 1996 MFGR#126 +635873891 1996 MFGR#127 +551010618 1996 MFGR#128 +560570630 1996 MFGR#129 +587013207 1997 MFGR#121 +616287892 1997 MFGR#1210 +548588761 1997 MFGR#1211 +589593892 1997 MFGR#1212 +424306670 1997 MFGR#1213 +511971910 1997 MFGR#1214 +631772246 1997 MFGR#1215 +692135140 1997 MFGR#1216 +777994957 1997 MFGR#1217 +707053720 1997 MFGR#1218 +561169527 1997 MFGR#1219 +664916245 1997 MFGR#122 +594466157 1997 MFGR#1220 +588848171 1997 MFGR#1221 +528988960 1997 MFGR#1222 +537098211 1997 MFGR#1223 +674763166 1997 MFGR#1224 +450402292 1997 MFGR#1225 +701360722 1997 MFGR#1226 +506011570 1997 MFGR#1227 +585578737 1997 MFGR#1228 +622744016 1997 MFGR#1229 +646503168 1997 MFGR#123 +571800941 1997 MFGR#1230 +502601790 1997 MFGR#1231 +677924656 1997 MFGR#1232 +534455976 1997 MFGR#1233 +714934715 1997 MFGR#1234 +767151420 1997 MFGR#1235 +618877179 1997 MFGR#1236 +639638057 1997 MFGR#1237 +401953419 1997 MFGR#1238 +610756714 1997 MFGR#1239 +543248087 1997 MFGR#124 +675132692 1997 MFGR#1240 +479099365 1997 MFGR#125 +570696568 1997 MFGR#126 +583074592 1997 MFGR#127 +695133104 1997 MFGR#128 +655638776 1997 MFGR#129 +344575925 1998 MFGR#121 +417152416 1998 MFGR#1210 +317068168 1998 MFGR#1211 +374341516 1998 MFGR#1212 +332740903 1998 MFGR#1213 +304873002 1998 MFGR#1214 +366101132 1998 MFGR#1215 +379133898 1998 MFGR#1216 +359508497 1998 MFGR#1217 +320623334 1998 MFGR#1218 +346182862 1998 MFGR#1219 +312440027 1998 MFGR#122 +348123961 1998 MFGR#1220 +339845398 1998 MFGR#1221 +355416161 1998 MFGR#1222 +344889822 1998 MFGR#1223 +396906691 1998 MFGR#1224 +290208878 1998 MFGR#1225 +419415707 1998 MFGR#1226 +358466340 1998 MFGR#1227 +251549955 1998 MFGR#1228 +383138860 1998 MFGR#1229 +296330561 1998 MFGR#123 +437181243 1998 MFGR#1230 +398944492 1998 MFGR#1231 +424062455 1998 MFGR#1232 +406967188 1998 MFGR#1233 +428867240 1998 MFGR#1234 +352277781 1998 MFGR#1235 +361827086 1998 MFGR#1236 +341618569 1998 MFGR#1237 +244739231 1998 MFGR#1238 +414151803 1998 MFGR#1239 +330082371 1998 MFGR#124 +415312453 1998 MFGR#1240 +360289624 1998 MFGR#125 +341657580 1998 MFGR#126 +377507061 1998 MFGR#127 +361416497 1998 MFGR#128 +318769573 1998 MFGR#129 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.2.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.2.out new file mode 100644 index 00000000000000..4c53da2be7cfff --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.2.out @@ -0,0 +1,59 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q2.2 -- +709524929 1992 MFGR#2221 +783846394 1992 MFGR#2222 +765052002 1992 MFGR#2223 +651488962 1992 MFGR#2224 +646528589 1992 MFGR#2225 +709650548 1992 MFGR#2226 +745556316 1992 MFGR#2227 +756901875 1992 MFGR#2228 +766521103 1993 MFGR#2221 +691475597 1993 MFGR#2222 +758220752 1993 MFGR#2223 +669662707 1993 MFGR#2224 +773854228 1993 MFGR#2225 +737087518 1993 MFGR#2226 +781967766 1993 MFGR#2227 +680880216 1993 MFGR#2228 +685777518 1994 MFGR#2221 +666524807 1994 MFGR#2222 +733993590 1994 MFGR#2223 +707869040 1994 MFGR#2224 +721251967 1994 MFGR#2225 +822495919 1994 MFGR#2226 +720837128 1994 MFGR#2227 +826225350 1994 MFGR#2228 +775437074 1995 MFGR#2221 +761354792 1995 MFGR#2222 +637832575 1995 MFGR#2223 +589765707 1995 MFGR#2224 +708290039 1995 MFGR#2225 +747356383 1995 MFGR#2226 +802502540 1995 MFGR#2227 +895936786 1995 MFGR#2228 +700010008 1996 MFGR#2221 +777310085 1996 MFGR#2222 +656095314 1996 MFGR#2223 +656859917 1996 MFGR#2224 +765820896 1996 MFGR#2225 +808177734 1996 MFGR#2226 +729563303 1996 MFGR#2227 +819665874 1996 MFGR#2228 +727342382 1997 MFGR#2221 +664533779 1997 MFGR#2222 +748288392 1997 MFGR#2223 +630422081 1997 MFGR#2224 +757391203 1997 MFGR#2225 +747889257 1997 MFGR#2226 +728857899 1997 MFGR#2227 +775312985 1997 MFGR#2228 +335304504 1998 MFGR#2221 +409347137 1998 MFGR#2222 +459109577 1998 MFGR#2223 +413318072 1998 MFGR#2224 +410402095 1998 MFGR#2225 +453515044 1998 MFGR#2226 +390506405 1998 MFGR#2227 +397939103 1998 MFGR#2228 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.3.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.3.out new file mode 100644 index 00000000000000..f1a716cfd008b8 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q2.3.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q2.3 -- +726427486 1992 MFGR#2239 +705238959 1993 MFGR#2239 +664145134 1994 MFGR#2239 +713839336 1995 MFGR#2239 +728492865 1996 MFGR#2239 +733896532 1997 MFGR#2239 +380255731 1998 MFGR#2239 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.1.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.1.out new file mode 100644 index 00000000000000..9afae94c3fa253 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.1.out @@ -0,0 +1,153 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q3.1 -- +INDONESIA INDONESIA 1992 6905698988 +CHINA INDONESIA 1992 6616078869 +CHINA CHINA 1992 6456431477 +VIETNAM INDONESIA 1992 6340181707 +VIETNAM CHINA 1992 6332844390 +INDONESIA CHINA 1992 6310709533 +INDIA INDONESIA 1992 6238807354 +JAPAN INDONESIA 1992 6222565638 +CHINA INDIA 1992 6189831351 +CHINA JAPAN 1992 6157678893 +JAPAN CHINA 1992 6067100655 +INDIA CHINA 1992 6066461811 +VIETNAM JAPAN 1992 5863786349 +JAPAN INDIA 1992 5802749985 +INDONESIA INDIA 1992 5770203218 +VIETNAM INDIA 1992 5698511401 +INDONESIA JAPAN 1992 5663765610 +JAPAN JAPAN 1992 5648034711 +INDIA JAPAN 1992 5421959281 +CHINA VIETNAM 1992 5412322026 +JAPAN VIETNAM 1992 5401692555 +INDIA INDIA 1992 5361243755 +INDONESIA VIETNAM 1992 5302638372 +INDIA VIETNAM 1992 5245330621 +VIETNAM VIETNAM 1992 5111731778 +INDONESIA INDONESIA 1993 6931363262 +INDONESIA CHINA 1993 6612891249 +CHINA INDONESIA 1993 6581513366 +VIETNAM INDONESIA 1993 6511639352 +CHINA CHINA 1993 6444513787 +CHINA INDIA 1993 6421694121 +VIETNAM CHINA 1993 6413579999 +INDIA INDONESIA 1993 6331058594 +JAPAN CHINA 1993 6292293995 +INDIA CHINA 1993 6209353792 +CHINA JAPAN 1993 6195466884 +VIETNAM INDIA 1993 6161174477 +INDONESIA INDIA 1993 6151664306 +INDONESIA JAPAN 1993 6147605249 +JAPAN INDONESIA 1993 6053946313 +INDIA JAPAN 1993 5995208985 +CHINA VIETNAM 1993 5885023228 +VIETNAM JAPAN 1993 5874266867 +INDONESIA VIETNAM 1993 5840287722 +JAPAN INDIA 1993 5823343157 +INDIA INDIA 1993 5571575799 +VIETNAM VIETNAM 1993 5554161183 +JAPAN JAPAN 1993 5430318583 +JAPAN VIETNAM 1993 5406569653 +INDIA VIETNAM 1993 5233871487 +VIETNAM CHINA 1994 6709883442 +CHINA CHINA 1994 6648942965 +INDONESIA CHINA 1994 6483600910 +CHINA JAPAN 1994 6349037021 +VIETNAM INDONESIA 1994 6347441931 +JAPAN CHINA 1994 6320009218 +INDONESIA INDONESIA 1994 6315331086 +CHINA INDIA 1994 6297582811 +CHINA INDONESIA 1994 6234787896 +VIETNAM JAPAN 1994 6231973047 +INDONESIA INDIA 1994 6198412245 +INDIA INDONESIA 1994 6168189859 +INDONESIA JAPAN 1994 6141195969 +JAPAN INDONESIA 1994 6013034618 +CHINA VIETNAM 1994 5885318733 +INDIA CHINA 1994 5815022714 +VIETNAM INDIA 1994 5789398691 +JAPAN JAPAN 1994 5753821482 +JAPAN INDIA 1994 5645818872 +INDONESIA VIETNAM 1994 5624346368 +INDIA INDIA 1994 5584936515 +VIETNAM VIETNAM 1994 5418498159 +INDIA JAPAN 1994 5394134974 +JAPAN VIETNAM 1994 5275821637 +INDIA VIETNAM 1994 5251039815 +CHINA INDONESIA 1995 7074539444 +INDONESIA CHINA 1995 6928620620 +CHINA CHINA 1995 6624666612 +JAPAN CHINA 1995 6519889385 +VIETNAM CHINA 1995 6332731268 +INDONESIA INDONESIA 1995 6268531321 +VIETNAM JAPAN 1995 6263957020 +CHINA INDIA 1995 6246656374 +VIETNAM INDIA 1995 6198441957 +INDONESIA INDIA 1995 6173805183 +VIETNAM INDONESIA 1995 6057820148 +CHINA JAPAN 1995 6021854130 +INDONESIA JAPAN 1995 6019093871 +INDIA CHINA 1995 6010532793 +INDIA INDONESIA 1995 5975518597 +JAPAN JAPAN 1995 5952279129 +JAPAN INDONESIA 1995 5947000735 +VIETNAM VIETNAM 1995 5754727607 +JAPAN INDIA 1995 5730743126 +INDONESIA VIETNAM 1995 5574974066 +INDIA INDIA 1995 5565995550 +JAPAN VIETNAM 1995 5501313775 +CHINA VIETNAM 1995 5489936063 +INDIA JAPAN 1995 5469203427 +INDIA VIETNAM 1995 5207063284 +INDONESIA INDONESIA 1996 6750056283 +CHINA INDONESIA 1996 6657125075 +INDONESIA CHINA 1996 6613439112 +CHINA CHINA 1996 6591697915 +VIETNAM CHINA 1996 6428653890 +VIETNAM INDONESIA 1996 6295626732 +JAPAN INDONESIA 1996 6227447856 +INDIA CHINA 1996 6198567819 +INDIA INDONESIA 1996 6189242058 +CHINA INDIA 1996 6153787365 +INDONESIA INDIA 1996 6138810863 +CHINA JAPAN 1996 6105566324 +JAPAN CHINA 1996 6088985564 +INDONESIA JAPAN 1996 6055638222 +VIETNAM JAPAN 1996 5919997150 +VIETNAM VIETNAM 1996 5860842302 +INDIA JAPAN 1996 5841664805 +VIETNAM INDIA 1996 5807486983 +JAPAN INDIA 1996 5644579616 +JAPAN JAPAN 1996 5566204795 +INDIA INDIA 1996 5532073103 +INDONESIA VIETNAM 1996 5438514387 +CHINA VIETNAM 1996 5434772818 +JAPAN VIETNAM 1996 5334277549 +INDIA VIETNAM 1996 5293891531 +CHINA INDONESIA 1997 6653234696 +INDONESIA CHINA 1997 6577396314 +CHINA CHINA 1997 6463794795 +JAPAN INDONESIA 1997 6429270126 +INDONESIA INDONESIA 1997 6398427821 +VIETNAM INDONESIA 1997 6363583120 +CHINA JAPAN 1997 6284733518 +VIETNAM CHINA 1997 6164218723 +INDIA CHINA 1997 6153282214 +CHINA INDIA 1997 6084283983 +INDONESIA JAPAN 1997 6001427956 +INDIA INDONESIA 1997 5983439300 +JAPAN CHINA 1997 5973849687 +CHINA VIETNAM 1997 5908254176 +JAPAN INDIA 1997 5796921992 +JAPAN JAPAN 1997 5790450039 +INDONESIA INDIA 1997 5789367105 +VIETNAM INDIA 1997 5730121608 +INDIA INDIA 1997 5693028907 +VIETNAM JAPAN 1997 5689345230 +INDONESIA VIETNAM 1997 5665678132 +VIETNAM VIETNAM 1997 5620251324 +INDIA JAPAN 1997 5587827913 +JAPAN VIETNAM 1997 5249874614 +INDIA VIETNAM 1997 5003624837 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.2.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.2.out new file mode 100644 index 00000000000000..d0f90c01a154e2 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.2.out @@ -0,0 +1,602 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q3.2 -- +UNITED ST4 UNITED ST1 1992 102027455 +UNITED ST1 UNITED ST0 1992 96989491 +UNITED ST7 UNITED ST0 1992 96078010 +UNITED ST9 UNITED ST0 1992 94813294 +UNITED ST4 UNITED ST0 1992 94644242 +UNITED ST2 UNITED ST4 1992 91180500 +UNITED ST5 UNITED ST0 1992 90432300 +UNITED ST6 UNITED ST7 1992 87658267 +UNITED ST3 UNITED ST9 1992 86142048 +UNITED ST6 UNITED ST5 1992 85882966 +UNITED ST7 UNITED ST3 1992 83765666 +UNITED ST2 UNITED ST9 1992 83705618 +UNITED ST4 UNITED ST6 1992 81886374 +UNITED ST2 UNITED ST1 1992 81839165 +UNITED ST9 UNITED ST1 1992 80795302 +UNITED ST6 UNITED ST3 1992 78778218 +UNITED ST6 UNITED ST0 1992 78696956 +UNITED ST0 UNITED ST1 1992 77267396 +UNITED ST0 UNITED ST0 1992 75622122 +UNITED ST1 UNITED ST9 1992 75367059 +UNITED ST3 UNITED ST1 1992 73637490 +UNITED ST2 UNITED ST0 1992 72210218 +UNITED ST1 UNITED ST7 1992 71472973 +UNITED ST6 UNITED ST4 1992 71086944 +UNITED ST4 UNITED ST4 1992 70111335 +UNITED ST6 UNITED ST1 1992 69408688 +UNITED ST4 UNITED ST3 1992 69001787 +UNITED ST5 UNITED ST7 1992 68333651 +UNITED ST4 UNITED ST9 1992 67837970 +UNITED ST7 UNITED ST6 1992 65513205 +UNITED ST4 UNITED ST5 1992 65057872 +UNITED ST7 UNITED ST4 1992 64900888 +UNITED ST1 UNITED ST1 1992 64669070 +UNITED ST3 UNITED ST5 1992 64239048 +UNITED ST0 UNITED ST9 1992 63979996 +UNITED ST3 UNITED ST4 1992 63144772 +UNITED ST5 UNITED ST6 1992 62628093 +UNITED ST4 UNITED ST7 1992 62529376 +UNITED ST3 UNITED ST0 1992 62441656 +UNITED ST9 UNITED ST4 1992 61489513 +UNITED ST8 UNITED ST6 1992 60540440 +UNITED ST7 UNITED ST9 1992 60326042 +UNITED ST7 UNITED ST7 1992 60121386 +UNITED ST5 UNITED ST1 1992 59945287 +UNITED ST5 UNITED ST4 1992 57625627 +UNITED ST7 UNITED ST5 1992 57566802 +UNITED ST2 UNITED ST5 1992 57021365 +UNITED ST9 UNITED ST7 1992 56883359 +UNITED ST0 UNITED ST3 1992 56359317 +UNITED ST1 UNITED ST3 1992 55727474 +UNITED ST5 UNITED ST3 1992 53963553 +UNITED ST0 UNITED ST7 1992 50583409 +UNITED ST5 UNITED ST9 1992 50191091 +UNITED ST7 UNITED ST1 1992 50167208 +UNITED ST0 UNITED ST8 1992 49732640 +UNITED ST0 UNITED ST4 1992 49676807 +UNITED ST9 UNITED ST3 1992 47681165 +UNITED ST8 UNITED ST4 1992 46757019 +UNITED ST3 UNITED ST3 1992 45087216 +UNITED ST8 UNITED ST0 1992 44368839 +UNITED ST0 UNITED ST6 1992 42471806 +UNITED ST6 UNITED ST8 1992 42463690 +UNITED ST8 UNITED ST7 1992 41897901 +UNITED ST3 UNITED ST8 1992 41275764 +UNITED ST6 UNITED ST9 1992 40591721 +UNITED ST0 UNITED ST5 1992 40120886 +UNITED ST1 UNITED ST4 1992 39326346 +UNITED ST3 UNITED ST7 1992 39028579 +UNITED ST3 UNITED ST6 1992 38798715 +UNITED ST9 UNITED ST9 1992 36048343 +UNITED ST2 UNITED ST8 1992 36046449 +UNITED ST2 UNITED ST3 1992 35981963 +UNITED ST8 UNITED ST1 1992 35680752 +UNITED ST1 UNITED ST6 1992 35404990 +UNITED ST8 UNITED ST5 1992 35187610 +UNITED ST1 UNITED ST5 1992 33971251 +UNITED ST5 UNITED ST8 1992 33878053 +UNITED ST2 UNITED ST7 1992 33702779 +UNITED ST8 UNITED ST3 1992 30949324 +UNITED ST8 UNITED ST8 1992 29309108 +UNITED ST5 UNITED ST5 1992 29279568 +UNITED ST1 UNITED ST8 1992 28565579 +UNITED ST9 UNITED ST5 1992 28075004 +UNITED ST2 UNITED ST6 1992 27813739 +UNITED ST0 UNITED ST2 1992 27718733 +UNITED ST2 UNITED ST2 1992 25743654 +UNITED ST8 UNITED ST9 1992 22684471 +UNITED ST4 UNITED ST8 1992 21928442 +UNITED ST9 UNITED ST8 1992 21386100 +UNITED ST5 UNITED ST2 1992 20495817 +UNITED ST6 UNITED ST6 1992 18137105 +UNITED ST9 UNITED ST6 1992 15879568 +UNITED ST4 UNITED ST2 1992 12061845 +UNITED ST7 UNITED ST8 1992 11895702 +UNITED ST6 UNITED ST2 1992 11820698 +UNITED ST9 UNITED ST2 1992 11623177 +UNITED ST8 UNITED ST2 1992 10971561 +UNITED ST1 UNITED ST2 1992 7706728 +UNITED ST7 UNITED ST2 1992 4943204 +UNITED ST3 UNITED ST2 1992 1097208 +UNITED ST0 UNITED ST9 1993 109834040 +UNITED ST7 UNITED ST0 1993 109788024 +UNITED ST5 UNITED ST0 1993 106822597 +UNITED ST0 UNITED ST0 1993 106742548 +UNITED ST1 UNITED ST0 1993 99305952 +UNITED ST4 UNITED ST4 1993 98150465 +UNITED ST3 UNITED ST4 1993 92493920 +UNITED ST0 UNITED ST1 1993 91196593 +UNITED ST4 UNITED ST1 1993 88521423 +UNITED ST8 UNITED ST0 1993 88356371 +UNITED ST4 UNITED ST7 1993 88172198 +UNITED ST4 UNITED ST0 1993 86918458 +UNITED ST6 UNITED ST3 1993 83417161 +UNITED ST6 UNITED ST1 1993 83345939 +UNITED ST7 UNITED ST9 1993 80310701 +UNITED ST3 UNITED ST1 1993 78230278 +UNITED ST6 UNITED ST7 1993 78197294 +UNITED ST5 UNITED ST9 1993 76286539 +UNITED ST0 UNITED ST3 1993 76171283 +UNITED ST5 UNITED ST8 1993 74359279 +UNITED ST9 UNITED ST1 1993 74059419 +UNITED ST4 UNITED ST9 1993 73296959 +UNITED ST5 UNITED ST1 1993 71454623 +UNITED ST6 UNITED ST4 1993 69646913 +UNITED ST2 UNITED ST1 1993 69631701 +UNITED ST6 UNITED ST0 1993 68247539 +UNITED ST7 UNITED ST7 1993 68109820 +UNITED ST2 UNITED ST3 1993 66972438 +UNITED ST3 UNITED ST0 1993 66626926 +UNITED ST9 UNITED ST7 1993 66625483 +UNITED ST1 UNITED ST8 1993 66146198 +UNITED ST2 UNITED ST7 1993 64185014 +UNITED ST5 UNITED ST4 1993 63415639 +UNITED ST9 UNITED ST9 1993 63260763 +UNITED ST1 UNITED ST4 1993 62884347 +UNITED ST7 UNITED ST4 1993 61656613 +UNITED ST3 UNITED ST6 1993 58584808 +UNITED ST2 UNITED ST4 1993 56650391 +UNITED ST3 UNITED ST5 1993 55907305 +UNITED ST6 UNITED ST9 1993 54900942 +UNITED ST1 UNITED ST7 1993 54851183 +UNITED ST3 UNITED ST9 1993 54762596 +UNITED ST8 UNITED ST6 1993 54633062 +UNITED ST8 UNITED ST3 1993 54049874 +UNITED ST5 UNITED ST7 1993 52745538 +UNITED ST0 UNITED ST5 1993 52701052 +UNITED ST1 UNITED ST9 1993 52514902 +UNITED ST8 UNITED ST5 1993 52237837 +UNITED ST1 UNITED ST3 1993 52097946 +UNITED ST8 UNITED ST4 1993 51419356 +UNITED ST0 UNITED ST6 1993 50432282 +UNITED ST5 UNITED ST5 1993 50357189 +UNITED ST3 UNITED ST7 1993 50135448 +UNITED ST0 UNITED ST4 1993 49260067 +UNITED ST0 UNITED ST7 1993 48796360 +UNITED ST2 UNITED ST9 1993 48188507 +UNITED ST1 UNITED ST1 1993 47538610 +UNITED ST9 UNITED ST3 1993 46943647 +UNITED ST7 UNITED ST5 1993 44763692 +UNITED ST1 UNITED ST6 1993 44728540 +UNITED ST8 UNITED ST1 1993 44415434 +UNITED ST7 UNITED ST8 1993 43902628 +UNITED ST9 UNITED ST6 1993 43867160 +UNITED ST2 UNITED ST0 1993 42845485 +UNITED ST3 UNITED ST8 1993 42251848 +UNITED ST0 UNITED ST8 1993 42002682 +UNITED ST4 UNITED ST8 1993 41657582 +UNITED ST1 UNITED ST5 1993 40693513 +UNITED ST9 UNITED ST5 1993 39685269 +UNITED ST7 UNITED ST3 1993 39523861 +UNITED ST8 UNITED ST8 1993 38790235 +UNITED ST8 UNITED ST9 1993 38516361 +UNITED ST2 UNITED ST5 1993 37406845 +UNITED ST9 UNITED ST8 1993 37184696 +UNITED ST8 UNITED ST7 1993 36902218 +UNITED ST6 UNITED ST8 1993 36456741 +UNITED ST7 UNITED ST1 1993 34391159 +UNITED ST6 UNITED ST5 1993 34229082 +UNITED ST5 UNITED ST3 1993 34031811 +UNITED ST2 UNITED ST8 1993 33445446 +UNITED ST4 UNITED ST3 1993 33129412 +UNITED ST6 UNITED ST6 1993 33050663 +UNITED ST9 UNITED ST0 1993 32653305 +UNITED ST4 UNITED ST6 1993 30699255 +UNITED ST9 UNITED ST4 1993 30644687 +UNITED ST4 UNITED ST5 1993 29119594 +UNITED ST7 UNITED ST6 1993 27100502 +UNITED ST4 UNITED ST2 1993 27012678 +UNITED ST2 UNITED ST6 1993 26482226 +UNITED ST5 UNITED ST6 1993 25357679 +UNITED ST3 UNITED ST3 1993 21777144 +UNITED ST3 UNITED ST2 1993 21559073 +UNITED ST5 UNITED ST2 1993 20610242 +UNITED ST7 UNITED ST2 1993 20219304 +UNITED ST6 UNITED ST2 1993 18814367 +UNITED ST9 UNITED ST2 1993 17718390 +UNITED ST1 UNITED ST2 1993 16844538 +UNITED ST0 UNITED ST2 1993 15042145 +UNITED ST2 UNITED ST2 1993 14521995 +UNITED ST8 UNITED ST2 1993 9984366 +UNITED ST8 UNITED ST0 1994 99220789 +UNITED ST3 UNITED ST9 1994 97476185 +UNITED ST6 UNITED ST1 1994 96937147 +UNITED ST6 UNITED ST9 1994 94683309 +UNITED ST9 UNITED ST1 1994 90440948 +UNITED ST0 UNITED ST9 1994 89865202 +UNITED ST5 UNITED ST7 1994 89089961 +UNITED ST9 UNITED ST0 1994 87670573 +UNITED ST3 UNITED ST1 1994 85523653 +UNITED ST4 UNITED ST9 1994 83822393 +UNITED ST0 UNITED ST0 1994 83526877 +UNITED ST7 UNITED ST0 1994 82265536 +UNITED ST2 UNITED ST1 1994 81300089 +UNITED ST5 UNITED ST0 1994 78648057 +UNITED ST4 UNITED ST7 1994 76954140 +UNITED ST4 UNITED ST4 1994 76837381 +UNITED ST0 UNITED ST1 1994 76613379 +UNITED ST1 UNITED ST3 1994 75992459 +UNITED ST7 UNITED ST1 1994 75320799 +UNITED ST4 UNITED ST0 1994 73719340 +UNITED ST5 UNITED ST1 1994 73508418 +UNITED ST4 UNITED ST1 1994 72219557 +UNITED ST2 UNITED ST9 1994 69671054 +UNITED ST6 UNITED ST5 1994 66461534 +UNITED ST2 UNITED ST3 1994 65620760 +UNITED ST3 UNITED ST0 1994 65522727 +UNITED ST5 UNITED ST3 1994 65334911 +UNITED ST7 UNITED ST4 1994 64778715 +UNITED ST3 UNITED ST4 1994 63412365 +UNITED ST8 UNITED ST4 1994 62141681 +UNITED ST0 UNITED ST4 1994 61519744 +UNITED ST0 UNITED ST7 1994 61480820 +UNITED ST0 UNITED ST6 1994 61288778 +UNITED ST2 UNITED ST0 1994 60182153 +UNITED ST6 UNITED ST4 1994 59829989 +UNITED ST4 UNITED ST5 1994 59397028 +UNITED ST8 UNITED ST9 1994 58666906 +UNITED ST4 UNITED ST6 1994 58633482 +UNITED ST5 UNITED ST5 1994 56235213 +UNITED ST6 UNITED ST3 1994 56123238 +UNITED ST2 UNITED ST4 1994 55679377 +UNITED ST8 UNITED ST3 1994 55203841 +UNITED ST1 UNITED ST1 1994 54383174 +UNITED ST1 UNITED ST7 1994 53853106 +UNITED ST6 UNITED ST0 1994 53728853 +UNITED ST5 UNITED ST9 1994 53367331 +UNITED ST9 UNITED ST9 1994 51980849 +UNITED ST5 UNITED ST4 1994 51799593 +UNITED ST7 UNITED ST9 1994 50144085 +UNITED ST7 UNITED ST7 1994 46446442 +UNITED ST6 UNITED ST6 1994 46199722 +UNITED ST7 UNITED ST5 1994 45895364 +UNITED ST3 UNITED ST3 1994 45627153 +UNITED ST8 UNITED ST5 1994 44553056 +UNITED ST9 UNITED ST4 1994 43910761 +UNITED ST1 UNITED ST0 1994 43225201 +UNITED ST1 UNITED ST9 1994 43000037 +UNITED ST7 UNITED ST8 1994 42776467 +UNITED ST0 UNITED ST5 1994 41808301 +UNITED ST2 UNITED ST6 1994 41526105 +UNITED ST1 UNITED ST4 1994 41381558 +UNITED ST3 UNITED ST7 1994 40935131 +UNITED ST8 UNITED ST1 1994 40152096 +UNITED ST9 UNITED ST8 1994 39278808 +UNITED ST5 UNITED ST6 1994 38658451 +UNITED ST2 UNITED ST5 1994 37640317 +UNITED ST8 UNITED ST7 1994 37600687 +UNITED ST9 UNITED ST5 1994 37146726 +UNITED ST6 UNITED ST7 1994 37057808 +UNITED ST8 UNITED ST6 1994 36776569 +UNITED ST3 UNITED ST6 1994 36290257 +UNITED ST9 UNITED ST3 1994 35846500 +UNITED ST2 UNITED ST8 1994 33767774 +UNITED ST0 UNITED ST8 1994 31845433 +UNITED ST4 UNITED ST3 1994 31599433 +UNITED ST9 UNITED ST7 1994 31586173 +UNITED ST1 UNITED ST6 1994 31287326 +UNITED ST1 UNITED ST8 1994 30245153 +UNITED ST7 UNITED ST3 1994 29424840 +UNITED ST9 UNITED ST6 1994 29212927 +UNITED ST0 UNITED ST3 1994 27327829 +UNITED ST6 UNITED ST8 1994 26592996 +UNITED ST3 UNITED ST5 1994 26197875 +UNITED ST6 UNITED ST2 1994 25809000 +UNITED ST1 UNITED ST5 1994 24939138 +UNITED ST7 UNITED ST6 1994 24631937 +UNITED ST1 UNITED ST2 1994 23556586 +UNITED ST4 UNITED ST2 1994 23035892 +UNITED ST2 UNITED ST7 1994 22182758 +UNITED ST0 UNITED ST2 1994 22017954 +UNITED ST4 UNITED ST8 1994 20685352 +UNITED ST7 UNITED ST2 1994 19655081 +UNITED ST5 UNITED ST8 1994 18931891 +UNITED ST2 UNITED ST2 1994 18068657 +UNITED ST3 UNITED ST8 1994 15936051 +UNITED ST8 UNITED ST8 1994 10023412 +UNITED ST3 UNITED ST2 1994 9995234 +UNITED ST9 UNITED ST2 1994 9700989 +UNITED ST5 UNITED ST2 1994 7162796 +UNITED ST8 UNITED ST2 1994 3789626 +UNITED ST5 UNITED ST1 1995 119793669 +UNITED ST4 UNITED ST9 1995 99490068 +UNITED ST7 UNITED ST0 1995 98031295 +UNITED ST6 UNITED ST0 1995 91718471 +UNITED ST4 UNITED ST1 1995 90878653 +UNITED ST0 UNITED ST1 1995 90763599 +UNITED ST8 UNITED ST9 1995 88855089 +UNITED ST7 UNITED ST7 1995 86571624 +UNITED ST3 UNITED ST0 1995 84462733 +UNITED ST9 UNITED ST1 1995 82938967 +UNITED ST2 UNITED ST4 1995 82432305 +UNITED ST1 UNITED ST0 1995 81676623 +UNITED ST5 UNITED ST4 1995 81016761 +UNITED ST7 UNITED ST1 1995 79964362 +UNITED ST5 UNITED ST3 1995 78099130 +UNITED ST5 UNITED ST0 1995 77615746 +UNITED ST9 UNITED ST9 1995 76515958 +UNITED ST7 UNITED ST9 1995 75317709 +UNITED ST4 UNITED ST4 1995 74587071 +UNITED ST9 UNITED ST4 1995 72525686 +UNITED ST1 UNITED ST9 1995 72470870 +UNITED ST4 UNITED ST7 1995 69264407 +UNITED ST6 UNITED ST3 1995 67513062 +UNITED ST2 UNITED ST3 1995 65218129 +UNITED ST2 UNITED ST9 1995 65055178 +UNITED ST7 UNITED ST6 1995 65020671 +UNITED ST3 UNITED ST1 1995 64762815 +UNITED ST1 UNITED ST1 1995 64199332 +UNITED ST6 UNITED ST9 1995 63457105 +UNITED ST0 UNITED ST9 1995 63253499 +UNITED ST5 UNITED ST9 1995 62364897 +UNITED ST4 UNITED ST5 1995 62081505 +UNITED ST1 UNITED ST7 1995 61515741 +UNITED ST2 UNITED ST7 1995 60000208 +UNITED ST8 UNITED ST6 1995 58990404 +UNITED ST1 UNITED ST4 1995 57535599 +UNITED ST0 UNITED ST3 1995 56360708 +UNITED ST8 UNITED ST0 1995 55231664 +UNITED ST5 UNITED ST7 1995 53740759 +UNITED ST2 UNITED ST0 1995 52560838 +UNITED ST3 UNITED ST7 1995 51579548 +UNITED ST9 UNITED ST0 1995 51548621 +UNITED ST6 UNITED ST6 1995 50954677 +UNITED ST5 UNITED ST5 1995 50394007 +UNITED ST7 UNITED ST4 1995 49899545 +UNITED ST3 UNITED ST3 1995 48158089 +UNITED ST6 UNITED ST4 1995 47697241 +UNITED ST9 UNITED ST7 1995 46464589 +UNITED ST4 UNITED ST0 1995 46142899 +UNITED ST1 UNITED ST3 1995 45823056 +UNITED ST2 UNITED ST1 1995 45437340 +UNITED ST6 UNITED ST5 1995 45428152 +UNITED ST8 UNITED ST5 1995 44994863 +UNITED ST7 UNITED ST3 1995 43699734 +UNITED ST9 UNITED ST6 1995 43482494 +UNITED ST2 UNITED ST5 1995 43382417 +UNITED ST6 UNITED ST8 1995 41973920 +UNITED ST0 UNITED ST6 1995 40718942 +UNITED ST3 UNITED ST5 1995 40057815 +UNITED ST7 UNITED ST5 1995 39015293 +UNITED ST0 UNITED ST0 1995 38984502 +UNITED ST6 UNITED ST1 1995 38328352 +UNITED ST4 UNITED ST6 1995 38109524 +UNITED ST3 UNITED ST9 1995 37364623 +UNITED ST4 UNITED ST3 1995 37356396 +UNITED ST2 UNITED ST6 1995 37146288 +UNITED ST9 UNITED ST5 1995 36009924 +UNITED ST1 UNITED ST8 1995 34918793 +UNITED ST8 UNITED ST1 1995 34217780 +UNITED ST0 UNITED ST7 1995 33395313 +UNITED ST1 UNITED ST5 1995 31857452 +UNITED ST8 UNITED ST7 1995 30918202 +UNITED ST2 UNITED ST8 1995 30504189 +UNITED ST3 UNITED ST4 1995 30422346 +UNITED ST5 UNITED ST6 1995 30204737 +UNITED ST8 UNITED ST3 1995 29349938 +UNITED ST0 UNITED ST4 1995 29170038 +UNITED ST1 UNITED ST6 1995 27139403 +UNITED ST7 UNITED ST8 1995 26444490 +UNITED ST6 UNITED ST7 1995 23833977 +UNITED ST4 UNITED ST8 1995 23053236 +UNITED ST4 UNITED ST2 1995 22787003 +UNITED ST3 UNITED ST8 1995 22505239 +UNITED ST9 UNITED ST8 1995 21292527 +UNITED ST8 UNITED ST4 1995 19287311 +UNITED ST8 UNITED ST2 1995 18282990 +UNITED ST9 UNITED ST3 1995 17539102 +UNITED ST3 UNITED ST6 1995 16738530 +UNITED ST0 UNITED ST8 1995 16393249 +UNITED ST5 UNITED ST2 1995 14951023 +UNITED ST2 UNITED ST2 1995 13455031 +UNITED ST5 UNITED ST8 1995 13346932 +UNITED ST3 UNITED ST2 1995 12886829 +UNITED ST9 UNITED ST2 1995 12653362 +UNITED ST0 UNITED ST5 1995 11453709 +UNITED ST6 UNITED ST2 1995 11363551 +UNITED ST8 UNITED ST8 1995 11285867 +UNITED ST1 UNITED ST2 1995 7991676 +UNITED ST0 UNITED ST2 1995 4776460 +UNITED ST7 UNITED ST2 1995 3973565 +UNITED ST6 UNITED ST0 1996 132286763 +UNITED ST4 UNITED ST0 1996 106897563 +UNITED ST5 UNITED ST0 1996 104501979 +UNITED ST0 UNITED ST4 1996 103228749 +UNITED ST9 UNITED ST1 1996 101983827 +UNITED ST1 UNITED ST0 1996 94861554 +UNITED ST0 UNITED ST1 1996 91948777 +UNITED ST6 UNITED ST1 1996 89705870 +UNITED ST2 UNITED ST1 1996 88256155 +UNITED ST1 UNITED ST1 1996 87265848 +UNITED ST4 UNITED ST7 1996 83677814 +UNITED ST6 UNITED ST3 1996 82168229 +UNITED ST2 UNITED ST7 1996 80468477 +UNITED ST8 UNITED ST1 1996 78971756 +UNITED ST7 UNITED ST4 1996 77941011 +UNITED ST1 UNITED ST3 1996 77610905 +UNITED ST9 UNITED ST9 1996 77301740 +UNITED ST0 UNITED ST9 1996 75935209 +UNITED ST7 UNITED ST0 1996 75602445 +UNITED ST3 UNITED ST1 1996 74746699 +UNITED ST7 UNITED ST7 1996 74040644 +UNITED ST4 UNITED ST1 1996 72819867 +UNITED ST5 UNITED ST9 1996 72614114 +UNITED ST1 UNITED ST9 1996 69823769 +UNITED ST9 UNITED ST4 1996 69616614 +UNITED ST6 UNITED ST4 1996 69296005 +UNITED ST2 UNITED ST0 1996 67095122 +UNITED ST5 UNITED ST1 1996 65346389 +UNITED ST6 UNITED ST9 1996 63256182 +UNITED ST4 UNITED ST6 1996 62189328 +UNITED ST0 UNITED ST0 1996 61528855 +UNITED ST8 UNITED ST9 1996 60466691 +UNITED ST3 UNITED ST0 1996 60226840 +UNITED ST8 UNITED ST6 1996 59746655 +UNITED ST2 UNITED ST3 1996 59648598 +UNITED ST0 UNITED ST5 1996 57762895 +UNITED ST8 UNITED ST7 1996 57523925 +UNITED ST2 UNITED ST4 1996 57487057 +UNITED ST6 UNITED ST7 1996 57090619 +UNITED ST3 UNITED ST4 1996 54838259 +UNITED ST4 UNITED ST9 1996 54134840 +UNITED ST1 UNITED ST6 1996 54056366 +UNITED ST3 UNITED ST7 1996 53987218 +UNITED ST2 UNITED ST9 1996 53491415 +UNITED ST4 UNITED ST8 1996 53035662 +UNITED ST9 UNITED ST5 1996 52825922 +UNITED ST7 UNITED ST3 1996 52356886 +UNITED ST6 UNITED ST8 1996 52136784 +UNITED ST1 UNITED ST5 1996 51189649 +UNITED ST8 UNITED ST4 1996 51033054 +UNITED ST1 UNITED ST4 1996 50135547 +UNITED ST3 UNITED ST9 1996 49612304 +UNITED ST9 UNITED ST0 1996 49590701 +UNITED ST3 UNITED ST3 1996 49263796 +UNITED ST9 UNITED ST7 1996 48798759 +UNITED ST7 UNITED ST1 1996 48784175 +UNITED ST9 UNITED ST6 1996 48685063 +UNITED ST2 UNITED ST5 1996 47028976 +UNITED ST9 UNITED ST3 1996 47021018 +UNITED ST2 UNITED ST8 1996 46865113 +UNITED ST4 UNITED ST3 1996 46460940 +UNITED ST6 UNITED ST5 1996 46030104 +UNITED ST2 UNITED ST6 1996 45416649 +UNITED ST8 UNITED ST5 1996 43480473 +UNITED ST5 UNITED ST5 1996 43020722 +UNITED ST6 UNITED ST6 1996 42923032 +UNITED ST4 UNITED ST5 1996 42808412 +UNITED ST3 UNITED ST6 1996 41881628 +UNITED ST1 UNITED ST7 1996 41721506 +UNITED ST5 UNITED ST7 1996 40946330 +UNITED ST8 UNITED ST0 1996 39845427 +UNITED ST8 UNITED ST3 1996 39535940 +UNITED ST1 UNITED ST8 1996 39430882 +UNITED ST5 UNITED ST6 1996 38332044 +UNITED ST0 UNITED ST6 1996 37232062 +UNITED ST7 UNITED ST6 1996 36535628 +UNITED ST9 UNITED ST8 1996 36112301 +UNITED ST3 UNITED ST8 1996 33924732 +UNITED ST3 UNITED ST5 1996 33717439 +UNITED ST5 UNITED ST4 1996 33424859 +UNITED ST5 UNITED ST3 1996 32919926 +UNITED ST4 UNITED ST4 1996 32787995 +UNITED ST7 UNITED ST5 1996 32784224 +UNITED ST5 UNITED ST8 1996 32415682 +UNITED ST0 UNITED ST7 1996 31391181 +UNITED ST0 UNITED ST3 1996 29795665 +UNITED ST7 UNITED ST9 1996 25028091 +UNITED ST7 UNITED ST8 1996 24348851 +UNITED ST6 UNITED ST2 1996 20447847 +UNITED ST8 UNITED ST8 1996 16340603 +UNITED ST0 UNITED ST8 1996 15168262 +UNITED ST4 UNITED ST2 1996 12451867 +UNITED ST1 UNITED ST2 1996 10082536 +UNITED ST5 UNITED ST2 1996 8544233 +UNITED ST7 UNITED ST2 1996 8390470 +UNITED ST9 UNITED ST2 1996 8108035 +UNITED ST8 UNITED ST2 1996 7028334 +UNITED ST0 UNITED ST2 1996 6907199 +UNITED ST3 UNITED ST2 1996 4311800 +UNITED ST5 UNITED ST0 1997 121457766 +UNITED ST0 UNITED ST9 1997 119856268 +UNITED ST5 UNITED ST1 1997 106900161 +UNITED ST9 UNITED ST9 1997 106222795 +UNITED ST5 UNITED ST4 1997 103432927 +UNITED ST7 UNITED ST1 1997 101326940 +UNITED ST5 UNITED ST9 1997 97392640 +UNITED ST8 UNITED ST0 1997 89434845 +UNITED ST1 UNITED ST3 1997 85175638 +UNITED ST4 UNITED ST1 1997 84611188 +UNITED ST4 UNITED ST7 1997 84606996 +UNITED ST1 UNITED ST4 1997 83092569 +UNITED ST0 UNITED ST1 1997 80167139 +UNITED ST4 UNITED ST9 1997 79697927 +UNITED ST1 UNITED ST0 1997 77667866 +UNITED ST2 UNITED ST0 1997 77591470 +UNITED ST1 UNITED ST7 1997 77045722 +UNITED ST2 UNITED ST7 1997 76483802 +UNITED ST1 UNITED ST1 1997 76340444 +UNITED ST0 UNITED ST7 1997 72577490 +UNITED ST4 UNITED ST0 1997 69875804 +UNITED ST6 UNITED ST3 1997 69725506 +UNITED ST2 UNITED ST9 1997 69543984 +UNITED ST7 UNITED ST0 1997 69354312 +UNITED ST9 UNITED ST7 1997 69052630 +UNITED ST8 UNITED ST3 1997 66918394 +UNITED ST0 UNITED ST0 1997 66308516 +UNITED ST9 UNITED ST0 1997 66066791 +UNITED ST2 UNITED ST3 1997 65429453 +UNITED ST2 UNITED ST1 1997 65396135 +UNITED ST4 UNITED ST4 1997 62532346 +UNITED ST9 UNITED ST1 1997 62418406 +UNITED ST3 UNITED ST7 1997 61095300 +UNITED ST7 UNITED ST4 1997 60123494 +UNITED ST4 UNITED ST3 1997 59634153 +UNITED ST3 UNITED ST4 1997 58356141 +UNITED ST6 UNITED ST9 1997 58231263 +UNITED ST6 UNITED ST4 1997 57215022 +UNITED ST2 UNITED ST4 1997 57012611 +UNITED ST5 UNITED ST3 1997 56789932 +UNITED ST9 UNITED ST5 1997 56091920 +UNITED ST6 UNITED ST0 1997 55824919 +UNITED ST6 UNITED ST1 1997 55117709 +UNITED ST7 UNITED ST9 1997 55039970 +UNITED ST5 UNITED ST7 1997 54534315 +UNITED ST3 UNITED ST1 1997 54150683 +UNITED ST3 UNITED ST0 1997 54050172 +UNITED ST3 UNITED ST9 1997 51370308 +UNITED ST1 UNITED ST5 1997 51052110 +UNITED ST6 UNITED ST7 1997 49795849 +UNITED ST5 UNITED ST6 1997 49030016 +UNITED ST1 UNITED ST9 1997 48944111 +UNITED ST3 UNITED ST3 1997 48385233 +UNITED ST0 UNITED ST5 1997 47988418 +UNITED ST2 UNITED ST8 1997 46391909 +UNITED ST4 UNITED ST6 1997 46236849 +UNITED ST9 UNITED ST3 1997 46121724 +UNITED ST8 UNITED ST9 1997 45852796 +UNITED ST7 UNITED ST8 1997 45476266 +UNITED ST8 UNITED ST1 1997 43284139 +UNITED ST7 UNITED ST7 1997 42566603 +UNITED ST0 UNITED ST4 1997 41193803 +UNITED ST8 UNITED ST7 1997 40878429 +UNITED ST8 UNITED ST8 1997 40749400 +UNITED ST2 UNITED ST5 1997 40662886 +UNITED ST0 UNITED ST3 1997 40078508 +UNITED ST6 UNITED ST8 1997 37988445 +UNITED ST9 UNITED ST6 1997 37596882 +UNITED ST6 UNITED ST5 1997 37571788 +UNITED ST2 UNITED ST2 1997 37034333 +UNITED ST7 UNITED ST5 1997 36889736 +UNITED ST8 UNITED ST4 1997 36600584 +UNITED ST3 UNITED ST6 1997 36075844 +UNITED ST7 UNITED ST3 1997 35168922 +UNITED ST2 UNITED ST6 1997 34274467 +UNITED ST5 UNITED ST8 1997 32910946 +UNITED ST3 UNITED ST5 1997 32811963 +UNITED ST4 UNITED ST8 1997 32599736 +UNITED ST5 UNITED ST5 1997 32568888 +UNITED ST4 UNITED ST5 1997 31995868 +UNITED ST9 UNITED ST4 1997 31765478 +UNITED ST7 UNITED ST2 1997 31409590 +UNITED ST9 UNITED ST8 1997 31272385 +UNITED ST0 UNITED ST8 1997 30241370 +UNITED ST3 UNITED ST8 1997 29102220 +UNITED ST7 UNITED ST6 1997 27539931 +UNITED ST8 UNITED ST5 1997 26673743 +UNITED ST6 UNITED ST6 1997 24983291 +UNITED ST0 UNITED ST2 1997 23584229 +UNITED ST0 UNITED ST6 1997 22924046 +UNITED ST1 UNITED ST2 1997 20599130 +UNITED ST8 UNITED ST6 1997 20073000 +UNITED ST1 UNITED ST6 1997 18205326 +UNITED ST1 UNITED ST8 1997 15375258 +UNITED ST6 UNITED ST2 1997 14746680 +UNITED ST5 UNITED ST2 1997 13863438 +UNITED ST8 UNITED ST2 1997 12053706 +UNITED ST3 UNITED ST2 1997 7891878 +UNITED ST4 UNITED ST2 1997 2848157 +UNITED ST9 UNITED ST2 1997 2661652 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.3.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.3.out new file mode 100644 index 00000000000000..7a8708105dcc21 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.3.out @@ -0,0 +1,27 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q3.3 -- +UNITED KI1 UNITED KI1 1992 46735995 +UNITED KI5 UNITED KI1 1992 36277055 +UNITED KI5 UNITED KI5 1992 25355267 +UNITED KI1 UNITED KI5 1992 21917846 +UNITED KI5 UNITED KI1 1993 61017607 +UNITED KI1 UNITED KI1 1993 45669535 +UNITED KI5 UNITED KI5 1993 34099392 +UNITED KI1 UNITED KI5 1993 21444206 +UNITED KI5 UNITED KI1 1994 36282163 +UNITED KI1 UNITED KI1 1994 34868441 +UNITED KI5 UNITED KI5 1994 34507284 +UNITED KI1 UNITED KI5 1994 21221780 +UNITED KI5 UNITED KI1 1995 82955896 +UNITED KI1 UNITED KI1 1995 35881186 +UNITED KI5 UNITED KI5 1995 20539805 +UNITED KI1 UNITED KI5 1995 17176510 +UNITED KI5 UNITED KI1 1996 65767049 +UNITED KI1 UNITED KI1 1996 59923037 +UNITED KI5 UNITED KI5 1996 46077342 +UNITED KI1 UNITED KI5 1996 13700254 +UNITED KI1 UNITED KI1 1997 70343133 +UNITED KI5 UNITED KI1 1997 64978359 +UNITED KI5 UNITED KI5 1997 27332027 +UNITED KI1 UNITED KI5 1997 16410668 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.4.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.4.out new file mode 100644 index 00000000000000..f46fd55119065a --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q3.4.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q3.4 -- +UNITED KI5 UNITED KI1 1997 9117846 +UNITED KI5 UNITED KI5 1997 6203801 +UNITED KI1 UNITED KI5 1997 1870070 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.1.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.1.out new file mode 100644 index 00000000000000..9e7504904245dd --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.1.out @@ -0,0 +1,38 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q4.1 -- +1992 ARGENTINA 9658964469 +1992 BRAZIL 9226983055 +1992 CANADA 9643176787 +1992 PERU 9410676597 +1992 UNITED STATES 9849427653 +1993 ARGENTINA 9976332853 +1993 BRAZIL 9468799229 +1993 CANADA 9897302420 +1993 PERU 9309445836 +1993 UNITED STATES 10003985086 +1994 ARGENTINA 9940305215 +1994 BRAZIL 9348651677 +1994 CANADA 9582647596 +1994 PERU 9295265013 +1994 UNITED STATES 9519880425 +1995 ARGENTINA 10143841380 +1995 BRAZIL 9156077350 +1995 CANADA 9562612160 +1995 PERU 9778087211 +1995 UNITED STATES 9310565244 +1996 ARGENTINA 10001927895 +1996 BRAZIL 9168485151 +1996 CANADA 10061604203 +1996 PERU 9355135674 +1996 UNITED STATES 9769712174 +1997 ARGENTINA 9854560261 +1997 BRAZIL 9121571047 +1997 CANADA 10097371778 +1997 PERU 9315525917 +1997 UNITED STATES 10506723879 +1998 ARGENTINA 5834240407 +1998 BRAZIL 5356398095 +1998 CANADA 5423211196 +1998 PERU 5726185970 +1998 UNITED STATES 6009462019 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.2.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.2.out new file mode 100644 index 00000000000000..65b6d696df521b --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.2.out @@ -0,0 +1,103 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q4.2 -- +1997 ARGENTINA MFGR#11 907071566 +1997 ARGENTINA MFGR#12 924115562 +1997 ARGENTINA MFGR#13 972861821 +1997 ARGENTINA MFGR#14 975410345 +1997 ARGENTINA MFGR#15 938867375 +1997 ARGENTINA MFGR#21 1014782574 +1997 ARGENTINA MFGR#22 873016783 +1997 ARGENTINA MFGR#23 1030357302 +1997 ARGENTINA MFGR#24 893460579 +1997 ARGENTINA MFGR#25 1006311403 +1997 BRAZIL MFGR#11 1073219328 +1997 BRAZIL MFGR#12 989858833 +1997 BRAZIL MFGR#13 1128480379 +1997 BRAZIL MFGR#14 1194256722 +1997 BRAZIL MFGR#15 1094419124 +1997 BRAZIL MFGR#21 910026832 +1997 BRAZIL MFGR#22 993142048 +1997 BRAZIL MFGR#23 1107672874 +1997 BRAZIL MFGR#24 1058013649 +1997 BRAZIL MFGR#25 994733764 +1997 CANADA MFGR#11 854725020 +1997 CANADA MFGR#12 938718164 +1997 CANADA MFGR#13 959265890 +1997 CANADA MFGR#14 1002812450 +1997 CANADA MFGR#15 848183013 +1997 CANADA MFGR#21 999805272 +1997 CANADA MFGR#22 778419763 +1997 CANADA MFGR#23 928359645 +1997 CANADA MFGR#24 849895128 +1997 CANADA MFGR#25 904587965 +1997 PERU MFGR#11 1100242877 +1997 PERU MFGR#12 994365350 +1997 PERU MFGR#13 847486105 +1997 PERU MFGR#14 947769683 +1997 PERU MFGR#15 999395678 +1997 PERU MFGR#21 867923394 +1997 PERU MFGR#22 988747459 +1997 PERU MFGR#23 1066645086 +1997 PERU MFGR#24 935665725 +1997 PERU MFGR#25 981454129 +1997 UNITED STATES MFGR#11 1046706048 +1997 UNITED STATES MFGR#12 909213709 +1997 UNITED STATES MFGR#13 1096278906 +1997 UNITED STATES MFGR#14 934282111 +1997 UNITED STATES MFGR#15 962760843 +1997 UNITED STATES MFGR#21 1000676474 +1997 UNITED STATES MFGR#22 1076947615 +1997 UNITED STATES MFGR#23 937288102 +1997 UNITED STATES MFGR#24 1003386363 +1997 UNITED STATES MFGR#25 1053666052 +1998 ARGENTINA MFGR#11 567612227 +1998 ARGENTINA MFGR#12 527025042 +1998 ARGENTINA MFGR#13 582791792 +1998 ARGENTINA MFGR#14 523726368 +1998 ARGENTINA MFGR#15 522078267 +1998 ARGENTINA MFGR#21 504712685 +1998 ARGENTINA MFGR#22 506061736 +1998 ARGENTINA MFGR#23 560479801 +1998 ARGENTINA MFGR#24 524579118 +1998 ARGENTINA MFGR#25 547840211 +1998 BRAZIL MFGR#11 638578488 +1998 BRAZIL MFGR#12 646312681 +1998 BRAZIL MFGR#13 655161772 +1998 BRAZIL MFGR#14 552676170 +1998 BRAZIL MFGR#15 663812709 +1998 BRAZIL MFGR#21 668822448 +1998 BRAZIL MFGR#22 591791918 +1998 BRAZIL MFGR#23 690648877 +1998 BRAZIL MFGR#24 562362220 +1998 BRAZIL MFGR#25 704182376 +1998 CANADA MFGR#11 509086125 +1998 CANADA MFGR#12 488089772 +1998 CANADA MFGR#13 486533297 +1998 CANADA MFGR#14 527837227 +1998 CANADA MFGR#15 535869299 +1998 CANADA MFGR#21 455868651 +1998 CANADA MFGR#22 594277308 +1998 CANADA MFGR#23 574125070 +1998 CANADA MFGR#24 508530424 +1998 CANADA MFGR#25 547757992 +1998 PERU MFGR#11 567838547 +1998 PERU MFGR#12 540544757 +1998 PERU MFGR#13 591331883 +1998 PERU MFGR#14 481335064 +1998 PERU MFGR#15 570246138 +1998 PERU MFGR#21 533733330 +1998 PERU MFGR#22 527790656 +1998 PERU MFGR#23 636393221 +1998 PERU MFGR#24 589075262 +1998 PERU MFGR#25 543251115 +1998 UNITED STATES MFGR#11 556276232 +1998 UNITED STATES MFGR#12 612385982 +1998 UNITED STATES MFGR#13 622413927 +1998 UNITED STATES MFGR#14 555006887 +1998 UNITED STATES MFGR#15 525119569 +1998 UNITED STATES MFGR#21 563406836 +1998 UNITED STATES MFGR#22 580478735 +1998 UNITED STATES MFGR#23 656080465 +1998 UNITED STATES MFGR#24 538445058 +1998 UNITED STATES MFGR#25 589111952 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.3.out b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.3.out new file mode 100644 index 00000000000000..16b854fc0a88c9 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/flat_q4.3.out @@ -0,0 +1,697 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q4.3 -- +1997 UNITED ST0 MFGR#141 15456288 +1997 UNITED ST0 MFGR#1410 32961113 +1997 UNITED ST0 MFGR#1411 36166400 +1997 UNITED ST0 MFGR#1412 18237451 +1997 UNITED ST0 MFGR#1413 40752813 +1997 UNITED ST0 MFGR#1414 24237588 +1997 UNITED ST0 MFGR#1415 6419001 +1997 UNITED ST0 MFGR#1416 15639876 +1997 UNITED ST0 MFGR#1417 15476211 +1997 UNITED ST0 MFGR#1418 8169956 +1997 UNITED ST0 MFGR#1419 18217252 +1997 UNITED ST0 MFGR#142 29361222 +1997 UNITED ST0 MFGR#1420 34587603 +1997 UNITED ST0 MFGR#1421 28560921 +1997 UNITED ST0 MFGR#1422 25154181 +1997 UNITED ST0 MFGR#1423 18144008 +1997 UNITED ST0 MFGR#1424 28064267 +1997 UNITED ST0 MFGR#1425 15255845 +1997 UNITED ST0 MFGR#1426 5610353 +1997 UNITED ST0 MFGR#1427 5750693 +1997 UNITED ST0 MFGR#1428 14081707 +1997 UNITED ST0 MFGR#1429 9024512 +1997 UNITED ST0 MFGR#143 5325193 +1997 UNITED ST0 MFGR#1430 16961739 +1997 UNITED ST0 MFGR#1431 19015253 +1997 UNITED ST0 MFGR#1432 6324917 +1997 UNITED ST0 MFGR#1433 6734461 +1997 UNITED ST0 MFGR#1434 19206393 +1997 UNITED ST0 MFGR#1435 679925 +1997 UNITED ST0 MFGR#1436 13834012 +1997 UNITED ST0 MFGR#1437 23575407 +1997 UNITED ST0 MFGR#1438 24756074 +1997 UNITED ST0 MFGR#1439 19644072 +1997 UNITED ST0 MFGR#144 20285157 +1997 UNITED ST0 MFGR#1440 14068558 +1997 UNITED ST0 MFGR#145 22646138 +1997 UNITED ST0 MFGR#146 16381159 +1997 UNITED ST0 MFGR#147 12267827 +1997 UNITED ST0 MFGR#148 30968614 +1997 UNITED ST0 MFGR#149 8640669 +1997 UNITED ST1 MFGR#141 10152087 +1997 UNITED ST1 MFGR#1410 32583300 +1997 UNITED ST1 MFGR#1411 15690352 +1997 UNITED ST1 MFGR#1412 16671403 +1997 UNITED ST1 MFGR#1413 4942775 +1997 UNITED ST1 MFGR#1414 9391597 +1997 UNITED ST1 MFGR#1415 26937296 +1997 UNITED ST1 MFGR#1416 25782871 +1997 UNITED ST1 MFGR#1417 23349641 +1997 UNITED ST1 MFGR#1418 10934620 +1997 UNITED ST1 MFGR#1419 22673030 +1997 UNITED ST1 MFGR#142 14048276 +1997 UNITED ST1 MFGR#1420 18100795 +1997 UNITED ST1 MFGR#1421 25405678 +1997 UNITED ST1 MFGR#1423 5713800 +1997 UNITED ST1 MFGR#1424 4131923 +1997 UNITED ST1 MFGR#1425 17190425 +1997 UNITED ST1 MFGR#1426 22314044 +1997 UNITED ST1 MFGR#1427 13635057 +1997 UNITED ST1 MFGR#1428 17735811 +1997 UNITED ST1 MFGR#1429 28740911 +1997 UNITED ST1 MFGR#143 27418033 +1997 UNITED ST1 MFGR#1430 18655292 +1997 UNITED ST1 MFGR#1431 11074399 +1997 UNITED ST1 MFGR#1432 7490170 +1997 UNITED ST1 MFGR#1433 28997266 +1997 UNITED ST1 MFGR#1434 4612211 +1997 UNITED ST1 MFGR#1435 16327124 +1997 UNITED ST1 MFGR#1436 17409665 +1997 UNITED ST1 MFGR#1437 33391635 +1997 UNITED ST1 MFGR#1438 37321134 +1997 UNITED ST1 MFGR#1439 10279941 +1997 UNITED ST1 MFGR#144 12304723 +1997 UNITED ST1 MFGR#1440 28921230 +1997 UNITED ST1 MFGR#145 17933684 +1997 UNITED ST1 MFGR#146 8709103 +1997 UNITED ST1 MFGR#147 20145983 +1997 UNITED ST1 MFGR#148 16940400 +1997 UNITED ST1 MFGR#149 10284949 +1997 UNITED ST2 MFGR#1412 6328240 +1997 UNITED ST2 MFGR#1415 5022682 +1997 UNITED ST2 MFGR#1416 4102214 +1997 UNITED ST2 MFGR#1417 5284444 +1997 UNITED ST2 MFGR#1418 2649908 +1997 UNITED ST2 MFGR#1419 4664673 +1997 UNITED ST2 MFGR#142 10784835 +1997 UNITED ST2 MFGR#1423 4095190 +1997 UNITED ST2 MFGR#1424 5376355 +1997 UNITED ST2 MFGR#1425 3949957 +1997 UNITED ST2 MFGR#143 2859829 +1997 UNITED ST2 MFGR#1430 4618266 +1997 UNITED ST2 MFGR#1432 5183060 +1997 UNITED ST2 MFGR#1433 10531371 +1997 UNITED ST2 MFGR#1434 77646 +1997 UNITED ST2 MFGR#1436 6538508 +1997 UNITED ST2 MFGR#1438 12324747 +1997 UNITED ST2 MFGR#144 2002870 +1997 UNITED ST2 MFGR#1440 12247229 +1997 UNITED ST2 MFGR#145 202685 +1997 UNITED ST2 MFGR#148 5998932 +1997 UNITED ST3 MFGR#141 12866083 +1997 UNITED ST3 MFGR#1410 31051703 +1997 UNITED ST3 MFGR#1411 14099543 +1997 UNITED ST3 MFGR#1412 4970551 +1997 UNITED ST3 MFGR#1413 26129762 +1997 UNITED ST3 MFGR#1414 22364787 +1997 UNITED ST3 MFGR#1415 27883428 +1997 UNITED ST3 MFGR#1416 25289630 +1997 UNITED ST3 MFGR#1417 5524810 +1997 UNITED ST3 MFGR#1419 3616608 +1997 UNITED ST3 MFGR#142 10776059 +1997 UNITED ST3 MFGR#1420 12958851 +1997 UNITED ST3 MFGR#1421 5815494 +1997 UNITED ST3 MFGR#1422 17694270 +1997 UNITED ST3 MFGR#1423 10285142 +1997 UNITED ST3 MFGR#1424 21555317 +1997 UNITED ST3 MFGR#1425 5214517 +1997 UNITED ST3 MFGR#1426 16810700 +1997 UNITED ST3 MFGR#1427 10086430 +1997 UNITED ST3 MFGR#1428 7733716 +1997 UNITED ST3 MFGR#1429 11922430 +1997 UNITED ST3 MFGR#1430 20105292 +1997 UNITED ST3 MFGR#1431 14716979 +1997 UNITED ST3 MFGR#1432 15779247 +1997 UNITED ST3 MFGR#1433 15308773 +1997 UNITED ST3 MFGR#1434 8515037 +1997 UNITED ST3 MFGR#1435 25486899 +1997 UNITED ST3 MFGR#1436 12739448 +1997 UNITED ST3 MFGR#1437 11249180 +1997 UNITED ST3 MFGR#1438 4156824 +1997 UNITED ST3 MFGR#1439 18358277 +1997 UNITED ST3 MFGR#144 10097718 +1997 UNITED ST3 MFGR#1440 22920494 +1997 UNITED ST3 MFGR#145 20285859 +1997 UNITED ST3 MFGR#146 18406423 +1997 UNITED ST3 MFGR#147 17295990 +1997 UNITED ST3 MFGR#148 26127145 +1997 UNITED ST3 MFGR#149 19895538 +1997 UNITED ST4 MFGR#141 8135477 +1997 UNITED ST4 MFGR#1410 5503247 +1997 UNITED ST4 MFGR#1411 6096435 +1997 UNITED ST4 MFGR#1412 19976043 +1997 UNITED ST4 MFGR#1413 22581282 +1997 UNITED ST4 MFGR#1414 10101379 +1997 UNITED ST4 MFGR#1415 8877444 +1997 UNITED ST4 MFGR#1416 17943398 +1997 UNITED ST4 MFGR#1417 9224501 +1997 UNITED ST4 MFGR#1418 16255176 +1997 UNITED ST4 MFGR#1419 683069 +1997 UNITED ST4 MFGR#142 23512944 +1997 UNITED ST4 MFGR#1420 27954262 +1997 UNITED ST4 MFGR#1421 54209185 +1997 UNITED ST4 MFGR#1422 1173957 +1997 UNITED ST4 MFGR#1423 20410710 +1997 UNITED ST4 MFGR#1424 2889270 +1997 UNITED ST4 MFGR#1425 13361796 +1997 UNITED ST4 MFGR#1426 18678819 +1997 UNITED ST4 MFGR#1427 18206066 +1997 UNITED ST4 MFGR#1428 18153974 +1997 UNITED ST4 MFGR#1429 12545076 +1997 UNITED ST4 MFGR#143 6158834 +1997 UNITED ST4 MFGR#1430 16128370 +1997 UNITED ST4 MFGR#1431 11153717 +1997 UNITED ST4 MFGR#1432 7745636 +1997 UNITED ST4 MFGR#1433 15983634 +1997 UNITED ST4 MFGR#1434 3662246 +1997 UNITED ST4 MFGR#1435 3060180 +1997 UNITED ST4 MFGR#1436 11524592 +1997 UNITED ST4 MFGR#1437 36824174 +1997 UNITED ST4 MFGR#1438 8226666 +1997 UNITED ST4 MFGR#1439 9872470 +1997 UNITED ST4 MFGR#1440 401207 +1997 UNITED ST4 MFGR#145 11698312 +1997 UNITED ST4 MFGR#146 18259956 +1997 UNITED ST4 MFGR#147 10256110 +1997 UNITED ST4 MFGR#148 15892628 +1997 UNITED ST4 MFGR#149 4322491 +1997 UNITED ST5 MFGR#141 8534275 +1997 UNITED ST5 MFGR#1411 2427261 +1997 UNITED ST5 MFGR#1412 9032422 +1997 UNITED ST5 MFGR#1413 6343588 +1997 UNITED ST5 MFGR#1414 2961025 +1997 UNITED ST5 MFGR#1415 21186568 +1997 UNITED ST5 MFGR#1416 9354080 +1997 UNITED ST5 MFGR#1417 11786246 +1997 UNITED ST5 MFGR#1418 15297245 +1997 UNITED ST5 MFGR#1420 15787608 +1997 UNITED ST5 MFGR#1421 17858301 +1997 UNITED ST5 MFGR#1422 14080987 +1997 UNITED ST5 MFGR#1423 12535186 +1997 UNITED ST5 MFGR#1424 3221960 +1997 UNITED ST5 MFGR#1425 8950317 +1997 UNITED ST5 MFGR#1426 13161486 +1997 UNITED ST5 MFGR#1427 6149737 +1997 UNITED ST5 MFGR#1428 6796520 +1997 UNITED ST5 MFGR#1429 26026300 +1997 UNITED ST5 MFGR#143 6712572 +1997 UNITED ST5 MFGR#1430 16092131 +1997 UNITED ST5 MFGR#1431 3425291 +1997 UNITED ST5 MFGR#1432 14630175 +1997 UNITED ST5 MFGR#1433 4967585 +1997 UNITED ST5 MFGR#1434 6716261 +1997 UNITED ST5 MFGR#1435 6290790 +1997 UNITED ST5 MFGR#1436 6787699 +1997 UNITED ST5 MFGR#1437 6045168 +1997 UNITED ST5 MFGR#1438 1082877 +1997 UNITED ST5 MFGR#1439 5268611 +1997 UNITED ST5 MFGR#144 8900840 +1997 UNITED ST5 MFGR#1440 2644336 +1997 UNITED ST5 MFGR#145 18054844 +1997 UNITED ST5 MFGR#146 4893668 +1997 UNITED ST5 MFGR#147 9615891 +1997 UNITED ST5 MFGR#148 1426124 +1997 UNITED ST5 MFGR#149 7320726 +1997 UNITED ST6 MFGR#141 10051337 +1997 UNITED ST6 MFGR#1410 23352963 +1997 UNITED ST6 MFGR#1411 1537303 +1997 UNITED ST6 MFGR#1412 4332646 +1997 UNITED ST6 MFGR#1413 14686752 +1997 UNITED ST6 MFGR#1414 7226409 +1997 UNITED ST6 MFGR#1415 15062686 +1997 UNITED ST6 MFGR#1416 8606480 +1997 UNITED ST6 MFGR#1417 7662761 +1997 UNITED ST6 MFGR#1418 1810860 +1997 UNITED ST6 MFGR#1419 20179568 +1997 UNITED ST6 MFGR#142 15506072 +1997 UNITED ST6 MFGR#1420 3844271 +1997 UNITED ST6 MFGR#1421 10608709 +1997 UNITED ST6 MFGR#1422 16083699 +1997 UNITED ST6 MFGR#1423 6789571 +1997 UNITED ST6 MFGR#1424 17631950 +1997 UNITED ST6 MFGR#1425 8477221 +1997 UNITED ST6 MFGR#1426 16847029 +1997 UNITED ST6 MFGR#1428 17314796 +1997 UNITED ST6 MFGR#1429 19175871 +1997 UNITED ST6 MFGR#143 8775848 +1997 UNITED ST6 MFGR#1431 5319063 +1997 UNITED ST6 MFGR#1432 7875606 +1997 UNITED ST6 MFGR#1433 9305264 +1997 UNITED ST6 MFGR#1434 8050404 +1997 UNITED ST6 MFGR#1435 17585836 +1997 UNITED ST6 MFGR#1436 13745758 +1997 UNITED ST6 MFGR#1438 14536324 +1997 UNITED ST6 MFGR#1439 11451329 +1997 UNITED ST6 MFGR#144 10794837 +1997 UNITED ST6 MFGR#1440 5719238 +1997 UNITED ST6 MFGR#145 25449058 +1997 UNITED ST6 MFGR#146 17821996 +1997 UNITED ST6 MFGR#147 18922333 +1997 UNITED ST6 MFGR#148 10825718 +1997 UNITED ST6 MFGR#149 14950868 +1997 UNITED ST7 MFGR#141 8291476 +1997 UNITED ST7 MFGR#1410 4622953 +1997 UNITED ST7 MFGR#1411 14885507 +1997 UNITED ST7 MFGR#1412 19011375 +1997 UNITED ST7 MFGR#1413 31178370 +1997 UNITED ST7 MFGR#1414 16429613 +1997 UNITED ST7 MFGR#1415 3091832 +1997 UNITED ST7 MFGR#1416 17008243 +1997 UNITED ST7 MFGR#1417 15609283 +1997 UNITED ST7 MFGR#1418 5059394 +1997 UNITED ST7 MFGR#1419 7234721 +1997 UNITED ST7 MFGR#142 6898938 +1997 UNITED ST7 MFGR#1420 6426590 +1997 UNITED ST7 MFGR#1421 6043221 +1997 UNITED ST7 MFGR#1422 13273021 +1997 UNITED ST7 MFGR#1423 30537249 +1997 UNITED ST7 MFGR#1424 9224296 +1997 UNITED ST7 MFGR#1425 12799069 +1997 UNITED ST7 MFGR#1426 30277891 +1997 UNITED ST7 MFGR#1427 8331493 +1997 UNITED ST7 MFGR#1428 16205423 +1997 UNITED ST7 MFGR#1429 14396627 +1997 UNITED ST7 MFGR#143 10209623 +1997 UNITED ST7 MFGR#1430 16556758 +1997 UNITED ST7 MFGR#1431 15775327 +1997 UNITED ST7 MFGR#1432 24602989 +1997 UNITED ST7 MFGR#1433 22927717 +1997 UNITED ST7 MFGR#1434 9486462 +1997 UNITED ST7 MFGR#1435 14254290 +1997 UNITED ST7 MFGR#1436 8645365 +1997 UNITED ST7 MFGR#1437 19026166 +1997 UNITED ST7 MFGR#1438 11291550 +1997 UNITED ST7 MFGR#1439 9590292 +1997 UNITED ST7 MFGR#144 14110404 +1997 UNITED ST7 MFGR#1440 21877355 +1997 UNITED ST7 MFGR#145 8246717 +1997 UNITED ST7 MFGR#146 9785781 +1997 UNITED ST7 MFGR#147 13412691 +1997 UNITED ST7 MFGR#148 14235939 +1997 UNITED ST7 MFGR#149 23450309 +1997 UNITED ST8 MFGR#141 7323416 +1997 UNITED ST8 MFGR#1410 7191961 +1997 UNITED ST8 MFGR#1411 2425849 +1997 UNITED ST8 MFGR#1412 26725641 +1997 UNITED ST8 MFGR#1413 7915896 +1997 UNITED ST8 MFGR#1414 14359044 +1997 UNITED ST8 MFGR#1415 15834927 +1997 UNITED ST8 MFGR#1416 7464470 +1997 UNITED ST8 MFGR#1417 15274496 +1997 UNITED ST8 MFGR#1418 3261391 +1997 UNITED ST8 MFGR#1419 8020386 +1997 UNITED ST8 MFGR#142 5985550 +1997 UNITED ST8 MFGR#1420 9441461 +1997 UNITED ST8 MFGR#1421 2998735 +1997 UNITED ST8 MFGR#1423 1565415 +1997 UNITED ST8 MFGR#1424 5613094 +1997 UNITED ST8 MFGR#1425 7186734 +1997 UNITED ST8 MFGR#1426 3153117 +1997 UNITED ST8 MFGR#1427 11303043 +1997 UNITED ST8 MFGR#1428 3564003 +1997 UNITED ST8 MFGR#143 6357477 +1997 UNITED ST8 MFGR#1430 6633950 +1997 UNITED ST8 MFGR#1431 867285 +1997 UNITED ST8 MFGR#1432 9962884 +1997 UNITED ST8 MFGR#1433 13708011 +1997 UNITED ST8 MFGR#1434 7357067 +1997 UNITED ST8 MFGR#1435 11993453 +1997 UNITED ST8 MFGR#1436 7231938 +1997 UNITED ST8 MFGR#1437 18632056 +1997 UNITED ST8 MFGR#1438 3609384 +1997 UNITED ST8 MFGR#144 4822734 +1997 UNITED ST8 MFGR#1440 2521917 +1997 UNITED ST8 MFGR#145 11683385 +1997 UNITED ST8 MFGR#146 3350415 +1997 UNITED ST8 MFGR#147 8425393 +1997 UNITED ST8 MFGR#148 15005341 +1997 UNITED ST8 MFGR#149 9018101 +1997 UNITED ST9 MFGR#141 21838850 +1997 UNITED ST9 MFGR#1410 8455387 +1997 UNITED ST9 MFGR#1411 27984689 +1997 UNITED ST9 MFGR#1412 26491401 +1997 UNITED ST9 MFGR#1413 30237705 +1997 UNITED ST9 MFGR#1414 8957476 +1997 UNITED ST9 MFGR#1415 35391583 +1997 UNITED ST9 MFGR#1416 8923782 +1997 UNITED ST9 MFGR#1417 3210984 +1997 UNITED ST9 MFGR#1418 12102607 +1997 UNITED ST9 MFGR#1419 14468627 +1997 UNITED ST9 MFGR#142 33363582 +1997 UNITED ST9 MFGR#1420 13139731 +1997 UNITED ST9 MFGR#1421 8410650 +1997 UNITED ST9 MFGR#1422 11268410 +1997 UNITED ST9 MFGR#1423 2846614 +1997 UNITED ST9 MFGR#1424 2305872 +1997 UNITED ST9 MFGR#1425 4896060 +1997 UNITED ST9 MFGR#1426 31544772 +1997 UNITED ST9 MFGR#1427 4113085 +1997 UNITED ST9 MFGR#1428 11137081 +1997 UNITED ST9 MFGR#1429 10633230 +1997 UNITED ST9 MFGR#143 11438203 +1997 UNITED ST9 MFGR#1430 2727389 +1997 UNITED ST9 MFGR#1431 10548538 +1997 UNITED ST9 MFGR#1433 26564495 +1997 UNITED ST9 MFGR#1434 11948206 +1997 UNITED ST9 MFGR#1435 2201909 +1997 UNITED ST9 MFGR#1436 10347631 +1997 UNITED ST9 MFGR#1437 40051690 +1997 UNITED ST9 MFGR#1438 2662945 +1997 UNITED ST9 MFGR#1439 18011455 +1997 UNITED ST9 MFGR#144 10202784 +1997 UNITED ST9 MFGR#1440 11123196 +1997 UNITED ST9 MFGR#145 11326624 +1997 UNITED ST9 MFGR#146 27825456 +1997 UNITED ST9 MFGR#147 17579502 +1997 UNITED ST9 MFGR#148 17001116 +1998 UNITED ST0 MFGR#141 12484786 +1998 UNITED ST0 MFGR#1410 5225999 +1998 UNITED ST0 MFGR#1411 2633395 +1998 UNITED ST0 MFGR#1412 9479536 +1998 UNITED ST0 MFGR#1413 18468711 +1998 UNITED ST0 MFGR#1414 19440601 +1998 UNITED ST0 MFGR#1415 16894643 +1998 UNITED ST0 MFGR#1416 7520117 +1998 UNITED ST0 MFGR#1417 464576 +1998 UNITED ST0 MFGR#1418 5171747 +1998 UNITED ST0 MFGR#1419 5970985 +1998 UNITED ST0 MFGR#142 23830409 +1998 UNITED ST0 MFGR#1420 2568835 +1998 UNITED ST0 MFGR#1421 6096442 +1998 UNITED ST0 MFGR#1422 10099456 +1998 UNITED ST0 MFGR#1423 24617746 +1998 UNITED ST0 MFGR#1424 22853888 +1998 UNITED ST0 MFGR#1425 25622112 +1998 UNITED ST0 MFGR#1426 9630701 +1998 UNITED ST0 MFGR#1427 4952625 +1998 UNITED ST0 MFGR#1428 4675471 +1998 UNITED ST0 MFGR#1429 19713527 +1998 UNITED ST0 MFGR#143 4971265 +1998 UNITED ST0 MFGR#1430 13948705 +1998 UNITED ST0 MFGR#1431 2955181 +1998 UNITED ST0 MFGR#1432 17946759 +1998 UNITED ST0 MFGR#1433 23325027 +1998 UNITED ST0 MFGR#1434 8580738 +1998 UNITED ST0 MFGR#1435 6663914 +1998 UNITED ST0 MFGR#1436 13468582 +1998 UNITED ST0 MFGR#1437 9419979 +1998 UNITED ST0 MFGR#1438 16164522 +1998 UNITED ST0 MFGR#1439 8692866 +1998 UNITED ST0 MFGR#144 17338711 +1998 UNITED ST0 MFGR#1440 8513736 +1998 UNITED ST0 MFGR#145 6212569 +1998 UNITED ST0 MFGR#146 2702349 +1998 UNITED ST0 MFGR#149 8844540 +1998 UNITED ST1 MFGR#141 13449110 +1998 UNITED ST1 MFGR#1410 5224626 +1998 UNITED ST1 MFGR#1411 16046399 +1998 UNITED ST1 MFGR#1412 8424392 +1998 UNITED ST1 MFGR#1413 7634982 +1998 UNITED ST1 MFGR#1414 8711782 +1998 UNITED ST1 MFGR#1415 5991201 +1998 UNITED ST1 MFGR#1416 11059645 +1998 UNITED ST1 MFGR#1417 8108087 +1998 UNITED ST1 MFGR#1418 1710875 +1998 UNITED ST1 MFGR#1419 452298 +1998 UNITED ST1 MFGR#142 6186526 +1998 UNITED ST1 MFGR#1421 4766963 +1998 UNITED ST1 MFGR#1422 5556563 +1998 UNITED ST1 MFGR#1423 12110257 +1998 UNITED ST1 MFGR#1424 8924948 +1998 UNITED ST1 MFGR#1425 5684710 +1998 UNITED ST1 MFGR#1426 6844444 +1998 UNITED ST1 MFGR#1427 7625525 +1998 UNITED ST1 MFGR#1428 5227395 +1998 UNITED ST1 MFGR#1429 2855272 +1998 UNITED ST1 MFGR#143 6964811 +1998 UNITED ST1 MFGR#1430 16952380 +1998 UNITED ST1 MFGR#1431 7655883 +1998 UNITED ST1 MFGR#1432 2975960 +1998 UNITED ST1 MFGR#1433 10175594 +1998 UNITED ST1 MFGR#1434 26358848 +1998 UNITED ST1 MFGR#1435 11408623 +1998 UNITED ST1 MFGR#1436 4438242 +1998 UNITED ST1 MFGR#1437 10014906 +1998 UNITED ST1 MFGR#1438 18817171 +1998 UNITED ST1 MFGR#1439 2129973 +1998 UNITED ST1 MFGR#144 9512727 +1998 UNITED ST1 MFGR#1440 20185179 +1998 UNITED ST1 MFGR#145 823605 +1998 UNITED ST1 MFGR#146 9358026 +1998 UNITED ST1 MFGR#147 6552312 +1998 UNITED ST1 MFGR#148 9144461 +1998 UNITED ST1 MFGR#149 2149639 +1998 UNITED ST2 MFGR#1410 8044368 +1998 UNITED ST2 MFGR#1413 727343 +1998 UNITED ST2 MFGR#1414 251088 +1998 UNITED ST2 MFGR#1416 5224696 +1998 UNITED ST2 MFGR#1419 7917142 +1998 UNITED ST2 MFGR#1420 9848986 +1998 UNITED ST2 MFGR#1427 4816976 +1998 UNITED ST2 MFGR#1429 4196069 +1998 UNITED ST2 MFGR#143 10073044 +1998 UNITED ST2 MFGR#1431 877123 +1998 UNITED ST2 MFGR#1433 3740836 +1998 UNITED ST2 MFGR#1434 857691 +1998 UNITED ST2 MFGR#1435 2277870 +1998 UNITED ST2 MFGR#145 4906521 +1998 UNITED ST2 MFGR#149 842348 +1998 UNITED ST3 MFGR#141 4181582 +1998 UNITED ST3 MFGR#1410 3169761 +1998 UNITED ST3 MFGR#1412 6590440 +1998 UNITED ST3 MFGR#1413 2015369 +1998 UNITED ST3 MFGR#1414 4408466 +1998 UNITED ST3 MFGR#1415 3963826 +1998 UNITED ST3 MFGR#1416 3179540 +1998 UNITED ST3 MFGR#1417 11440999 +1998 UNITED ST3 MFGR#1418 8725751 +1998 UNITED ST3 MFGR#1419 10646990 +1998 UNITED ST3 MFGR#142 5056172 +1998 UNITED ST3 MFGR#1420 3583150 +1998 UNITED ST3 MFGR#1421 7085689 +1998 UNITED ST3 MFGR#1422 20858904 +1998 UNITED ST3 MFGR#1423 12825743 +1998 UNITED ST3 MFGR#1424 3533518 +1998 UNITED ST3 MFGR#1425 4938324 +1998 UNITED ST3 MFGR#1426 15164664 +1998 UNITED ST3 MFGR#1427 2619162 +1998 UNITED ST3 MFGR#1428 7723699 +1998 UNITED ST3 MFGR#143 24362893 +1998 UNITED ST3 MFGR#1430 7139650 +1998 UNITED ST3 MFGR#1431 10306394 +1998 UNITED ST3 MFGR#1432 14769004 +1998 UNITED ST3 MFGR#1433 9610545 +1998 UNITED ST3 MFGR#1435 6611760 +1998 UNITED ST3 MFGR#1436 941348 +1998 UNITED ST3 MFGR#1437 4802162 +1998 UNITED ST3 MFGR#1438 5079750 +1998 UNITED ST3 MFGR#1439 5905118 +1998 UNITED ST3 MFGR#144 5436839 +1998 UNITED ST3 MFGR#1440 9815517 +1998 UNITED ST3 MFGR#145 7580271 +1998 UNITED ST3 MFGR#146 14051404 +1998 UNITED ST3 MFGR#147 7786017 +1998 UNITED ST3 MFGR#149 1271813 +1998 UNITED ST4 MFGR#141 26703183 +1998 UNITED ST4 MFGR#1410 18094129 +1998 UNITED ST4 MFGR#1411 615999 +1998 UNITED ST4 MFGR#1412 13988709 +1998 UNITED ST4 MFGR#1413 10115647 +1998 UNITED ST4 MFGR#1414 1586551 +1998 UNITED ST4 MFGR#1415 14357244 +1998 UNITED ST4 MFGR#1416 9430036 +1998 UNITED ST4 MFGR#1417 17361773 +1998 UNITED ST4 MFGR#1418 10187338 +1998 UNITED ST4 MFGR#1419 7116094 +1998 UNITED ST4 MFGR#142 18454502 +1998 UNITED ST4 MFGR#1420 1621340 +1998 UNITED ST4 MFGR#1422 13861291 +1998 UNITED ST4 MFGR#1423 4208006 +1998 UNITED ST4 MFGR#1424 3242576 +1998 UNITED ST4 MFGR#1425 1470642 +1998 UNITED ST4 MFGR#1426 11987205 +1998 UNITED ST4 MFGR#1427 12975399 +1998 UNITED ST4 MFGR#1428 11756155 +1998 UNITED ST4 MFGR#1429 6956191 +1998 UNITED ST4 MFGR#143 11499476 +1998 UNITED ST4 MFGR#1430 4355096 +1998 UNITED ST4 MFGR#1431 20166612 +1998 UNITED ST4 MFGR#1432 8849789 +1998 UNITED ST4 MFGR#1433 10786037 +1998 UNITED ST4 MFGR#1435 13208151 +1998 UNITED ST4 MFGR#1436 4747808 +1998 UNITED ST4 MFGR#1437 12999351 +1998 UNITED ST4 MFGR#1438 17360787 +1998 UNITED ST4 MFGR#1439 2557841 +1998 UNITED ST4 MFGR#144 22030911 +1998 UNITED ST4 MFGR#145 5272280 +1998 UNITED ST4 MFGR#146 4307958 +1998 UNITED ST4 MFGR#147 7995050 +1998 UNITED ST4 MFGR#148 2348889 +1998 UNITED ST5 MFGR#141 8396190 +1998 UNITED ST5 MFGR#1410 17422889 +1998 UNITED ST5 MFGR#1411 764246 +1998 UNITED ST5 MFGR#1413 8051561 +1998 UNITED ST5 MFGR#1414 1500867 +1998 UNITED ST5 MFGR#1416 3874315 +1998 UNITED ST5 MFGR#1417 7550747 +1998 UNITED ST5 MFGR#1418 14241755 +1998 UNITED ST5 MFGR#1419 4893846 +1998 UNITED ST5 MFGR#142 5418299 +1998 UNITED ST5 MFGR#1420 3810727 +1998 UNITED ST5 MFGR#1421 3321336 +1998 UNITED ST5 MFGR#1423 5021100 +1998 UNITED ST5 MFGR#1424 6125645 +1998 UNITED ST5 MFGR#1426 880680 +1998 UNITED ST5 MFGR#1427 1284890 +1998 UNITED ST5 MFGR#1428 3813108 +1998 UNITED ST5 MFGR#1429 9429868 +1998 UNITED ST5 MFGR#143 15699981 +1998 UNITED ST5 MFGR#1430 7977654 +1998 UNITED ST5 MFGR#1431 5545969 +1998 UNITED ST5 MFGR#1432 1803434 +1998 UNITED ST5 MFGR#1435 12957056 +1998 UNITED ST5 MFGR#1436 20666755 +1998 UNITED ST5 MFGR#1437 9536733 +1998 UNITED ST5 MFGR#1439 11784214 +1998 UNITED ST5 MFGR#1440 613089 +1998 UNITED ST5 MFGR#145 8237944 +1998 UNITED ST5 MFGR#147 5370351 +1998 UNITED ST6 MFGR#141 6683977 +1998 UNITED ST6 MFGR#1411 11589040 +1998 UNITED ST6 MFGR#1412 4800954 +1998 UNITED ST6 MFGR#1416 5848432 +1998 UNITED ST6 MFGR#1417 1804324 +1998 UNITED ST6 MFGR#1418 4642727 +1998 UNITED ST6 MFGR#1419 10079621 +1998 UNITED ST6 MFGR#142 2533570 +1998 UNITED ST6 MFGR#1420 9016258 +1998 UNITED ST6 MFGR#1421 7636150 +1998 UNITED ST6 MFGR#1422 1272436 +1998 UNITED ST6 MFGR#1423 9600097 +1998 UNITED ST6 MFGR#1424 7036762 +1998 UNITED ST6 MFGR#1425 10867426 +1998 UNITED ST6 MFGR#1426 7033994 +1998 UNITED ST6 MFGR#1427 6481679 +1998 UNITED ST6 MFGR#1428 6269185 +1998 UNITED ST6 MFGR#1429 12055631 +1998 UNITED ST6 MFGR#143 6500459 +1998 UNITED ST6 MFGR#1430 6816243 +1998 UNITED ST6 MFGR#1432 7752973 +1998 UNITED ST6 MFGR#1436 6292813 +1998 UNITED ST6 MFGR#1437 2334894 +1998 UNITED ST6 MFGR#1438 4459455 +1998 UNITED ST6 MFGR#144 640220 +1998 UNITED ST6 MFGR#1440 748200 +1998 UNITED ST6 MFGR#145 7948794 +1998 UNITED ST6 MFGR#146 2734333 +1998 UNITED ST6 MFGR#147 2995632 +1998 UNITED ST6 MFGR#148 10695287 +1998 UNITED ST6 MFGR#149 1475864 +1998 UNITED ST7 MFGR#141 4816637 +1998 UNITED ST7 MFGR#1410 24393912 +1998 UNITED ST7 MFGR#1411 1283038 +1998 UNITED ST7 MFGR#1412 12607459 +1998 UNITED ST7 MFGR#1413 15421293 +1998 UNITED ST7 MFGR#1414 7681668 +1998 UNITED ST7 MFGR#1415 7138377 +1998 UNITED ST7 MFGR#1416 9246990 +1998 UNITED ST7 MFGR#1417 5242972 +1998 UNITED ST7 MFGR#1419 13548941 +1998 UNITED ST7 MFGR#142 17758762 +1998 UNITED ST7 MFGR#1420 11980437 +1998 UNITED ST7 MFGR#1421 11166952 +1998 UNITED ST7 MFGR#1422 9231883 +1998 UNITED ST7 MFGR#1424 5796389 +1998 UNITED ST7 MFGR#1425 4943582 +1998 UNITED ST7 MFGR#1426 16361100 +1998 UNITED ST7 MFGR#1427 9659296 +1998 UNITED ST7 MFGR#1428 2998020 +1998 UNITED ST7 MFGR#1429 5766768 +1998 UNITED ST7 MFGR#143 5526167 +1998 UNITED ST7 MFGR#1430 4191892 +1998 UNITED ST7 MFGR#1431 12979559 +1998 UNITED ST7 MFGR#1432 5832068 +1998 UNITED ST7 MFGR#1433 11410227 +1998 UNITED ST7 MFGR#1434 10525792 +1998 UNITED ST7 MFGR#1435 5982420 +1998 UNITED ST7 MFGR#1436 7142934 +1998 UNITED ST7 MFGR#1437 7499671 +1998 UNITED ST7 MFGR#1438 9177564 +1998 UNITED ST7 MFGR#1439 1974444 +1998 UNITED ST7 MFGR#144 2428463 +1998 UNITED ST7 MFGR#1440 3300838 +1998 UNITED ST7 MFGR#146 1771645 +1998 UNITED ST7 MFGR#147 4681835 +1998 UNITED ST7 MFGR#148 7124367 +1998 UNITED ST7 MFGR#149 11662970 +1998 UNITED ST8 MFGR#141 12753354 +1998 UNITED ST8 MFGR#1410 1097292 +1998 UNITED ST8 MFGR#1411 4069881 +1998 UNITED ST8 MFGR#1414 9474831 +1998 UNITED ST8 MFGR#1415 6406304 +1998 UNITED ST8 MFGR#1418 2392133 +1998 UNITED ST8 MFGR#1419 2953230 +1998 UNITED ST8 MFGR#142 10702563 +1998 UNITED ST8 MFGR#1420 3719432 +1998 UNITED ST8 MFGR#1421 3854657 +1998 UNITED ST8 MFGR#1422 9711968 +1998 UNITED ST8 MFGR#1424 1102516 +1998 UNITED ST8 MFGR#1426 8937532 +1998 UNITED ST8 MFGR#1427 5255637 +1998 UNITED ST8 MFGR#1428 1695112 +1998 UNITED ST8 MFGR#143 6231349 +1998 UNITED ST8 MFGR#1430 4678664 +1998 UNITED ST8 MFGR#1431 2459741 +1998 UNITED ST8 MFGR#1433 1724339 +1998 UNITED ST8 MFGR#1434 8222328 +1998 UNITED ST8 MFGR#1435 4087165 +1998 UNITED ST8 MFGR#1437 7379288 +1998 UNITED ST8 MFGR#1439 1947163 +1998 UNITED ST8 MFGR#144 21846266 +1998 UNITED ST8 MFGR#146 13768113 +1998 UNITED ST8 MFGR#147 5279643 +1998 UNITED ST8 MFGR#149 15221410 +1998 UNITED ST9 MFGR#141 5544000 +1998 UNITED ST9 MFGR#1410 7971104 +1998 UNITED ST9 MFGR#1411 26426609 +1998 UNITED ST9 MFGR#1412 1701777 +1998 UNITED ST9 MFGR#1413 18786672 +1998 UNITED ST9 MFGR#1414 15787780 +1998 UNITED ST9 MFGR#1415 11214818 +1998 UNITED ST9 MFGR#1416 2076120 +1998 UNITED ST9 MFGR#1417 14310334 +1998 UNITED ST9 MFGR#1418 3558355 +1998 UNITED ST9 MFGR#1419 6134867 +1998 UNITED ST9 MFGR#142 257897 +1998 UNITED ST9 MFGR#1420 7716338 +1998 UNITED ST9 MFGR#1421 10291775 +1998 UNITED ST9 MFGR#1422 6105716 +1998 UNITED ST9 MFGR#1423 11378613 +1998 UNITED ST9 MFGR#1424 1904550 +1998 UNITED ST9 MFGR#1425 18831147 +1998 UNITED ST9 MFGR#1426 18562642 +1998 UNITED ST9 MFGR#1427 12565890 +1998 UNITED ST9 MFGR#1428 10720914 +1998 UNITED ST9 MFGR#1429 21940718 +1998 UNITED ST9 MFGR#143 21480722 +1998 UNITED ST9 MFGR#1430 30935173 +1998 UNITED ST9 MFGR#1431 7793012 +1998 UNITED ST9 MFGR#1432 10026104 +1998 UNITED ST9 MFGR#1433 17874435 +1998 UNITED ST9 MFGR#1434 5493785 +1998 UNITED ST9 MFGR#1435 10738769 +1998 UNITED ST9 MFGR#1436 43504 +1998 UNITED ST9 MFGR#1437 13559646 +1998 UNITED ST9 MFGR#1438 12654258 +1998 UNITED ST9 MFGR#1439 5972923 +1998 UNITED ST9 MFGR#144 15702123 +1998 UNITED ST9 MFGR#1440 6681029 +1998 UNITED ST9 MFGR#145 920718 +1998 UNITED ST9 MFGR#146 10435931 +1998 UNITED ST9 MFGR#147 4759326 +1998 UNITED ST9 MFGR#148 9970848 +1998 UNITED ST9 MFGR#149 7510940 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q1.1.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q1.1.out new file mode 100644 index 00000000000000..ba450f16d5c36a --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q1.1.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1.1 -- +446031203850 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q1.2.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q1.2.out new file mode 100644 index 00000000000000..9e6e59831d352c --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q1.2.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1.2 -- +98714004603 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q1.3.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q1.3.out new file mode 100644 index 00000000000000..dd5cabbd7c348e --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q1.3.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1.3 -- +26232701005 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q2.1.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q2.1.out new file mode 100644 index 00000000000000..e1ebddab4123d7 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q2.1.out @@ -0,0 +1,283 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q2.1 -- +567838207 1992 MFGR#121 +610663790 1992 MFGR#1210 +550769662 1992 MFGR#1211 +649205856 1992 MFGR#1212 +624031241 1992 MFGR#1213 +670488468 1992 MFGR#1214 +633152470 1992 MFGR#1215 +674846781 1992 MFGR#1216 +675093435 1992 MFGR#1217 +600202070 1992 MFGR#1218 +538043594 1992 MFGR#1219 +655326672 1992 MFGR#122 +540262882 1992 MFGR#1220 +556120633 1992 MFGR#1221 +590762777 1992 MFGR#1222 +535448651 1992 MFGR#1223 +703752611 1992 MFGR#1224 +570832868 1992 MFGR#1225 +614061593 1992 MFGR#1226 +581759388 1992 MFGR#1227 +644642592 1992 MFGR#1228 +640858430 1992 MFGR#1229 +789755835 1992 MFGR#123 +468535087 1992 MFGR#1230 +592436656 1992 MFGR#1231 +664275152 1992 MFGR#1232 +613885100 1992 MFGR#1233 +667399281 1992 MFGR#1234 +640290070 1992 MFGR#1235 +501892561 1992 MFGR#1236 +591481503 1992 MFGR#1237 +477423770 1992 MFGR#1238 +638259374 1992 MFGR#1239 +572354196 1992 MFGR#124 +740479248 1992 MFGR#1240 +478777095 1992 MFGR#125 +592174616 1992 MFGR#126 +706151632 1992 MFGR#127 +542306646 1992 MFGR#128 +581987352 1992 MFGR#129 +823087702 1993 MFGR#121 +648160706 1993 MFGR#1210 +634743898 1993 MFGR#1211 +785639283 1993 MFGR#1212 +638255029 1993 MFGR#1213 +616837237 1993 MFGR#1214 +634687975 1993 MFGR#1215 +638353900 1993 MFGR#1216 +663372951 1993 MFGR#1217 +683985855 1993 MFGR#1218 +646950033 1993 MFGR#1219 +622532984 1993 MFGR#122 +530830127 1993 MFGR#1220 +543346337 1993 MFGR#1221 +756921203 1993 MFGR#1222 +533544350 1993 MFGR#1223 +915916085 1993 MFGR#1224 +473007381 1993 MFGR#1225 +739036124 1993 MFGR#1226 +592178887 1993 MFGR#1227 +583507058 1993 MFGR#1228 +617453491 1993 MFGR#1229 +637863868 1993 MFGR#123 +625534310 1993 MFGR#1230 +580327635 1993 MFGR#1231 +697373098 1993 MFGR#1232 +515571416 1993 MFGR#1233 +651935758 1993 MFGR#1234 +575779480 1993 MFGR#1235 +591878667 1993 MFGR#1236 +609618576 1993 MFGR#1237 +444614010 1993 MFGR#1238 +595256327 1993 MFGR#1239 +660586237 1993 MFGR#124 +788730059 1993 MFGR#1240 +616224539 1993 MFGR#125 +617126754 1993 MFGR#126 +654438324 1993 MFGR#127 +731657001 1993 MFGR#128 +548048395 1993 MFGR#129 +564405648 1994 MFGR#121 +645404849 1994 MFGR#1210 +631620635 1994 MFGR#1211 +568332348 1994 MFGR#1212 +678785857 1994 MFGR#1213 +534002330 1994 MFGR#1214 +654400242 1994 MFGR#1215 +558646341 1994 MFGR#1216 +687845641 1994 MFGR#1217 +546674347 1994 MFGR#1218 +567272942 1994 MFGR#1219 +659884062 1994 MFGR#122 +562582172 1994 MFGR#1220 +598618997 1994 MFGR#1221 +601016441 1994 MFGR#1222 +555134404 1994 MFGR#1223 +737422302 1994 MFGR#1224 +570745955 1994 MFGR#1225 +746302245 1994 MFGR#1226 +651707481 1994 MFGR#1227 +573693547 1994 MFGR#1228 +647918373 1994 MFGR#1229 +580449592 1994 MFGR#123 +493270412 1994 MFGR#1230 +603546148 1994 MFGR#1231 +719865331 1994 MFGR#1232 +638982238 1994 MFGR#1233 +743247677 1994 MFGR#1234 +598680959 1994 MFGR#1235 +615726097 1994 MFGR#1236 +542569815 1994 MFGR#1237 +573510781 1994 MFGR#1238 +579855853 1994 MFGR#1239 +684573322 1994 MFGR#124 +873735737 1994 MFGR#1240 +560488304 1994 MFGR#125 +657036514 1994 MFGR#126 +622571183 1994 MFGR#127 +586845664 1994 MFGR#128 +534541525 1994 MFGR#129 +706469511 1995 MFGR#121 +602892803 1995 MFGR#1210 +645166092 1995 MFGR#1211 +613289283 1995 MFGR#1212 +599586479 1995 MFGR#1213 +562570804 1995 MFGR#1214 +672528755 1995 MFGR#1215 +669000972 1995 MFGR#1216 +725362449 1995 MFGR#1217 +657026635 1995 MFGR#1218 +519659003 1995 MFGR#1219 +724727741 1995 MFGR#122 +517956131 1995 MFGR#1220 +635741351 1995 MFGR#1221 +564368410 1995 MFGR#1222 +600665149 1995 MFGR#1223 +762700351 1995 MFGR#1224 +671669586 1995 MFGR#1225 +572568748 1995 MFGR#1226 +530361300 1995 MFGR#1227 +633357085 1995 MFGR#1228 +547960244 1995 MFGR#1229 +660711077 1995 MFGR#123 +602735858 1995 MFGR#1230 +499852146 1995 MFGR#1231 +715300753 1995 MFGR#1232 +557149571 1995 MFGR#1233 +710023059 1995 MFGR#1234 +622425239 1995 MFGR#1235 +634565501 1995 MFGR#1236 +572847270 1995 MFGR#1237 +549318912 1995 MFGR#1238 +593851712 1995 MFGR#1239 +585421815 1995 MFGR#124 +707207888 1995 MFGR#1240 +538246872 1995 MFGR#125 +605799021 1995 MFGR#126 +665978112 1995 MFGR#127 +646960956 1995 MFGR#128 +508749401 1995 MFGR#129 +523879145 1996 MFGR#121 +643645053 1996 MFGR#1210 +595065339 1996 MFGR#1211 +674626440 1996 MFGR#1212 +496297087 1996 MFGR#1213 +583249505 1996 MFGR#1214 +702184857 1996 MFGR#1215 +601809334 1996 MFGR#1216 +704898387 1996 MFGR#1217 +528843086 1996 MFGR#1218 +586246330 1996 MFGR#1219 +712110492 1996 MFGR#122 +518444215 1996 MFGR#1220 +499319414 1996 MFGR#1221 +679469356 1996 MFGR#1222 +628762754 1996 MFGR#1223 +724844856 1996 MFGR#1224 +660620587 1996 MFGR#1225 +667674729 1996 MFGR#1226 +483838085 1996 MFGR#1227 +609855391 1996 MFGR#1228 +658959557 1996 MFGR#1229 +566217852 1996 MFGR#123 +528879998 1996 MFGR#1230 +589481194 1996 MFGR#1231 +702805896 1996 MFGR#1232 +663679947 1996 MFGR#1233 +571149450 1996 MFGR#1234 +478648074 1996 MFGR#1235 +568249365 1996 MFGR#1236 +592616167 1996 MFGR#1237 +466676148 1996 MFGR#1238 +670693719 1996 MFGR#1239 +560667719 1996 MFGR#124 +821167950 1996 MFGR#1240 +476864333 1996 MFGR#125 +558030884 1996 MFGR#126 +635873891 1996 MFGR#127 +551010618 1996 MFGR#128 +560570630 1996 MFGR#129 +587013207 1997 MFGR#121 +616287892 1997 MFGR#1210 +548588761 1997 MFGR#1211 +589593892 1997 MFGR#1212 +424306670 1997 MFGR#1213 +511971910 1997 MFGR#1214 +631772246 1997 MFGR#1215 +692135140 1997 MFGR#1216 +777994957 1997 MFGR#1217 +707053720 1997 MFGR#1218 +561169527 1997 MFGR#1219 +664916245 1997 MFGR#122 +594466157 1997 MFGR#1220 +588848171 1997 MFGR#1221 +528988960 1997 MFGR#1222 +537098211 1997 MFGR#1223 +674763166 1997 MFGR#1224 +450402292 1997 MFGR#1225 +701360722 1997 MFGR#1226 +506011570 1997 MFGR#1227 +585578737 1997 MFGR#1228 +622744016 1997 MFGR#1229 +646503168 1997 MFGR#123 +571800941 1997 MFGR#1230 +502601790 1997 MFGR#1231 +677924656 1997 MFGR#1232 +534455976 1997 MFGR#1233 +714934715 1997 MFGR#1234 +767151420 1997 MFGR#1235 +618877179 1997 MFGR#1236 +639638057 1997 MFGR#1237 +401953419 1997 MFGR#1238 +610756714 1997 MFGR#1239 +543248087 1997 MFGR#124 +675132692 1997 MFGR#1240 +479099365 1997 MFGR#125 +570696568 1997 MFGR#126 +583074592 1997 MFGR#127 +695133104 1997 MFGR#128 +655638776 1997 MFGR#129 +344575925 1998 MFGR#121 +417152416 1998 MFGR#1210 +317068168 1998 MFGR#1211 +374341516 1998 MFGR#1212 +332740903 1998 MFGR#1213 +304873002 1998 MFGR#1214 +366101132 1998 MFGR#1215 +379133898 1998 MFGR#1216 +359508497 1998 MFGR#1217 +320623334 1998 MFGR#1218 +346182862 1998 MFGR#1219 +312440027 1998 MFGR#122 +348123961 1998 MFGR#1220 +339845398 1998 MFGR#1221 +355416161 1998 MFGR#1222 +344889822 1998 MFGR#1223 +396906691 1998 MFGR#1224 +290208878 1998 MFGR#1225 +419415707 1998 MFGR#1226 +358466340 1998 MFGR#1227 +251549955 1998 MFGR#1228 +383138860 1998 MFGR#1229 +296330561 1998 MFGR#123 +437181243 1998 MFGR#1230 +398944492 1998 MFGR#1231 +424062455 1998 MFGR#1232 +406967188 1998 MFGR#1233 +428867240 1998 MFGR#1234 +352277781 1998 MFGR#1235 +361827086 1998 MFGR#1236 +341618569 1998 MFGR#1237 +244739231 1998 MFGR#1238 +414151803 1998 MFGR#1239 +330082371 1998 MFGR#124 +415312453 1998 MFGR#1240 +360289624 1998 MFGR#125 +341657580 1998 MFGR#126 +377507061 1998 MFGR#127 +361416497 1998 MFGR#128 +318769573 1998 MFGR#129 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q2.2.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q2.2.out new file mode 100644 index 00000000000000..8677d312f47d44 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q2.2.out @@ -0,0 +1,59 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q2.2 -- +709524929 1992 MFGR#2221 +783846394 1992 MFGR#2222 +765052002 1992 MFGR#2223 +651488962 1992 MFGR#2224 +646528589 1992 MFGR#2225 +709650548 1992 MFGR#2226 +745556316 1992 MFGR#2227 +756901875 1992 MFGR#2228 +766521103 1993 MFGR#2221 +691475597 1993 MFGR#2222 +758220752 1993 MFGR#2223 +669662707 1993 MFGR#2224 +773854228 1993 MFGR#2225 +737087518 1993 MFGR#2226 +781967766 1993 MFGR#2227 +680880216 1993 MFGR#2228 +685777518 1994 MFGR#2221 +666524807 1994 MFGR#2222 +733993590 1994 MFGR#2223 +707869040 1994 MFGR#2224 +721251967 1994 MFGR#2225 +822495919 1994 MFGR#2226 +720837128 1994 MFGR#2227 +826225350 1994 MFGR#2228 +775437074 1995 MFGR#2221 +761354792 1995 MFGR#2222 +637832575 1995 MFGR#2223 +589765707 1995 MFGR#2224 +708290039 1995 MFGR#2225 +747356383 1995 MFGR#2226 +802502540 1995 MFGR#2227 +895936786 1995 MFGR#2228 +700010008 1996 MFGR#2221 +777310085 1996 MFGR#2222 +656095314 1996 MFGR#2223 +656859917 1996 MFGR#2224 +765820896 1996 MFGR#2225 +808177734 1996 MFGR#2226 +729563303 1996 MFGR#2227 +819665874 1996 MFGR#2228 +727342382 1997 MFGR#2221 +664533779 1997 MFGR#2222 +748288392 1997 MFGR#2223 +630422081 1997 MFGR#2224 +757391203 1997 MFGR#2225 +747889257 1997 MFGR#2226 +728857899 1997 MFGR#2227 +775312985 1997 MFGR#2228 +335304504 1998 MFGR#2221 +409347137 1998 MFGR#2222 +459109577 1998 MFGR#2223 +413318072 1998 MFGR#2224 +410402095 1998 MFGR#2225 +453515044 1998 MFGR#2226 +390506405 1998 MFGR#2227 +397939103 1998 MFGR#2228 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q2.3.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q2.3.out new file mode 100644 index 00000000000000..cb87e345040a0a --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q2.3.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q2.3 -- +726427486 1992 MFGR#2239 +705238959 1993 MFGR#2239 +664145134 1994 MFGR#2239 +713839336 1995 MFGR#2239 +728492865 1996 MFGR#2239 +733896532 1997 MFGR#2239 +380255731 1998 MFGR#2239 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q3.1.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q3.1.out new file mode 100644 index 00000000000000..c618f9495d97c2 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q3.1.out @@ -0,0 +1,153 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q3.1 -- +INDONESIA INDONESIA 1992 6905698988 +CHINA INDONESIA 1992 6616078869 +CHINA CHINA 1992 6456431477 +VIETNAM INDONESIA 1992 6340181707 +VIETNAM CHINA 1992 6332844390 +INDONESIA CHINA 1992 6310709533 +INDIA INDONESIA 1992 6238807354 +JAPAN INDONESIA 1992 6222565638 +CHINA INDIA 1992 6189831351 +CHINA JAPAN 1992 6157678893 +JAPAN CHINA 1992 6067100655 +INDIA CHINA 1992 6066461811 +VIETNAM JAPAN 1992 5863786349 +JAPAN INDIA 1992 5802749985 +INDONESIA INDIA 1992 5770203218 +VIETNAM INDIA 1992 5698511401 +INDONESIA JAPAN 1992 5663765610 +JAPAN JAPAN 1992 5648034711 +INDIA JAPAN 1992 5421959281 +CHINA VIETNAM 1992 5412322026 +JAPAN VIETNAM 1992 5401692555 +INDIA INDIA 1992 5361243755 +INDONESIA VIETNAM 1992 5302638372 +INDIA VIETNAM 1992 5245330621 +VIETNAM VIETNAM 1992 5111731778 +INDONESIA INDONESIA 1993 6931363262 +INDONESIA CHINA 1993 6612891249 +CHINA INDONESIA 1993 6581513366 +VIETNAM INDONESIA 1993 6511639352 +CHINA CHINA 1993 6444513787 +CHINA INDIA 1993 6421694121 +VIETNAM CHINA 1993 6413579999 +INDIA INDONESIA 1993 6331058594 +JAPAN CHINA 1993 6292293995 +INDIA CHINA 1993 6209353792 +CHINA JAPAN 1993 6195466884 +VIETNAM INDIA 1993 6161174477 +INDONESIA INDIA 1993 6151664306 +INDONESIA JAPAN 1993 6147605249 +JAPAN INDONESIA 1993 6053946313 +INDIA JAPAN 1993 5995208985 +CHINA VIETNAM 1993 5885023228 +VIETNAM JAPAN 1993 5874266867 +INDONESIA VIETNAM 1993 5840287722 +JAPAN INDIA 1993 5823343157 +INDIA INDIA 1993 5571575799 +VIETNAM VIETNAM 1993 5554161183 +JAPAN JAPAN 1993 5430318583 +JAPAN VIETNAM 1993 5406569653 +INDIA VIETNAM 1993 5233871487 +VIETNAM CHINA 1994 6709883442 +CHINA CHINA 1994 6648942965 +INDONESIA CHINA 1994 6483600910 +CHINA JAPAN 1994 6349037021 +VIETNAM INDONESIA 1994 6347441931 +JAPAN CHINA 1994 6320009218 +INDONESIA INDONESIA 1994 6315331086 +CHINA INDIA 1994 6297582811 +CHINA INDONESIA 1994 6234787896 +VIETNAM JAPAN 1994 6231973047 +INDONESIA INDIA 1994 6198412245 +INDIA INDONESIA 1994 6168189859 +INDONESIA JAPAN 1994 6141195969 +JAPAN INDONESIA 1994 6013034618 +CHINA VIETNAM 1994 5885318733 +INDIA CHINA 1994 5815022714 +VIETNAM INDIA 1994 5789398691 +JAPAN JAPAN 1994 5753821482 +JAPAN INDIA 1994 5645818872 +INDONESIA VIETNAM 1994 5624346368 +INDIA INDIA 1994 5584936515 +VIETNAM VIETNAM 1994 5418498159 +INDIA JAPAN 1994 5394134974 +JAPAN VIETNAM 1994 5275821637 +INDIA VIETNAM 1994 5251039815 +CHINA INDONESIA 1995 7074539444 +INDONESIA CHINA 1995 6928620620 +CHINA CHINA 1995 6624666612 +JAPAN CHINA 1995 6519889385 +VIETNAM CHINA 1995 6332731268 +INDONESIA INDONESIA 1995 6268531321 +VIETNAM JAPAN 1995 6263957020 +CHINA INDIA 1995 6246656374 +VIETNAM INDIA 1995 6198441957 +INDONESIA INDIA 1995 6173805183 +VIETNAM INDONESIA 1995 6057820148 +CHINA JAPAN 1995 6021854130 +INDONESIA JAPAN 1995 6019093871 +INDIA CHINA 1995 6010532793 +INDIA INDONESIA 1995 5975518597 +JAPAN JAPAN 1995 5952279129 +JAPAN INDONESIA 1995 5947000735 +VIETNAM VIETNAM 1995 5754727607 +JAPAN INDIA 1995 5730743126 +INDONESIA VIETNAM 1995 5574974066 +INDIA INDIA 1995 5565995550 +JAPAN VIETNAM 1995 5501313775 +CHINA VIETNAM 1995 5489936063 +INDIA JAPAN 1995 5469203427 +INDIA VIETNAM 1995 5207063284 +INDONESIA INDONESIA 1996 6750056283 +CHINA INDONESIA 1996 6657125075 +INDONESIA CHINA 1996 6613439112 +CHINA CHINA 1996 6591697915 +VIETNAM CHINA 1996 6428653890 +VIETNAM INDONESIA 1996 6295626732 +JAPAN INDONESIA 1996 6227447856 +INDIA CHINA 1996 6198567819 +INDIA INDONESIA 1996 6189242058 +CHINA INDIA 1996 6153787365 +INDONESIA INDIA 1996 6138810863 +CHINA JAPAN 1996 6105566324 +JAPAN CHINA 1996 6088985564 +INDONESIA JAPAN 1996 6055638222 +VIETNAM JAPAN 1996 5919997150 +VIETNAM VIETNAM 1996 5860842302 +INDIA JAPAN 1996 5841664805 +VIETNAM INDIA 1996 5807486983 +JAPAN INDIA 1996 5644579616 +JAPAN JAPAN 1996 5566204795 +INDIA INDIA 1996 5532073103 +INDONESIA VIETNAM 1996 5438514387 +CHINA VIETNAM 1996 5434772818 +JAPAN VIETNAM 1996 5334277549 +INDIA VIETNAM 1996 5293891531 +CHINA INDONESIA 1997 6653234696 +INDONESIA CHINA 1997 6577396314 +CHINA CHINA 1997 6463794795 +JAPAN INDONESIA 1997 6429270126 +INDONESIA INDONESIA 1997 6398427821 +VIETNAM INDONESIA 1997 6363583120 +CHINA JAPAN 1997 6284733518 +VIETNAM CHINA 1997 6164218723 +INDIA CHINA 1997 6153282214 +CHINA INDIA 1997 6084283983 +INDONESIA JAPAN 1997 6001427956 +INDIA INDONESIA 1997 5983439300 +JAPAN CHINA 1997 5973849687 +CHINA VIETNAM 1997 5908254176 +JAPAN INDIA 1997 5796921992 +JAPAN JAPAN 1997 5790450039 +INDONESIA INDIA 1997 5789367105 +VIETNAM INDIA 1997 5730121608 +INDIA INDIA 1997 5693028907 +VIETNAM JAPAN 1997 5689345230 +INDONESIA VIETNAM 1997 5665678132 +VIETNAM VIETNAM 1997 5620251324 +INDIA JAPAN 1997 5587827913 +JAPAN VIETNAM 1997 5249874614 +INDIA VIETNAM 1997 5003624837 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q3.2.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q3.2.out new file mode 100644 index 00000000000000..6fa0c9ba6d26f7 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q3.2.out @@ -0,0 +1,602 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q3.2 -- +UNITED ST4 UNITED ST1 1992 102027455 +UNITED ST1 UNITED ST0 1992 96989491 +UNITED ST7 UNITED ST0 1992 96078010 +UNITED ST9 UNITED ST0 1992 94813294 +UNITED ST4 UNITED ST0 1992 94644242 +UNITED ST2 UNITED ST4 1992 91180500 +UNITED ST5 UNITED ST0 1992 90432300 +UNITED ST6 UNITED ST7 1992 87658267 +UNITED ST3 UNITED ST9 1992 86142048 +UNITED ST6 UNITED ST5 1992 85882966 +UNITED ST7 UNITED ST3 1992 83765666 +UNITED ST2 UNITED ST9 1992 83705618 +UNITED ST4 UNITED ST6 1992 81886374 +UNITED ST2 UNITED ST1 1992 81839165 +UNITED ST9 UNITED ST1 1992 80795302 +UNITED ST6 UNITED ST3 1992 78778218 +UNITED ST6 UNITED ST0 1992 78696956 +UNITED ST0 UNITED ST1 1992 77267396 +UNITED ST0 UNITED ST0 1992 75622122 +UNITED ST1 UNITED ST9 1992 75367059 +UNITED ST3 UNITED ST1 1992 73637490 +UNITED ST2 UNITED ST0 1992 72210218 +UNITED ST1 UNITED ST7 1992 71472973 +UNITED ST6 UNITED ST4 1992 71086944 +UNITED ST4 UNITED ST4 1992 70111335 +UNITED ST6 UNITED ST1 1992 69408688 +UNITED ST4 UNITED ST3 1992 69001787 +UNITED ST5 UNITED ST7 1992 68333651 +UNITED ST4 UNITED ST9 1992 67837970 +UNITED ST7 UNITED ST6 1992 65513205 +UNITED ST4 UNITED ST5 1992 65057872 +UNITED ST7 UNITED ST4 1992 64900888 +UNITED ST1 UNITED ST1 1992 64669070 +UNITED ST3 UNITED ST5 1992 64239048 +UNITED ST0 UNITED ST9 1992 63979996 +UNITED ST3 UNITED ST4 1992 63144772 +UNITED ST5 UNITED ST6 1992 62628093 +UNITED ST4 UNITED ST7 1992 62529376 +UNITED ST3 UNITED ST0 1992 62441656 +UNITED ST9 UNITED ST4 1992 61489513 +UNITED ST8 UNITED ST6 1992 60540440 +UNITED ST7 UNITED ST9 1992 60326042 +UNITED ST7 UNITED ST7 1992 60121386 +UNITED ST5 UNITED ST1 1992 59945287 +UNITED ST5 UNITED ST4 1992 57625627 +UNITED ST7 UNITED ST5 1992 57566802 +UNITED ST2 UNITED ST5 1992 57021365 +UNITED ST9 UNITED ST7 1992 56883359 +UNITED ST0 UNITED ST3 1992 56359317 +UNITED ST1 UNITED ST3 1992 55727474 +UNITED ST5 UNITED ST3 1992 53963553 +UNITED ST0 UNITED ST7 1992 50583409 +UNITED ST5 UNITED ST9 1992 50191091 +UNITED ST7 UNITED ST1 1992 50167208 +UNITED ST0 UNITED ST8 1992 49732640 +UNITED ST0 UNITED ST4 1992 49676807 +UNITED ST9 UNITED ST3 1992 47681165 +UNITED ST8 UNITED ST4 1992 46757019 +UNITED ST3 UNITED ST3 1992 45087216 +UNITED ST8 UNITED ST0 1992 44368839 +UNITED ST0 UNITED ST6 1992 42471806 +UNITED ST6 UNITED ST8 1992 42463690 +UNITED ST8 UNITED ST7 1992 41897901 +UNITED ST3 UNITED ST8 1992 41275764 +UNITED ST6 UNITED ST9 1992 40591721 +UNITED ST0 UNITED ST5 1992 40120886 +UNITED ST1 UNITED ST4 1992 39326346 +UNITED ST3 UNITED ST7 1992 39028579 +UNITED ST3 UNITED ST6 1992 38798715 +UNITED ST9 UNITED ST9 1992 36048343 +UNITED ST2 UNITED ST8 1992 36046449 +UNITED ST2 UNITED ST3 1992 35981963 +UNITED ST8 UNITED ST1 1992 35680752 +UNITED ST1 UNITED ST6 1992 35404990 +UNITED ST8 UNITED ST5 1992 35187610 +UNITED ST1 UNITED ST5 1992 33971251 +UNITED ST5 UNITED ST8 1992 33878053 +UNITED ST2 UNITED ST7 1992 33702779 +UNITED ST8 UNITED ST3 1992 30949324 +UNITED ST8 UNITED ST8 1992 29309108 +UNITED ST5 UNITED ST5 1992 29279568 +UNITED ST1 UNITED ST8 1992 28565579 +UNITED ST9 UNITED ST5 1992 28075004 +UNITED ST2 UNITED ST6 1992 27813739 +UNITED ST0 UNITED ST2 1992 27718733 +UNITED ST2 UNITED ST2 1992 25743654 +UNITED ST8 UNITED ST9 1992 22684471 +UNITED ST4 UNITED ST8 1992 21928442 +UNITED ST9 UNITED ST8 1992 21386100 +UNITED ST5 UNITED ST2 1992 20495817 +UNITED ST6 UNITED ST6 1992 18137105 +UNITED ST9 UNITED ST6 1992 15879568 +UNITED ST4 UNITED ST2 1992 12061845 +UNITED ST7 UNITED ST8 1992 11895702 +UNITED ST6 UNITED ST2 1992 11820698 +UNITED ST9 UNITED ST2 1992 11623177 +UNITED ST8 UNITED ST2 1992 10971561 +UNITED ST1 UNITED ST2 1992 7706728 +UNITED ST7 UNITED ST2 1992 4943204 +UNITED ST3 UNITED ST2 1992 1097208 +UNITED ST0 UNITED ST9 1993 109834040 +UNITED ST7 UNITED ST0 1993 109788024 +UNITED ST5 UNITED ST0 1993 106822597 +UNITED ST0 UNITED ST0 1993 106742548 +UNITED ST1 UNITED ST0 1993 99305952 +UNITED ST4 UNITED ST4 1993 98150465 +UNITED ST3 UNITED ST4 1993 92493920 +UNITED ST0 UNITED ST1 1993 91196593 +UNITED ST4 UNITED ST1 1993 88521423 +UNITED ST8 UNITED ST0 1993 88356371 +UNITED ST4 UNITED ST7 1993 88172198 +UNITED ST4 UNITED ST0 1993 86918458 +UNITED ST6 UNITED ST3 1993 83417161 +UNITED ST6 UNITED ST1 1993 83345939 +UNITED ST7 UNITED ST9 1993 80310701 +UNITED ST3 UNITED ST1 1993 78230278 +UNITED ST6 UNITED ST7 1993 78197294 +UNITED ST5 UNITED ST9 1993 76286539 +UNITED ST0 UNITED ST3 1993 76171283 +UNITED ST5 UNITED ST8 1993 74359279 +UNITED ST9 UNITED ST1 1993 74059419 +UNITED ST4 UNITED ST9 1993 73296959 +UNITED ST5 UNITED ST1 1993 71454623 +UNITED ST6 UNITED ST4 1993 69646913 +UNITED ST2 UNITED ST1 1993 69631701 +UNITED ST6 UNITED ST0 1993 68247539 +UNITED ST7 UNITED ST7 1993 68109820 +UNITED ST2 UNITED ST3 1993 66972438 +UNITED ST3 UNITED ST0 1993 66626926 +UNITED ST9 UNITED ST7 1993 66625483 +UNITED ST1 UNITED ST8 1993 66146198 +UNITED ST2 UNITED ST7 1993 64185014 +UNITED ST5 UNITED ST4 1993 63415639 +UNITED ST9 UNITED ST9 1993 63260763 +UNITED ST1 UNITED ST4 1993 62884347 +UNITED ST7 UNITED ST4 1993 61656613 +UNITED ST3 UNITED ST6 1993 58584808 +UNITED ST2 UNITED ST4 1993 56650391 +UNITED ST3 UNITED ST5 1993 55907305 +UNITED ST6 UNITED ST9 1993 54900942 +UNITED ST1 UNITED ST7 1993 54851183 +UNITED ST3 UNITED ST9 1993 54762596 +UNITED ST8 UNITED ST6 1993 54633062 +UNITED ST8 UNITED ST3 1993 54049874 +UNITED ST5 UNITED ST7 1993 52745538 +UNITED ST0 UNITED ST5 1993 52701052 +UNITED ST1 UNITED ST9 1993 52514902 +UNITED ST8 UNITED ST5 1993 52237837 +UNITED ST1 UNITED ST3 1993 52097946 +UNITED ST8 UNITED ST4 1993 51419356 +UNITED ST0 UNITED ST6 1993 50432282 +UNITED ST5 UNITED ST5 1993 50357189 +UNITED ST3 UNITED ST7 1993 50135448 +UNITED ST0 UNITED ST4 1993 49260067 +UNITED ST0 UNITED ST7 1993 48796360 +UNITED ST2 UNITED ST9 1993 48188507 +UNITED ST1 UNITED ST1 1993 47538610 +UNITED ST9 UNITED ST3 1993 46943647 +UNITED ST7 UNITED ST5 1993 44763692 +UNITED ST1 UNITED ST6 1993 44728540 +UNITED ST8 UNITED ST1 1993 44415434 +UNITED ST7 UNITED ST8 1993 43902628 +UNITED ST9 UNITED ST6 1993 43867160 +UNITED ST2 UNITED ST0 1993 42845485 +UNITED ST3 UNITED ST8 1993 42251848 +UNITED ST0 UNITED ST8 1993 42002682 +UNITED ST4 UNITED ST8 1993 41657582 +UNITED ST1 UNITED ST5 1993 40693513 +UNITED ST9 UNITED ST5 1993 39685269 +UNITED ST7 UNITED ST3 1993 39523861 +UNITED ST8 UNITED ST8 1993 38790235 +UNITED ST8 UNITED ST9 1993 38516361 +UNITED ST2 UNITED ST5 1993 37406845 +UNITED ST9 UNITED ST8 1993 37184696 +UNITED ST8 UNITED ST7 1993 36902218 +UNITED ST6 UNITED ST8 1993 36456741 +UNITED ST7 UNITED ST1 1993 34391159 +UNITED ST6 UNITED ST5 1993 34229082 +UNITED ST5 UNITED ST3 1993 34031811 +UNITED ST2 UNITED ST8 1993 33445446 +UNITED ST4 UNITED ST3 1993 33129412 +UNITED ST6 UNITED ST6 1993 33050663 +UNITED ST9 UNITED ST0 1993 32653305 +UNITED ST4 UNITED ST6 1993 30699255 +UNITED ST9 UNITED ST4 1993 30644687 +UNITED ST4 UNITED ST5 1993 29119594 +UNITED ST7 UNITED ST6 1993 27100502 +UNITED ST4 UNITED ST2 1993 27012678 +UNITED ST2 UNITED ST6 1993 26482226 +UNITED ST5 UNITED ST6 1993 25357679 +UNITED ST3 UNITED ST3 1993 21777144 +UNITED ST3 UNITED ST2 1993 21559073 +UNITED ST5 UNITED ST2 1993 20610242 +UNITED ST7 UNITED ST2 1993 20219304 +UNITED ST6 UNITED ST2 1993 18814367 +UNITED ST9 UNITED ST2 1993 17718390 +UNITED ST1 UNITED ST2 1993 16844538 +UNITED ST0 UNITED ST2 1993 15042145 +UNITED ST2 UNITED ST2 1993 14521995 +UNITED ST8 UNITED ST2 1993 9984366 +UNITED ST8 UNITED ST0 1994 99220789 +UNITED ST3 UNITED ST9 1994 97476185 +UNITED ST6 UNITED ST1 1994 96937147 +UNITED ST6 UNITED ST9 1994 94683309 +UNITED ST9 UNITED ST1 1994 90440948 +UNITED ST0 UNITED ST9 1994 89865202 +UNITED ST5 UNITED ST7 1994 89089961 +UNITED ST9 UNITED ST0 1994 87670573 +UNITED ST3 UNITED ST1 1994 85523653 +UNITED ST4 UNITED ST9 1994 83822393 +UNITED ST0 UNITED ST0 1994 83526877 +UNITED ST7 UNITED ST0 1994 82265536 +UNITED ST2 UNITED ST1 1994 81300089 +UNITED ST5 UNITED ST0 1994 78648057 +UNITED ST4 UNITED ST7 1994 76954140 +UNITED ST4 UNITED ST4 1994 76837381 +UNITED ST0 UNITED ST1 1994 76613379 +UNITED ST1 UNITED ST3 1994 75992459 +UNITED ST7 UNITED ST1 1994 75320799 +UNITED ST4 UNITED ST0 1994 73719340 +UNITED ST5 UNITED ST1 1994 73508418 +UNITED ST4 UNITED ST1 1994 72219557 +UNITED ST2 UNITED ST9 1994 69671054 +UNITED ST6 UNITED ST5 1994 66461534 +UNITED ST2 UNITED ST3 1994 65620760 +UNITED ST3 UNITED ST0 1994 65522727 +UNITED ST5 UNITED ST3 1994 65334911 +UNITED ST7 UNITED ST4 1994 64778715 +UNITED ST3 UNITED ST4 1994 63412365 +UNITED ST8 UNITED ST4 1994 62141681 +UNITED ST0 UNITED ST4 1994 61519744 +UNITED ST0 UNITED ST7 1994 61480820 +UNITED ST0 UNITED ST6 1994 61288778 +UNITED ST2 UNITED ST0 1994 60182153 +UNITED ST6 UNITED ST4 1994 59829989 +UNITED ST4 UNITED ST5 1994 59397028 +UNITED ST8 UNITED ST9 1994 58666906 +UNITED ST4 UNITED ST6 1994 58633482 +UNITED ST5 UNITED ST5 1994 56235213 +UNITED ST6 UNITED ST3 1994 56123238 +UNITED ST2 UNITED ST4 1994 55679377 +UNITED ST8 UNITED ST3 1994 55203841 +UNITED ST1 UNITED ST1 1994 54383174 +UNITED ST1 UNITED ST7 1994 53853106 +UNITED ST6 UNITED ST0 1994 53728853 +UNITED ST5 UNITED ST9 1994 53367331 +UNITED ST9 UNITED ST9 1994 51980849 +UNITED ST5 UNITED ST4 1994 51799593 +UNITED ST7 UNITED ST9 1994 50144085 +UNITED ST7 UNITED ST7 1994 46446442 +UNITED ST6 UNITED ST6 1994 46199722 +UNITED ST7 UNITED ST5 1994 45895364 +UNITED ST3 UNITED ST3 1994 45627153 +UNITED ST8 UNITED ST5 1994 44553056 +UNITED ST9 UNITED ST4 1994 43910761 +UNITED ST1 UNITED ST0 1994 43225201 +UNITED ST1 UNITED ST9 1994 43000037 +UNITED ST7 UNITED ST8 1994 42776467 +UNITED ST0 UNITED ST5 1994 41808301 +UNITED ST2 UNITED ST6 1994 41526105 +UNITED ST1 UNITED ST4 1994 41381558 +UNITED ST3 UNITED ST7 1994 40935131 +UNITED ST8 UNITED ST1 1994 40152096 +UNITED ST9 UNITED ST8 1994 39278808 +UNITED ST5 UNITED ST6 1994 38658451 +UNITED ST2 UNITED ST5 1994 37640317 +UNITED ST8 UNITED ST7 1994 37600687 +UNITED ST9 UNITED ST5 1994 37146726 +UNITED ST6 UNITED ST7 1994 37057808 +UNITED ST8 UNITED ST6 1994 36776569 +UNITED ST3 UNITED ST6 1994 36290257 +UNITED ST9 UNITED ST3 1994 35846500 +UNITED ST2 UNITED ST8 1994 33767774 +UNITED ST0 UNITED ST8 1994 31845433 +UNITED ST4 UNITED ST3 1994 31599433 +UNITED ST9 UNITED ST7 1994 31586173 +UNITED ST1 UNITED ST6 1994 31287326 +UNITED ST1 UNITED ST8 1994 30245153 +UNITED ST7 UNITED ST3 1994 29424840 +UNITED ST9 UNITED ST6 1994 29212927 +UNITED ST0 UNITED ST3 1994 27327829 +UNITED ST6 UNITED ST8 1994 26592996 +UNITED ST3 UNITED ST5 1994 26197875 +UNITED ST6 UNITED ST2 1994 25809000 +UNITED ST1 UNITED ST5 1994 24939138 +UNITED ST7 UNITED ST6 1994 24631937 +UNITED ST1 UNITED ST2 1994 23556586 +UNITED ST4 UNITED ST2 1994 23035892 +UNITED ST2 UNITED ST7 1994 22182758 +UNITED ST0 UNITED ST2 1994 22017954 +UNITED ST4 UNITED ST8 1994 20685352 +UNITED ST7 UNITED ST2 1994 19655081 +UNITED ST5 UNITED ST8 1994 18931891 +UNITED ST2 UNITED ST2 1994 18068657 +UNITED ST3 UNITED ST8 1994 15936051 +UNITED ST8 UNITED ST8 1994 10023412 +UNITED ST3 UNITED ST2 1994 9995234 +UNITED ST9 UNITED ST2 1994 9700989 +UNITED ST5 UNITED ST2 1994 7162796 +UNITED ST8 UNITED ST2 1994 3789626 +UNITED ST5 UNITED ST1 1995 119793669 +UNITED ST4 UNITED ST9 1995 99490068 +UNITED ST7 UNITED ST0 1995 98031295 +UNITED ST6 UNITED ST0 1995 91718471 +UNITED ST4 UNITED ST1 1995 90878653 +UNITED ST0 UNITED ST1 1995 90763599 +UNITED ST8 UNITED ST9 1995 88855089 +UNITED ST7 UNITED ST7 1995 86571624 +UNITED ST3 UNITED ST0 1995 84462733 +UNITED ST9 UNITED ST1 1995 82938967 +UNITED ST2 UNITED ST4 1995 82432305 +UNITED ST1 UNITED ST0 1995 81676623 +UNITED ST5 UNITED ST4 1995 81016761 +UNITED ST7 UNITED ST1 1995 79964362 +UNITED ST5 UNITED ST3 1995 78099130 +UNITED ST5 UNITED ST0 1995 77615746 +UNITED ST9 UNITED ST9 1995 76515958 +UNITED ST7 UNITED ST9 1995 75317709 +UNITED ST4 UNITED ST4 1995 74587071 +UNITED ST9 UNITED ST4 1995 72525686 +UNITED ST1 UNITED ST9 1995 72470870 +UNITED ST4 UNITED ST7 1995 69264407 +UNITED ST6 UNITED ST3 1995 67513062 +UNITED ST2 UNITED ST3 1995 65218129 +UNITED ST2 UNITED ST9 1995 65055178 +UNITED ST7 UNITED ST6 1995 65020671 +UNITED ST3 UNITED ST1 1995 64762815 +UNITED ST1 UNITED ST1 1995 64199332 +UNITED ST6 UNITED ST9 1995 63457105 +UNITED ST0 UNITED ST9 1995 63253499 +UNITED ST5 UNITED ST9 1995 62364897 +UNITED ST4 UNITED ST5 1995 62081505 +UNITED ST1 UNITED ST7 1995 61515741 +UNITED ST2 UNITED ST7 1995 60000208 +UNITED ST8 UNITED ST6 1995 58990404 +UNITED ST1 UNITED ST4 1995 57535599 +UNITED ST0 UNITED ST3 1995 56360708 +UNITED ST8 UNITED ST0 1995 55231664 +UNITED ST5 UNITED ST7 1995 53740759 +UNITED ST2 UNITED ST0 1995 52560838 +UNITED ST3 UNITED ST7 1995 51579548 +UNITED ST9 UNITED ST0 1995 51548621 +UNITED ST6 UNITED ST6 1995 50954677 +UNITED ST5 UNITED ST5 1995 50394007 +UNITED ST7 UNITED ST4 1995 49899545 +UNITED ST3 UNITED ST3 1995 48158089 +UNITED ST6 UNITED ST4 1995 47697241 +UNITED ST9 UNITED ST7 1995 46464589 +UNITED ST4 UNITED ST0 1995 46142899 +UNITED ST1 UNITED ST3 1995 45823056 +UNITED ST2 UNITED ST1 1995 45437340 +UNITED ST6 UNITED ST5 1995 45428152 +UNITED ST8 UNITED ST5 1995 44994863 +UNITED ST7 UNITED ST3 1995 43699734 +UNITED ST9 UNITED ST6 1995 43482494 +UNITED ST2 UNITED ST5 1995 43382417 +UNITED ST6 UNITED ST8 1995 41973920 +UNITED ST0 UNITED ST6 1995 40718942 +UNITED ST3 UNITED ST5 1995 40057815 +UNITED ST7 UNITED ST5 1995 39015293 +UNITED ST0 UNITED ST0 1995 38984502 +UNITED ST6 UNITED ST1 1995 38328352 +UNITED ST4 UNITED ST6 1995 38109524 +UNITED ST3 UNITED ST9 1995 37364623 +UNITED ST4 UNITED ST3 1995 37356396 +UNITED ST2 UNITED ST6 1995 37146288 +UNITED ST9 UNITED ST5 1995 36009924 +UNITED ST1 UNITED ST8 1995 34918793 +UNITED ST8 UNITED ST1 1995 34217780 +UNITED ST0 UNITED ST7 1995 33395313 +UNITED ST1 UNITED ST5 1995 31857452 +UNITED ST8 UNITED ST7 1995 30918202 +UNITED ST2 UNITED ST8 1995 30504189 +UNITED ST3 UNITED ST4 1995 30422346 +UNITED ST5 UNITED ST6 1995 30204737 +UNITED ST8 UNITED ST3 1995 29349938 +UNITED ST0 UNITED ST4 1995 29170038 +UNITED ST1 UNITED ST6 1995 27139403 +UNITED ST7 UNITED ST8 1995 26444490 +UNITED ST6 UNITED ST7 1995 23833977 +UNITED ST4 UNITED ST8 1995 23053236 +UNITED ST4 UNITED ST2 1995 22787003 +UNITED ST3 UNITED ST8 1995 22505239 +UNITED ST9 UNITED ST8 1995 21292527 +UNITED ST8 UNITED ST4 1995 19287311 +UNITED ST8 UNITED ST2 1995 18282990 +UNITED ST9 UNITED ST3 1995 17539102 +UNITED ST3 UNITED ST6 1995 16738530 +UNITED ST0 UNITED ST8 1995 16393249 +UNITED ST5 UNITED ST2 1995 14951023 +UNITED ST2 UNITED ST2 1995 13455031 +UNITED ST5 UNITED ST8 1995 13346932 +UNITED ST3 UNITED ST2 1995 12886829 +UNITED ST9 UNITED ST2 1995 12653362 +UNITED ST0 UNITED ST5 1995 11453709 +UNITED ST6 UNITED ST2 1995 11363551 +UNITED ST8 UNITED ST8 1995 11285867 +UNITED ST1 UNITED ST2 1995 7991676 +UNITED ST0 UNITED ST2 1995 4776460 +UNITED ST7 UNITED ST2 1995 3973565 +UNITED ST6 UNITED ST0 1996 132286763 +UNITED ST4 UNITED ST0 1996 106897563 +UNITED ST5 UNITED ST0 1996 104501979 +UNITED ST0 UNITED ST4 1996 103228749 +UNITED ST9 UNITED ST1 1996 101983827 +UNITED ST1 UNITED ST0 1996 94861554 +UNITED ST0 UNITED ST1 1996 91948777 +UNITED ST6 UNITED ST1 1996 89705870 +UNITED ST2 UNITED ST1 1996 88256155 +UNITED ST1 UNITED ST1 1996 87265848 +UNITED ST4 UNITED ST7 1996 83677814 +UNITED ST6 UNITED ST3 1996 82168229 +UNITED ST2 UNITED ST7 1996 80468477 +UNITED ST8 UNITED ST1 1996 78971756 +UNITED ST7 UNITED ST4 1996 77941011 +UNITED ST1 UNITED ST3 1996 77610905 +UNITED ST9 UNITED ST9 1996 77301740 +UNITED ST0 UNITED ST9 1996 75935209 +UNITED ST7 UNITED ST0 1996 75602445 +UNITED ST3 UNITED ST1 1996 74746699 +UNITED ST7 UNITED ST7 1996 74040644 +UNITED ST4 UNITED ST1 1996 72819867 +UNITED ST5 UNITED ST9 1996 72614114 +UNITED ST1 UNITED ST9 1996 69823769 +UNITED ST9 UNITED ST4 1996 69616614 +UNITED ST6 UNITED ST4 1996 69296005 +UNITED ST2 UNITED ST0 1996 67095122 +UNITED ST5 UNITED ST1 1996 65346389 +UNITED ST6 UNITED ST9 1996 63256182 +UNITED ST4 UNITED ST6 1996 62189328 +UNITED ST0 UNITED ST0 1996 61528855 +UNITED ST8 UNITED ST9 1996 60466691 +UNITED ST3 UNITED ST0 1996 60226840 +UNITED ST8 UNITED ST6 1996 59746655 +UNITED ST2 UNITED ST3 1996 59648598 +UNITED ST0 UNITED ST5 1996 57762895 +UNITED ST8 UNITED ST7 1996 57523925 +UNITED ST2 UNITED ST4 1996 57487057 +UNITED ST6 UNITED ST7 1996 57090619 +UNITED ST3 UNITED ST4 1996 54838259 +UNITED ST4 UNITED ST9 1996 54134840 +UNITED ST1 UNITED ST6 1996 54056366 +UNITED ST3 UNITED ST7 1996 53987218 +UNITED ST2 UNITED ST9 1996 53491415 +UNITED ST4 UNITED ST8 1996 53035662 +UNITED ST9 UNITED ST5 1996 52825922 +UNITED ST7 UNITED ST3 1996 52356886 +UNITED ST6 UNITED ST8 1996 52136784 +UNITED ST1 UNITED ST5 1996 51189649 +UNITED ST8 UNITED ST4 1996 51033054 +UNITED ST1 UNITED ST4 1996 50135547 +UNITED ST3 UNITED ST9 1996 49612304 +UNITED ST9 UNITED ST0 1996 49590701 +UNITED ST3 UNITED ST3 1996 49263796 +UNITED ST9 UNITED ST7 1996 48798759 +UNITED ST7 UNITED ST1 1996 48784175 +UNITED ST9 UNITED ST6 1996 48685063 +UNITED ST2 UNITED ST5 1996 47028976 +UNITED ST9 UNITED ST3 1996 47021018 +UNITED ST2 UNITED ST8 1996 46865113 +UNITED ST4 UNITED ST3 1996 46460940 +UNITED ST6 UNITED ST5 1996 46030104 +UNITED ST2 UNITED ST6 1996 45416649 +UNITED ST8 UNITED ST5 1996 43480473 +UNITED ST5 UNITED ST5 1996 43020722 +UNITED ST6 UNITED ST6 1996 42923032 +UNITED ST4 UNITED ST5 1996 42808412 +UNITED ST3 UNITED ST6 1996 41881628 +UNITED ST1 UNITED ST7 1996 41721506 +UNITED ST5 UNITED ST7 1996 40946330 +UNITED ST8 UNITED ST0 1996 39845427 +UNITED ST8 UNITED ST3 1996 39535940 +UNITED ST1 UNITED ST8 1996 39430882 +UNITED ST5 UNITED ST6 1996 38332044 +UNITED ST0 UNITED ST6 1996 37232062 +UNITED ST7 UNITED ST6 1996 36535628 +UNITED ST9 UNITED ST8 1996 36112301 +UNITED ST3 UNITED ST8 1996 33924732 +UNITED ST3 UNITED ST5 1996 33717439 +UNITED ST5 UNITED ST4 1996 33424859 +UNITED ST5 UNITED ST3 1996 32919926 +UNITED ST4 UNITED ST4 1996 32787995 +UNITED ST7 UNITED ST5 1996 32784224 +UNITED ST5 UNITED ST8 1996 32415682 +UNITED ST0 UNITED ST7 1996 31391181 +UNITED ST0 UNITED ST3 1996 29795665 +UNITED ST7 UNITED ST9 1996 25028091 +UNITED ST7 UNITED ST8 1996 24348851 +UNITED ST6 UNITED ST2 1996 20447847 +UNITED ST8 UNITED ST8 1996 16340603 +UNITED ST0 UNITED ST8 1996 15168262 +UNITED ST4 UNITED ST2 1996 12451867 +UNITED ST1 UNITED ST2 1996 10082536 +UNITED ST5 UNITED ST2 1996 8544233 +UNITED ST7 UNITED ST2 1996 8390470 +UNITED ST9 UNITED ST2 1996 8108035 +UNITED ST8 UNITED ST2 1996 7028334 +UNITED ST0 UNITED ST2 1996 6907199 +UNITED ST3 UNITED ST2 1996 4311800 +UNITED ST5 UNITED ST0 1997 121457766 +UNITED ST0 UNITED ST9 1997 119856268 +UNITED ST5 UNITED ST1 1997 106900161 +UNITED ST9 UNITED ST9 1997 106222795 +UNITED ST5 UNITED ST4 1997 103432927 +UNITED ST7 UNITED ST1 1997 101326940 +UNITED ST5 UNITED ST9 1997 97392640 +UNITED ST8 UNITED ST0 1997 89434845 +UNITED ST1 UNITED ST3 1997 85175638 +UNITED ST4 UNITED ST1 1997 84611188 +UNITED ST4 UNITED ST7 1997 84606996 +UNITED ST1 UNITED ST4 1997 83092569 +UNITED ST0 UNITED ST1 1997 80167139 +UNITED ST4 UNITED ST9 1997 79697927 +UNITED ST1 UNITED ST0 1997 77667866 +UNITED ST2 UNITED ST0 1997 77591470 +UNITED ST1 UNITED ST7 1997 77045722 +UNITED ST2 UNITED ST7 1997 76483802 +UNITED ST1 UNITED ST1 1997 76340444 +UNITED ST0 UNITED ST7 1997 72577490 +UNITED ST4 UNITED ST0 1997 69875804 +UNITED ST6 UNITED ST3 1997 69725506 +UNITED ST2 UNITED ST9 1997 69543984 +UNITED ST7 UNITED ST0 1997 69354312 +UNITED ST9 UNITED ST7 1997 69052630 +UNITED ST8 UNITED ST3 1997 66918394 +UNITED ST0 UNITED ST0 1997 66308516 +UNITED ST9 UNITED ST0 1997 66066791 +UNITED ST2 UNITED ST3 1997 65429453 +UNITED ST2 UNITED ST1 1997 65396135 +UNITED ST4 UNITED ST4 1997 62532346 +UNITED ST9 UNITED ST1 1997 62418406 +UNITED ST3 UNITED ST7 1997 61095300 +UNITED ST7 UNITED ST4 1997 60123494 +UNITED ST4 UNITED ST3 1997 59634153 +UNITED ST3 UNITED ST4 1997 58356141 +UNITED ST6 UNITED ST9 1997 58231263 +UNITED ST6 UNITED ST4 1997 57215022 +UNITED ST2 UNITED ST4 1997 57012611 +UNITED ST5 UNITED ST3 1997 56789932 +UNITED ST9 UNITED ST5 1997 56091920 +UNITED ST6 UNITED ST0 1997 55824919 +UNITED ST6 UNITED ST1 1997 55117709 +UNITED ST7 UNITED ST9 1997 55039970 +UNITED ST5 UNITED ST7 1997 54534315 +UNITED ST3 UNITED ST1 1997 54150683 +UNITED ST3 UNITED ST0 1997 54050172 +UNITED ST3 UNITED ST9 1997 51370308 +UNITED ST1 UNITED ST5 1997 51052110 +UNITED ST6 UNITED ST7 1997 49795849 +UNITED ST5 UNITED ST6 1997 49030016 +UNITED ST1 UNITED ST9 1997 48944111 +UNITED ST3 UNITED ST3 1997 48385233 +UNITED ST0 UNITED ST5 1997 47988418 +UNITED ST2 UNITED ST8 1997 46391909 +UNITED ST4 UNITED ST6 1997 46236849 +UNITED ST9 UNITED ST3 1997 46121724 +UNITED ST8 UNITED ST9 1997 45852796 +UNITED ST7 UNITED ST8 1997 45476266 +UNITED ST8 UNITED ST1 1997 43284139 +UNITED ST7 UNITED ST7 1997 42566603 +UNITED ST0 UNITED ST4 1997 41193803 +UNITED ST8 UNITED ST7 1997 40878429 +UNITED ST8 UNITED ST8 1997 40749400 +UNITED ST2 UNITED ST5 1997 40662886 +UNITED ST0 UNITED ST3 1997 40078508 +UNITED ST6 UNITED ST8 1997 37988445 +UNITED ST9 UNITED ST6 1997 37596882 +UNITED ST6 UNITED ST5 1997 37571788 +UNITED ST2 UNITED ST2 1997 37034333 +UNITED ST7 UNITED ST5 1997 36889736 +UNITED ST8 UNITED ST4 1997 36600584 +UNITED ST3 UNITED ST6 1997 36075844 +UNITED ST7 UNITED ST3 1997 35168922 +UNITED ST2 UNITED ST6 1997 34274467 +UNITED ST5 UNITED ST8 1997 32910946 +UNITED ST3 UNITED ST5 1997 32811963 +UNITED ST4 UNITED ST8 1997 32599736 +UNITED ST5 UNITED ST5 1997 32568888 +UNITED ST4 UNITED ST5 1997 31995868 +UNITED ST9 UNITED ST4 1997 31765478 +UNITED ST7 UNITED ST2 1997 31409590 +UNITED ST9 UNITED ST8 1997 31272385 +UNITED ST0 UNITED ST8 1997 30241370 +UNITED ST3 UNITED ST8 1997 29102220 +UNITED ST7 UNITED ST6 1997 27539931 +UNITED ST8 UNITED ST5 1997 26673743 +UNITED ST6 UNITED ST6 1997 24983291 +UNITED ST0 UNITED ST2 1997 23584229 +UNITED ST0 UNITED ST6 1997 22924046 +UNITED ST1 UNITED ST2 1997 20599130 +UNITED ST8 UNITED ST6 1997 20073000 +UNITED ST1 UNITED ST6 1997 18205326 +UNITED ST1 UNITED ST8 1997 15375258 +UNITED ST6 UNITED ST2 1997 14746680 +UNITED ST5 UNITED ST2 1997 13863438 +UNITED ST8 UNITED ST2 1997 12053706 +UNITED ST3 UNITED ST2 1997 7891878 +UNITED ST4 UNITED ST2 1997 2848157 +UNITED ST9 UNITED ST2 1997 2661652 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q3.3.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q3.3.out new file mode 100644 index 00000000000000..96695e04c80746 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q3.3.out @@ -0,0 +1,27 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q3.3 -- +UNITED KI1 UNITED KI1 1992 46735995 +UNITED KI5 UNITED KI1 1992 36277055 +UNITED KI5 UNITED KI5 1992 25355267 +UNITED KI1 UNITED KI5 1992 21917846 +UNITED KI5 UNITED KI1 1993 61017607 +UNITED KI1 UNITED KI1 1993 45669535 +UNITED KI5 UNITED KI5 1993 34099392 +UNITED KI1 UNITED KI5 1993 21444206 +UNITED KI5 UNITED KI1 1994 36282163 +UNITED KI1 UNITED KI1 1994 34868441 +UNITED KI5 UNITED KI5 1994 34507284 +UNITED KI1 UNITED KI5 1994 21221780 +UNITED KI5 UNITED KI1 1995 82955896 +UNITED KI1 UNITED KI1 1995 35881186 +UNITED KI5 UNITED KI5 1995 20539805 +UNITED KI1 UNITED KI5 1995 17176510 +UNITED KI5 UNITED KI1 1996 65767049 +UNITED KI1 UNITED KI1 1996 59923037 +UNITED KI5 UNITED KI5 1996 46077342 +UNITED KI1 UNITED KI5 1996 13700254 +UNITED KI1 UNITED KI1 1997 70343133 +UNITED KI5 UNITED KI1 1997 64978359 +UNITED KI5 UNITED KI5 1997 27332027 +UNITED KI1 UNITED KI5 1997 16410668 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q3.4.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q3.4.out new file mode 100644 index 00000000000000..136aefa3508328 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q3.4.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q3.4 -- +UNITED KI5 UNITED KI1 1997 9117846 +UNITED KI5 UNITED KI5 1997 6203801 +UNITED KI1 UNITED KI5 1997 1870070 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q4.1.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q4.1.out new file mode 100644 index 00000000000000..485ce1aa922775 --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q4.1.out @@ -0,0 +1,38 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q4.1 -- +1992 ARGENTINA 9658964469 +1992 BRAZIL 9226983055 +1992 CANADA 9643176787 +1992 PERU 9410676597 +1992 UNITED STATES 9849427653 +1993 ARGENTINA 9976332853 +1993 BRAZIL 9468799229 +1993 CANADA 9897302420 +1993 PERU 9309445836 +1993 UNITED STATES 10003985086 +1994 ARGENTINA 9940305215 +1994 BRAZIL 9348651677 +1994 CANADA 9582647596 +1994 PERU 9295265013 +1994 UNITED STATES 9519880425 +1995 ARGENTINA 10143841380 +1995 BRAZIL 9156077350 +1995 CANADA 9562612160 +1995 PERU 9778087211 +1995 UNITED STATES 9310565244 +1996 ARGENTINA 10001927895 +1996 BRAZIL 9168485151 +1996 CANADA 10061604203 +1996 PERU 9355135674 +1996 UNITED STATES 9769712174 +1997 ARGENTINA 9854560261 +1997 BRAZIL 9121571047 +1997 CANADA 10097371778 +1997 PERU 9315525917 +1997 UNITED STATES 10506723879 +1998 ARGENTINA 5834240407 +1998 BRAZIL 5356398095 +1998 CANADA 5423211196 +1998 PERU 5726185970 +1998 UNITED STATES 6009462019 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q4.2.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q4.2.out new file mode 100644 index 00000000000000..600a018fb40d8f --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q4.2.out @@ -0,0 +1,103 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q4.2 -- +1997 ARGENTINA MFGR#11 907071566 +1997 ARGENTINA MFGR#12 924115562 +1997 ARGENTINA MFGR#13 972861821 +1997 ARGENTINA MFGR#14 975410345 +1997 ARGENTINA MFGR#15 938867375 +1997 ARGENTINA MFGR#21 1014782574 +1997 ARGENTINA MFGR#22 873016783 +1997 ARGENTINA MFGR#23 1030357302 +1997 ARGENTINA MFGR#24 893460579 +1997 ARGENTINA MFGR#25 1006311403 +1997 BRAZIL MFGR#11 1073219328 +1997 BRAZIL MFGR#12 989858833 +1997 BRAZIL MFGR#13 1128480379 +1997 BRAZIL MFGR#14 1194256722 +1997 BRAZIL MFGR#15 1094419124 +1997 BRAZIL MFGR#21 910026832 +1997 BRAZIL MFGR#22 993142048 +1997 BRAZIL MFGR#23 1107672874 +1997 BRAZIL MFGR#24 1058013649 +1997 BRAZIL MFGR#25 994733764 +1997 CANADA MFGR#11 854725020 +1997 CANADA MFGR#12 938718164 +1997 CANADA MFGR#13 959265890 +1997 CANADA MFGR#14 1002812450 +1997 CANADA MFGR#15 848183013 +1997 CANADA MFGR#21 999805272 +1997 CANADA MFGR#22 778419763 +1997 CANADA MFGR#23 928359645 +1997 CANADA MFGR#24 849895128 +1997 CANADA MFGR#25 904587965 +1997 PERU MFGR#11 1100242877 +1997 PERU MFGR#12 994365350 +1997 PERU MFGR#13 847486105 +1997 PERU MFGR#14 947769683 +1997 PERU MFGR#15 999395678 +1997 PERU MFGR#21 867923394 +1997 PERU MFGR#22 988747459 +1997 PERU MFGR#23 1066645086 +1997 PERU MFGR#24 935665725 +1997 PERU MFGR#25 981454129 +1997 UNITED STATES MFGR#11 1046706048 +1997 UNITED STATES MFGR#12 909213709 +1997 UNITED STATES MFGR#13 1096278906 +1997 UNITED STATES MFGR#14 934282111 +1997 UNITED STATES MFGR#15 962760843 +1997 UNITED STATES MFGR#21 1000676474 +1997 UNITED STATES MFGR#22 1076947615 +1997 UNITED STATES MFGR#23 937288102 +1997 UNITED STATES MFGR#24 1003386363 +1997 UNITED STATES MFGR#25 1053666052 +1998 ARGENTINA MFGR#11 567612227 +1998 ARGENTINA MFGR#12 527025042 +1998 ARGENTINA MFGR#13 582791792 +1998 ARGENTINA MFGR#14 523726368 +1998 ARGENTINA MFGR#15 522078267 +1998 ARGENTINA MFGR#21 504712685 +1998 ARGENTINA MFGR#22 506061736 +1998 ARGENTINA MFGR#23 560479801 +1998 ARGENTINA MFGR#24 524579118 +1998 ARGENTINA MFGR#25 547840211 +1998 BRAZIL MFGR#11 638578488 +1998 BRAZIL MFGR#12 646312681 +1998 BRAZIL MFGR#13 655161772 +1998 BRAZIL MFGR#14 552676170 +1998 BRAZIL MFGR#15 663812709 +1998 BRAZIL MFGR#21 668822448 +1998 BRAZIL MFGR#22 591791918 +1998 BRAZIL MFGR#23 690648877 +1998 BRAZIL MFGR#24 562362220 +1998 BRAZIL MFGR#25 704182376 +1998 CANADA MFGR#11 509086125 +1998 CANADA MFGR#12 488089772 +1998 CANADA MFGR#13 486533297 +1998 CANADA MFGR#14 527837227 +1998 CANADA MFGR#15 535869299 +1998 CANADA MFGR#21 455868651 +1998 CANADA MFGR#22 594277308 +1998 CANADA MFGR#23 574125070 +1998 CANADA MFGR#24 508530424 +1998 CANADA MFGR#25 547757992 +1998 PERU MFGR#11 567838547 +1998 PERU MFGR#12 540544757 +1998 PERU MFGR#13 591331883 +1998 PERU MFGR#14 481335064 +1998 PERU MFGR#15 570246138 +1998 PERU MFGR#21 533733330 +1998 PERU MFGR#22 527790656 +1998 PERU MFGR#23 636393221 +1998 PERU MFGR#24 589075262 +1998 PERU MFGR#25 543251115 +1998 UNITED STATES MFGR#11 556276232 +1998 UNITED STATES MFGR#12 612385982 +1998 UNITED STATES MFGR#13 622413927 +1998 UNITED STATES MFGR#14 555006887 +1998 UNITED STATES MFGR#15 525119569 +1998 UNITED STATES MFGR#21 563406836 +1998 UNITED STATES MFGR#22 580478735 +1998 UNITED STATES MFGR#23 656080465 +1998 UNITED STATES MFGR#24 538445058 +1998 UNITED STATES MFGR#25 589111952 + diff --git a/regression-test/data/s3_vault/multi_vault_p2/sql/q4.3.out b/regression-test/data/s3_vault/multi_vault_p2/sql/q4.3.out new file mode 100644 index 00000000000000..09c8399d008f6b --- /dev/null +++ b/regression-test/data/s3_vault/multi_vault_p2/sql/q4.3.out @@ -0,0 +1,697 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q4.3 -- +1997 UNITED ST0 MFGR#141 15456288 +1997 UNITED ST0 MFGR#1410 32961113 +1997 UNITED ST0 MFGR#1411 36166400 +1997 UNITED ST0 MFGR#1412 18237451 +1997 UNITED ST0 MFGR#1413 40752813 +1997 UNITED ST0 MFGR#1414 24237588 +1997 UNITED ST0 MFGR#1415 6419001 +1997 UNITED ST0 MFGR#1416 15639876 +1997 UNITED ST0 MFGR#1417 15476211 +1997 UNITED ST0 MFGR#1418 8169956 +1997 UNITED ST0 MFGR#1419 18217252 +1997 UNITED ST0 MFGR#142 29361222 +1997 UNITED ST0 MFGR#1420 34587603 +1997 UNITED ST0 MFGR#1421 28560921 +1997 UNITED ST0 MFGR#1422 25154181 +1997 UNITED ST0 MFGR#1423 18144008 +1997 UNITED ST0 MFGR#1424 28064267 +1997 UNITED ST0 MFGR#1425 15255845 +1997 UNITED ST0 MFGR#1426 5610353 +1997 UNITED ST0 MFGR#1427 5750693 +1997 UNITED ST0 MFGR#1428 14081707 +1997 UNITED ST0 MFGR#1429 9024512 +1997 UNITED ST0 MFGR#143 5325193 +1997 UNITED ST0 MFGR#1430 16961739 +1997 UNITED ST0 MFGR#1431 19015253 +1997 UNITED ST0 MFGR#1432 6324917 +1997 UNITED ST0 MFGR#1433 6734461 +1997 UNITED ST0 MFGR#1434 19206393 +1997 UNITED ST0 MFGR#1435 679925 +1997 UNITED ST0 MFGR#1436 13834012 +1997 UNITED ST0 MFGR#1437 23575407 +1997 UNITED ST0 MFGR#1438 24756074 +1997 UNITED ST0 MFGR#1439 19644072 +1997 UNITED ST0 MFGR#144 20285157 +1997 UNITED ST0 MFGR#1440 14068558 +1997 UNITED ST0 MFGR#145 22646138 +1997 UNITED ST0 MFGR#146 16381159 +1997 UNITED ST0 MFGR#147 12267827 +1997 UNITED ST0 MFGR#148 30968614 +1997 UNITED ST0 MFGR#149 8640669 +1997 UNITED ST1 MFGR#141 10152087 +1997 UNITED ST1 MFGR#1410 32583300 +1997 UNITED ST1 MFGR#1411 15690352 +1997 UNITED ST1 MFGR#1412 16671403 +1997 UNITED ST1 MFGR#1413 4942775 +1997 UNITED ST1 MFGR#1414 9391597 +1997 UNITED ST1 MFGR#1415 26937296 +1997 UNITED ST1 MFGR#1416 25782871 +1997 UNITED ST1 MFGR#1417 23349641 +1997 UNITED ST1 MFGR#1418 10934620 +1997 UNITED ST1 MFGR#1419 22673030 +1997 UNITED ST1 MFGR#142 14048276 +1997 UNITED ST1 MFGR#1420 18100795 +1997 UNITED ST1 MFGR#1421 25405678 +1997 UNITED ST1 MFGR#1423 5713800 +1997 UNITED ST1 MFGR#1424 4131923 +1997 UNITED ST1 MFGR#1425 17190425 +1997 UNITED ST1 MFGR#1426 22314044 +1997 UNITED ST1 MFGR#1427 13635057 +1997 UNITED ST1 MFGR#1428 17735811 +1997 UNITED ST1 MFGR#1429 28740911 +1997 UNITED ST1 MFGR#143 27418033 +1997 UNITED ST1 MFGR#1430 18655292 +1997 UNITED ST1 MFGR#1431 11074399 +1997 UNITED ST1 MFGR#1432 7490170 +1997 UNITED ST1 MFGR#1433 28997266 +1997 UNITED ST1 MFGR#1434 4612211 +1997 UNITED ST1 MFGR#1435 16327124 +1997 UNITED ST1 MFGR#1436 17409665 +1997 UNITED ST1 MFGR#1437 33391635 +1997 UNITED ST1 MFGR#1438 37321134 +1997 UNITED ST1 MFGR#1439 10279941 +1997 UNITED ST1 MFGR#144 12304723 +1997 UNITED ST1 MFGR#1440 28921230 +1997 UNITED ST1 MFGR#145 17933684 +1997 UNITED ST1 MFGR#146 8709103 +1997 UNITED ST1 MFGR#147 20145983 +1997 UNITED ST1 MFGR#148 16940400 +1997 UNITED ST1 MFGR#149 10284949 +1997 UNITED ST2 MFGR#1412 6328240 +1997 UNITED ST2 MFGR#1415 5022682 +1997 UNITED ST2 MFGR#1416 4102214 +1997 UNITED ST2 MFGR#1417 5284444 +1997 UNITED ST2 MFGR#1418 2649908 +1997 UNITED ST2 MFGR#1419 4664673 +1997 UNITED ST2 MFGR#142 10784835 +1997 UNITED ST2 MFGR#1423 4095190 +1997 UNITED ST2 MFGR#1424 5376355 +1997 UNITED ST2 MFGR#1425 3949957 +1997 UNITED ST2 MFGR#143 2859829 +1997 UNITED ST2 MFGR#1430 4618266 +1997 UNITED ST2 MFGR#1432 5183060 +1997 UNITED ST2 MFGR#1433 10531371 +1997 UNITED ST2 MFGR#1434 77646 +1997 UNITED ST2 MFGR#1436 6538508 +1997 UNITED ST2 MFGR#1438 12324747 +1997 UNITED ST2 MFGR#144 2002870 +1997 UNITED ST2 MFGR#1440 12247229 +1997 UNITED ST2 MFGR#145 202685 +1997 UNITED ST2 MFGR#148 5998932 +1997 UNITED ST3 MFGR#141 12866083 +1997 UNITED ST3 MFGR#1410 31051703 +1997 UNITED ST3 MFGR#1411 14099543 +1997 UNITED ST3 MFGR#1412 4970551 +1997 UNITED ST3 MFGR#1413 26129762 +1997 UNITED ST3 MFGR#1414 22364787 +1997 UNITED ST3 MFGR#1415 27883428 +1997 UNITED ST3 MFGR#1416 25289630 +1997 UNITED ST3 MFGR#1417 5524810 +1997 UNITED ST3 MFGR#1419 3616608 +1997 UNITED ST3 MFGR#142 10776059 +1997 UNITED ST3 MFGR#1420 12958851 +1997 UNITED ST3 MFGR#1421 5815494 +1997 UNITED ST3 MFGR#1422 17694270 +1997 UNITED ST3 MFGR#1423 10285142 +1997 UNITED ST3 MFGR#1424 21555317 +1997 UNITED ST3 MFGR#1425 5214517 +1997 UNITED ST3 MFGR#1426 16810700 +1997 UNITED ST3 MFGR#1427 10086430 +1997 UNITED ST3 MFGR#1428 7733716 +1997 UNITED ST3 MFGR#1429 11922430 +1997 UNITED ST3 MFGR#1430 20105292 +1997 UNITED ST3 MFGR#1431 14716979 +1997 UNITED ST3 MFGR#1432 15779247 +1997 UNITED ST3 MFGR#1433 15308773 +1997 UNITED ST3 MFGR#1434 8515037 +1997 UNITED ST3 MFGR#1435 25486899 +1997 UNITED ST3 MFGR#1436 12739448 +1997 UNITED ST3 MFGR#1437 11249180 +1997 UNITED ST3 MFGR#1438 4156824 +1997 UNITED ST3 MFGR#1439 18358277 +1997 UNITED ST3 MFGR#144 10097718 +1997 UNITED ST3 MFGR#1440 22920494 +1997 UNITED ST3 MFGR#145 20285859 +1997 UNITED ST3 MFGR#146 18406423 +1997 UNITED ST3 MFGR#147 17295990 +1997 UNITED ST3 MFGR#148 26127145 +1997 UNITED ST3 MFGR#149 19895538 +1997 UNITED ST4 MFGR#141 8135477 +1997 UNITED ST4 MFGR#1410 5503247 +1997 UNITED ST4 MFGR#1411 6096435 +1997 UNITED ST4 MFGR#1412 19976043 +1997 UNITED ST4 MFGR#1413 22581282 +1997 UNITED ST4 MFGR#1414 10101379 +1997 UNITED ST4 MFGR#1415 8877444 +1997 UNITED ST4 MFGR#1416 17943398 +1997 UNITED ST4 MFGR#1417 9224501 +1997 UNITED ST4 MFGR#1418 16255176 +1997 UNITED ST4 MFGR#1419 683069 +1997 UNITED ST4 MFGR#142 23512944 +1997 UNITED ST4 MFGR#1420 27954262 +1997 UNITED ST4 MFGR#1421 54209185 +1997 UNITED ST4 MFGR#1422 1173957 +1997 UNITED ST4 MFGR#1423 20410710 +1997 UNITED ST4 MFGR#1424 2889270 +1997 UNITED ST4 MFGR#1425 13361796 +1997 UNITED ST4 MFGR#1426 18678819 +1997 UNITED ST4 MFGR#1427 18206066 +1997 UNITED ST4 MFGR#1428 18153974 +1997 UNITED ST4 MFGR#1429 12545076 +1997 UNITED ST4 MFGR#143 6158834 +1997 UNITED ST4 MFGR#1430 16128370 +1997 UNITED ST4 MFGR#1431 11153717 +1997 UNITED ST4 MFGR#1432 7745636 +1997 UNITED ST4 MFGR#1433 15983634 +1997 UNITED ST4 MFGR#1434 3662246 +1997 UNITED ST4 MFGR#1435 3060180 +1997 UNITED ST4 MFGR#1436 11524592 +1997 UNITED ST4 MFGR#1437 36824174 +1997 UNITED ST4 MFGR#1438 8226666 +1997 UNITED ST4 MFGR#1439 9872470 +1997 UNITED ST4 MFGR#1440 401207 +1997 UNITED ST4 MFGR#145 11698312 +1997 UNITED ST4 MFGR#146 18259956 +1997 UNITED ST4 MFGR#147 10256110 +1997 UNITED ST4 MFGR#148 15892628 +1997 UNITED ST4 MFGR#149 4322491 +1997 UNITED ST5 MFGR#141 8534275 +1997 UNITED ST5 MFGR#1411 2427261 +1997 UNITED ST5 MFGR#1412 9032422 +1997 UNITED ST5 MFGR#1413 6343588 +1997 UNITED ST5 MFGR#1414 2961025 +1997 UNITED ST5 MFGR#1415 21186568 +1997 UNITED ST5 MFGR#1416 9354080 +1997 UNITED ST5 MFGR#1417 11786246 +1997 UNITED ST5 MFGR#1418 15297245 +1997 UNITED ST5 MFGR#1420 15787608 +1997 UNITED ST5 MFGR#1421 17858301 +1997 UNITED ST5 MFGR#1422 14080987 +1997 UNITED ST5 MFGR#1423 12535186 +1997 UNITED ST5 MFGR#1424 3221960 +1997 UNITED ST5 MFGR#1425 8950317 +1997 UNITED ST5 MFGR#1426 13161486 +1997 UNITED ST5 MFGR#1427 6149737 +1997 UNITED ST5 MFGR#1428 6796520 +1997 UNITED ST5 MFGR#1429 26026300 +1997 UNITED ST5 MFGR#143 6712572 +1997 UNITED ST5 MFGR#1430 16092131 +1997 UNITED ST5 MFGR#1431 3425291 +1997 UNITED ST5 MFGR#1432 14630175 +1997 UNITED ST5 MFGR#1433 4967585 +1997 UNITED ST5 MFGR#1434 6716261 +1997 UNITED ST5 MFGR#1435 6290790 +1997 UNITED ST5 MFGR#1436 6787699 +1997 UNITED ST5 MFGR#1437 6045168 +1997 UNITED ST5 MFGR#1438 1082877 +1997 UNITED ST5 MFGR#1439 5268611 +1997 UNITED ST5 MFGR#144 8900840 +1997 UNITED ST5 MFGR#1440 2644336 +1997 UNITED ST5 MFGR#145 18054844 +1997 UNITED ST5 MFGR#146 4893668 +1997 UNITED ST5 MFGR#147 9615891 +1997 UNITED ST5 MFGR#148 1426124 +1997 UNITED ST5 MFGR#149 7320726 +1997 UNITED ST6 MFGR#141 10051337 +1997 UNITED ST6 MFGR#1410 23352963 +1997 UNITED ST6 MFGR#1411 1537303 +1997 UNITED ST6 MFGR#1412 4332646 +1997 UNITED ST6 MFGR#1413 14686752 +1997 UNITED ST6 MFGR#1414 7226409 +1997 UNITED ST6 MFGR#1415 15062686 +1997 UNITED ST6 MFGR#1416 8606480 +1997 UNITED ST6 MFGR#1417 7662761 +1997 UNITED ST6 MFGR#1418 1810860 +1997 UNITED ST6 MFGR#1419 20179568 +1997 UNITED ST6 MFGR#142 15506072 +1997 UNITED ST6 MFGR#1420 3844271 +1997 UNITED ST6 MFGR#1421 10608709 +1997 UNITED ST6 MFGR#1422 16083699 +1997 UNITED ST6 MFGR#1423 6789571 +1997 UNITED ST6 MFGR#1424 17631950 +1997 UNITED ST6 MFGR#1425 8477221 +1997 UNITED ST6 MFGR#1426 16847029 +1997 UNITED ST6 MFGR#1428 17314796 +1997 UNITED ST6 MFGR#1429 19175871 +1997 UNITED ST6 MFGR#143 8775848 +1997 UNITED ST6 MFGR#1431 5319063 +1997 UNITED ST6 MFGR#1432 7875606 +1997 UNITED ST6 MFGR#1433 9305264 +1997 UNITED ST6 MFGR#1434 8050404 +1997 UNITED ST6 MFGR#1435 17585836 +1997 UNITED ST6 MFGR#1436 13745758 +1997 UNITED ST6 MFGR#1438 14536324 +1997 UNITED ST6 MFGR#1439 11451329 +1997 UNITED ST6 MFGR#144 10794837 +1997 UNITED ST6 MFGR#1440 5719238 +1997 UNITED ST6 MFGR#145 25449058 +1997 UNITED ST6 MFGR#146 17821996 +1997 UNITED ST6 MFGR#147 18922333 +1997 UNITED ST6 MFGR#148 10825718 +1997 UNITED ST6 MFGR#149 14950868 +1997 UNITED ST7 MFGR#141 8291476 +1997 UNITED ST7 MFGR#1410 4622953 +1997 UNITED ST7 MFGR#1411 14885507 +1997 UNITED ST7 MFGR#1412 19011375 +1997 UNITED ST7 MFGR#1413 31178370 +1997 UNITED ST7 MFGR#1414 16429613 +1997 UNITED ST7 MFGR#1415 3091832 +1997 UNITED ST7 MFGR#1416 17008243 +1997 UNITED ST7 MFGR#1417 15609283 +1997 UNITED ST7 MFGR#1418 5059394 +1997 UNITED ST7 MFGR#1419 7234721 +1997 UNITED ST7 MFGR#142 6898938 +1997 UNITED ST7 MFGR#1420 6426590 +1997 UNITED ST7 MFGR#1421 6043221 +1997 UNITED ST7 MFGR#1422 13273021 +1997 UNITED ST7 MFGR#1423 30537249 +1997 UNITED ST7 MFGR#1424 9224296 +1997 UNITED ST7 MFGR#1425 12799069 +1997 UNITED ST7 MFGR#1426 30277891 +1997 UNITED ST7 MFGR#1427 8331493 +1997 UNITED ST7 MFGR#1428 16205423 +1997 UNITED ST7 MFGR#1429 14396627 +1997 UNITED ST7 MFGR#143 10209623 +1997 UNITED ST7 MFGR#1430 16556758 +1997 UNITED ST7 MFGR#1431 15775327 +1997 UNITED ST7 MFGR#1432 24602989 +1997 UNITED ST7 MFGR#1433 22927717 +1997 UNITED ST7 MFGR#1434 9486462 +1997 UNITED ST7 MFGR#1435 14254290 +1997 UNITED ST7 MFGR#1436 8645365 +1997 UNITED ST7 MFGR#1437 19026166 +1997 UNITED ST7 MFGR#1438 11291550 +1997 UNITED ST7 MFGR#1439 9590292 +1997 UNITED ST7 MFGR#144 14110404 +1997 UNITED ST7 MFGR#1440 21877355 +1997 UNITED ST7 MFGR#145 8246717 +1997 UNITED ST7 MFGR#146 9785781 +1997 UNITED ST7 MFGR#147 13412691 +1997 UNITED ST7 MFGR#148 14235939 +1997 UNITED ST7 MFGR#149 23450309 +1997 UNITED ST8 MFGR#141 7323416 +1997 UNITED ST8 MFGR#1410 7191961 +1997 UNITED ST8 MFGR#1411 2425849 +1997 UNITED ST8 MFGR#1412 26725641 +1997 UNITED ST8 MFGR#1413 7915896 +1997 UNITED ST8 MFGR#1414 14359044 +1997 UNITED ST8 MFGR#1415 15834927 +1997 UNITED ST8 MFGR#1416 7464470 +1997 UNITED ST8 MFGR#1417 15274496 +1997 UNITED ST8 MFGR#1418 3261391 +1997 UNITED ST8 MFGR#1419 8020386 +1997 UNITED ST8 MFGR#142 5985550 +1997 UNITED ST8 MFGR#1420 9441461 +1997 UNITED ST8 MFGR#1421 2998735 +1997 UNITED ST8 MFGR#1423 1565415 +1997 UNITED ST8 MFGR#1424 5613094 +1997 UNITED ST8 MFGR#1425 7186734 +1997 UNITED ST8 MFGR#1426 3153117 +1997 UNITED ST8 MFGR#1427 11303043 +1997 UNITED ST8 MFGR#1428 3564003 +1997 UNITED ST8 MFGR#143 6357477 +1997 UNITED ST8 MFGR#1430 6633950 +1997 UNITED ST8 MFGR#1431 867285 +1997 UNITED ST8 MFGR#1432 9962884 +1997 UNITED ST8 MFGR#1433 13708011 +1997 UNITED ST8 MFGR#1434 7357067 +1997 UNITED ST8 MFGR#1435 11993453 +1997 UNITED ST8 MFGR#1436 7231938 +1997 UNITED ST8 MFGR#1437 18632056 +1997 UNITED ST8 MFGR#1438 3609384 +1997 UNITED ST8 MFGR#144 4822734 +1997 UNITED ST8 MFGR#1440 2521917 +1997 UNITED ST8 MFGR#145 11683385 +1997 UNITED ST8 MFGR#146 3350415 +1997 UNITED ST8 MFGR#147 8425393 +1997 UNITED ST8 MFGR#148 15005341 +1997 UNITED ST8 MFGR#149 9018101 +1997 UNITED ST9 MFGR#141 21838850 +1997 UNITED ST9 MFGR#1410 8455387 +1997 UNITED ST9 MFGR#1411 27984689 +1997 UNITED ST9 MFGR#1412 26491401 +1997 UNITED ST9 MFGR#1413 30237705 +1997 UNITED ST9 MFGR#1414 8957476 +1997 UNITED ST9 MFGR#1415 35391583 +1997 UNITED ST9 MFGR#1416 8923782 +1997 UNITED ST9 MFGR#1417 3210984 +1997 UNITED ST9 MFGR#1418 12102607 +1997 UNITED ST9 MFGR#1419 14468627 +1997 UNITED ST9 MFGR#142 33363582 +1997 UNITED ST9 MFGR#1420 13139731 +1997 UNITED ST9 MFGR#1421 8410650 +1997 UNITED ST9 MFGR#1422 11268410 +1997 UNITED ST9 MFGR#1423 2846614 +1997 UNITED ST9 MFGR#1424 2305872 +1997 UNITED ST9 MFGR#1425 4896060 +1997 UNITED ST9 MFGR#1426 31544772 +1997 UNITED ST9 MFGR#1427 4113085 +1997 UNITED ST9 MFGR#1428 11137081 +1997 UNITED ST9 MFGR#1429 10633230 +1997 UNITED ST9 MFGR#143 11438203 +1997 UNITED ST9 MFGR#1430 2727389 +1997 UNITED ST9 MFGR#1431 10548538 +1997 UNITED ST9 MFGR#1433 26564495 +1997 UNITED ST9 MFGR#1434 11948206 +1997 UNITED ST9 MFGR#1435 2201909 +1997 UNITED ST9 MFGR#1436 10347631 +1997 UNITED ST9 MFGR#1437 40051690 +1997 UNITED ST9 MFGR#1438 2662945 +1997 UNITED ST9 MFGR#1439 18011455 +1997 UNITED ST9 MFGR#144 10202784 +1997 UNITED ST9 MFGR#1440 11123196 +1997 UNITED ST9 MFGR#145 11326624 +1997 UNITED ST9 MFGR#146 27825456 +1997 UNITED ST9 MFGR#147 17579502 +1997 UNITED ST9 MFGR#148 17001116 +1998 UNITED ST0 MFGR#141 12484786 +1998 UNITED ST0 MFGR#1410 5225999 +1998 UNITED ST0 MFGR#1411 2633395 +1998 UNITED ST0 MFGR#1412 9479536 +1998 UNITED ST0 MFGR#1413 18468711 +1998 UNITED ST0 MFGR#1414 19440601 +1998 UNITED ST0 MFGR#1415 16894643 +1998 UNITED ST0 MFGR#1416 7520117 +1998 UNITED ST0 MFGR#1417 464576 +1998 UNITED ST0 MFGR#1418 5171747 +1998 UNITED ST0 MFGR#1419 5970985 +1998 UNITED ST0 MFGR#142 23830409 +1998 UNITED ST0 MFGR#1420 2568835 +1998 UNITED ST0 MFGR#1421 6096442 +1998 UNITED ST0 MFGR#1422 10099456 +1998 UNITED ST0 MFGR#1423 24617746 +1998 UNITED ST0 MFGR#1424 22853888 +1998 UNITED ST0 MFGR#1425 25622112 +1998 UNITED ST0 MFGR#1426 9630701 +1998 UNITED ST0 MFGR#1427 4952625 +1998 UNITED ST0 MFGR#1428 4675471 +1998 UNITED ST0 MFGR#1429 19713527 +1998 UNITED ST0 MFGR#143 4971265 +1998 UNITED ST0 MFGR#1430 13948705 +1998 UNITED ST0 MFGR#1431 2955181 +1998 UNITED ST0 MFGR#1432 17946759 +1998 UNITED ST0 MFGR#1433 23325027 +1998 UNITED ST0 MFGR#1434 8580738 +1998 UNITED ST0 MFGR#1435 6663914 +1998 UNITED ST0 MFGR#1436 13468582 +1998 UNITED ST0 MFGR#1437 9419979 +1998 UNITED ST0 MFGR#1438 16164522 +1998 UNITED ST0 MFGR#1439 8692866 +1998 UNITED ST0 MFGR#144 17338711 +1998 UNITED ST0 MFGR#1440 8513736 +1998 UNITED ST0 MFGR#145 6212569 +1998 UNITED ST0 MFGR#146 2702349 +1998 UNITED ST0 MFGR#149 8844540 +1998 UNITED ST1 MFGR#141 13449110 +1998 UNITED ST1 MFGR#1410 5224626 +1998 UNITED ST1 MFGR#1411 16046399 +1998 UNITED ST1 MFGR#1412 8424392 +1998 UNITED ST1 MFGR#1413 7634982 +1998 UNITED ST1 MFGR#1414 8711782 +1998 UNITED ST1 MFGR#1415 5991201 +1998 UNITED ST1 MFGR#1416 11059645 +1998 UNITED ST1 MFGR#1417 8108087 +1998 UNITED ST1 MFGR#1418 1710875 +1998 UNITED ST1 MFGR#1419 452298 +1998 UNITED ST1 MFGR#142 6186526 +1998 UNITED ST1 MFGR#1421 4766963 +1998 UNITED ST1 MFGR#1422 5556563 +1998 UNITED ST1 MFGR#1423 12110257 +1998 UNITED ST1 MFGR#1424 8924948 +1998 UNITED ST1 MFGR#1425 5684710 +1998 UNITED ST1 MFGR#1426 6844444 +1998 UNITED ST1 MFGR#1427 7625525 +1998 UNITED ST1 MFGR#1428 5227395 +1998 UNITED ST1 MFGR#1429 2855272 +1998 UNITED ST1 MFGR#143 6964811 +1998 UNITED ST1 MFGR#1430 16952380 +1998 UNITED ST1 MFGR#1431 7655883 +1998 UNITED ST1 MFGR#1432 2975960 +1998 UNITED ST1 MFGR#1433 10175594 +1998 UNITED ST1 MFGR#1434 26358848 +1998 UNITED ST1 MFGR#1435 11408623 +1998 UNITED ST1 MFGR#1436 4438242 +1998 UNITED ST1 MFGR#1437 10014906 +1998 UNITED ST1 MFGR#1438 18817171 +1998 UNITED ST1 MFGR#1439 2129973 +1998 UNITED ST1 MFGR#144 9512727 +1998 UNITED ST1 MFGR#1440 20185179 +1998 UNITED ST1 MFGR#145 823605 +1998 UNITED ST1 MFGR#146 9358026 +1998 UNITED ST1 MFGR#147 6552312 +1998 UNITED ST1 MFGR#148 9144461 +1998 UNITED ST1 MFGR#149 2149639 +1998 UNITED ST2 MFGR#1410 8044368 +1998 UNITED ST2 MFGR#1413 727343 +1998 UNITED ST2 MFGR#1414 251088 +1998 UNITED ST2 MFGR#1416 5224696 +1998 UNITED ST2 MFGR#1419 7917142 +1998 UNITED ST2 MFGR#1420 9848986 +1998 UNITED ST2 MFGR#1427 4816976 +1998 UNITED ST2 MFGR#1429 4196069 +1998 UNITED ST2 MFGR#143 10073044 +1998 UNITED ST2 MFGR#1431 877123 +1998 UNITED ST2 MFGR#1433 3740836 +1998 UNITED ST2 MFGR#1434 857691 +1998 UNITED ST2 MFGR#1435 2277870 +1998 UNITED ST2 MFGR#145 4906521 +1998 UNITED ST2 MFGR#149 842348 +1998 UNITED ST3 MFGR#141 4181582 +1998 UNITED ST3 MFGR#1410 3169761 +1998 UNITED ST3 MFGR#1412 6590440 +1998 UNITED ST3 MFGR#1413 2015369 +1998 UNITED ST3 MFGR#1414 4408466 +1998 UNITED ST3 MFGR#1415 3963826 +1998 UNITED ST3 MFGR#1416 3179540 +1998 UNITED ST3 MFGR#1417 11440999 +1998 UNITED ST3 MFGR#1418 8725751 +1998 UNITED ST3 MFGR#1419 10646990 +1998 UNITED ST3 MFGR#142 5056172 +1998 UNITED ST3 MFGR#1420 3583150 +1998 UNITED ST3 MFGR#1421 7085689 +1998 UNITED ST3 MFGR#1422 20858904 +1998 UNITED ST3 MFGR#1423 12825743 +1998 UNITED ST3 MFGR#1424 3533518 +1998 UNITED ST3 MFGR#1425 4938324 +1998 UNITED ST3 MFGR#1426 15164664 +1998 UNITED ST3 MFGR#1427 2619162 +1998 UNITED ST3 MFGR#1428 7723699 +1998 UNITED ST3 MFGR#143 24362893 +1998 UNITED ST3 MFGR#1430 7139650 +1998 UNITED ST3 MFGR#1431 10306394 +1998 UNITED ST3 MFGR#1432 14769004 +1998 UNITED ST3 MFGR#1433 9610545 +1998 UNITED ST3 MFGR#1435 6611760 +1998 UNITED ST3 MFGR#1436 941348 +1998 UNITED ST3 MFGR#1437 4802162 +1998 UNITED ST3 MFGR#1438 5079750 +1998 UNITED ST3 MFGR#1439 5905118 +1998 UNITED ST3 MFGR#144 5436839 +1998 UNITED ST3 MFGR#1440 9815517 +1998 UNITED ST3 MFGR#145 7580271 +1998 UNITED ST3 MFGR#146 14051404 +1998 UNITED ST3 MFGR#147 7786017 +1998 UNITED ST3 MFGR#149 1271813 +1998 UNITED ST4 MFGR#141 26703183 +1998 UNITED ST4 MFGR#1410 18094129 +1998 UNITED ST4 MFGR#1411 615999 +1998 UNITED ST4 MFGR#1412 13988709 +1998 UNITED ST4 MFGR#1413 10115647 +1998 UNITED ST4 MFGR#1414 1586551 +1998 UNITED ST4 MFGR#1415 14357244 +1998 UNITED ST4 MFGR#1416 9430036 +1998 UNITED ST4 MFGR#1417 17361773 +1998 UNITED ST4 MFGR#1418 10187338 +1998 UNITED ST4 MFGR#1419 7116094 +1998 UNITED ST4 MFGR#142 18454502 +1998 UNITED ST4 MFGR#1420 1621340 +1998 UNITED ST4 MFGR#1422 13861291 +1998 UNITED ST4 MFGR#1423 4208006 +1998 UNITED ST4 MFGR#1424 3242576 +1998 UNITED ST4 MFGR#1425 1470642 +1998 UNITED ST4 MFGR#1426 11987205 +1998 UNITED ST4 MFGR#1427 12975399 +1998 UNITED ST4 MFGR#1428 11756155 +1998 UNITED ST4 MFGR#1429 6956191 +1998 UNITED ST4 MFGR#143 11499476 +1998 UNITED ST4 MFGR#1430 4355096 +1998 UNITED ST4 MFGR#1431 20166612 +1998 UNITED ST4 MFGR#1432 8849789 +1998 UNITED ST4 MFGR#1433 10786037 +1998 UNITED ST4 MFGR#1435 13208151 +1998 UNITED ST4 MFGR#1436 4747808 +1998 UNITED ST4 MFGR#1437 12999351 +1998 UNITED ST4 MFGR#1438 17360787 +1998 UNITED ST4 MFGR#1439 2557841 +1998 UNITED ST4 MFGR#144 22030911 +1998 UNITED ST4 MFGR#145 5272280 +1998 UNITED ST4 MFGR#146 4307958 +1998 UNITED ST4 MFGR#147 7995050 +1998 UNITED ST4 MFGR#148 2348889 +1998 UNITED ST5 MFGR#141 8396190 +1998 UNITED ST5 MFGR#1410 17422889 +1998 UNITED ST5 MFGR#1411 764246 +1998 UNITED ST5 MFGR#1413 8051561 +1998 UNITED ST5 MFGR#1414 1500867 +1998 UNITED ST5 MFGR#1416 3874315 +1998 UNITED ST5 MFGR#1417 7550747 +1998 UNITED ST5 MFGR#1418 14241755 +1998 UNITED ST5 MFGR#1419 4893846 +1998 UNITED ST5 MFGR#142 5418299 +1998 UNITED ST5 MFGR#1420 3810727 +1998 UNITED ST5 MFGR#1421 3321336 +1998 UNITED ST5 MFGR#1423 5021100 +1998 UNITED ST5 MFGR#1424 6125645 +1998 UNITED ST5 MFGR#1426 880680 +1998 UNITED ST5 MFGR#1427 1284890 +1998 UNITED ST5 MFGR#1428 3813108 +1998 UNITED ST5 MFGR#1429 9429868 +1998 UNITED ST5 MFGR#143 15699981 +1998 UNITED ST5 MFGR#1430 7977654 +1998 UNITED ST5 MFGR#1431 5545969 +1998 UNITED ST5 MFGR#1432 1803434 +1998 UNITED ST5 MFGR#1435 12957056 +1998 UNITED ST5 MFGR#1436 20666755 +1998 UNITED ST5 MFGR#1437 9536733 +1998 UNITED ST5 MFGR#1439 11784214 +1998 UNITED ST5 MFGR#1440 613089 +1998 UNITED ST5 MFGR#145 8237944 +1998 UNITED ST5 MFGR#147 5370351 +1998 UNITED ST6 MFGR#141 6683977 +1998 UNITED ST6 MFGR#1411 11589040 +1998 UNITED ST6 MFGR#1412 4800954 +1998 UNITED ST6 MFGR#1416 5848432 +1998 UNITED ST6 MFGR#1417 1804324 +1998 UNITED ST6 MFGR#1418 4642727 +1998 UNITED ST6 MFGR#1419 10079621 +1998 UNITED ST6 MFGR#142 2533570 +1998 UNITED ST6 MFGR#1420 9016258 +1998 UNITED ST6 MFGR#1421 7636150 +1998 UNITED ST6 MFGR#1422 1272436 +1998 UNITED ST6 MFGR#1423 9600097 +1998 UNITED ST6 MFGR#1424 7036762 +1998 UNITED ST6 MFGR#1425 10867426 +1998 UNITED ST6 MFGR#1426 7033994 +1998 UNITED ST6 MFGR#1427 6481679 +1998 UNITED ST6 MFGR#1428 6269185 +1998 UNITED ST6 MFGR#1429 12055631 +1998 UNITED ST6 MFGR#143 6500459 +1998 UNITED ST6 MFGR#1430 6816243 +1998 UNITED ST6 MFGR#1432 7752973 +1998 UNITED ST6 MFGR#1436 6292813 +1998 UNITED ST6 MFGR#1437 2334894 +1998 UNITED ST6 MFGR#1438 4459455 +1998 UNITED ST6 MFGR#144 640220 +1998 UNITED ST6 MFGR#1440 748200 +1998 UNITED ST6 MFGR#145 7948794 +1998 UNITED ST6 MFGR#146 2734333 +1998 UNITED ST6 MFGR#147 2995632 +1998 UNITED ST6 MFGR#148 10695287 +1998 UNITED ST6 MFGR#149 1475864 +1998 UNITED ST7 MFGR#141 4816637 +1998 UNITED ST7 MFGR#1410 24393912 +1998 UNITED ST7 MFGR#1411 1283038 +1998 UNITED ST7 MFGR#1412 12607459 +1998 UNITED ST7 MFGR#1413 15421293 +1998 UNITED ST7 MFGR#1414 7681668 +1998 UNITED ST7 MFGR#1415 7138377 +1998 UNITED ST7 MFGR#1416 9246990 +1998 UNITED ST7 MFGR#1417 5242972 +1998 UNITED ST7 MFGR#1419 13548941 +1998 UNITED ST7 MFGR#142 17758762 +1998 UNITED ST7 MFGR#1420 11980437 +1998 UNITED ST7 MFGR#1421 11166952 +1998 UNITED ST7 MFGR#1422 9231883 +1998 UNITED ST7 MFGR#1424 5796389 +1998 UNITED ST7 MFGR#1425 4943582 +1998 UNITED ST7 MFGR#1426 16361100 +1998 UNITED ST7 MFGR#1427 9659296 +1998 UNITED ST7 MFGR#1428 2998020 +1998 UNITED ST7 MFGR#1429 5766768 +1998 UNITED ST7 MFGR#143 5526167 +1998 UNITED ST7 MFGR#1430 4191892 +1998 UNITED ST7 MFGR#1431 12979559 +1998 UNITED ST7 MFGR#1432 5832068 +1998 UNITED ST7 MFGR#1433 11410227 +1998 UNITED ST7 MFGR#1434 10525792 +1998 UNITED ST7 MFGR#1435 5982420 +1998 UNITED ST7 MFGR#1436 7142934 +1998 UNITED ST7 MFGR#1437 7499671 +1998 UNITED ST7 MFGR#1438 9177564 +1998 UNITED ST7 MFGR#1439 1974444 +1998 UNITED ST7 MFGR#144 2428463 +1998 UNITED ST7 MFGR#1440 3300838 +1998 UNITED ST7 MFGR#146 1771645 +1998 UNITED ST7 MFGR#147 4681835 +1998 UNITED ST7 MFGR#148 7124367 +1998 UNITED ST7 MFGR#149 11662970 +1998 UNITED ST8 MFGR#141 12753354 +1998 UNITED ST8 MFGR#1410 1097292 +1998 UNITED ST8 MFGR#1411 4069881 +1998 UNITED ST8 MFGR#1414 9474831 +1998 UNITED ST8 MFGR#1415 6406304 +1998 UNITED ST8 MFGR#1418 2392133 +1998 UNITED ST8 MFGR#1419 2953230 +1998 UNITED ST8 MFGR#142 10702563 +1998 UNITED ST8 MFGR#1420 3719432 +1998 UNITED ST8 MFGR#1421 3854657 +1998 UNITED ST8 MFGR#1422 9711968 +1998 UNITED ST8 MFGR#1424 1102516 +1998 UNITED ST8 MFGR#1426 8937532 +1998 UNITED ST8 MFGR#1427 5255637 +1998 UNITED ST8 MFGR#1428 1695112 +1998 UNITED ST8 MFGR#143 6231349 +1998 UNITED ST8 MFGR#1430 4678664 +1998 UNITED ST8 MFGR#1431 2459741 +1998 UNITED ST8 MFGR#1433 1724339 +1998 UNITED ST8 MFGR#1434 8222328 +1998 UNITED ST8 MFGR#1435 4087165 +1998 UNITED ST8 MFGR#1437 7379288 +1998 UNITED ST8 MFGR#1439 1947163 +1998 UNITED ST8 MFGR#144 21846266 +1998 UNITED ST8 MFGR#146 13768113 +1998 UNITED ST8 MFGR#147 5279643 +1998 UNITED ST8 MFGR#149 15221410 +1998 UNITED ST9 MFGR#141 5544000 +1998 UNITED ST9 MFGR#1410 7971104 +1998 UNITED ST9 MFGR#1411 26426609 +1998 UNITED ST9 MFGR#1412 1701777 +1998 UNITED ST9 MFGR#1413 18786672 +1998 UNITED ST9 MFGR#1414 15787780 +1998 UNITED ST9 MFGR#1415 11214818 +1998 UNITED ST9 MFGR#1416 2076120 +1998 UNITED ST9 MFGR#1417 14310334 +1998 UNITED ST9 MFGR#1418 3558355 +1998 UNITED ST9 MFGR#1419 6134867 +1998 UNITED ST9 MFGR#142 257897 +1998 UNITED ST9 MFGR#1420 7716338 +1998 UNITED ST9 MFGR#1421 10291775 +1998 UNITED ST9 MFGR#1422 6105716 +1998 UNITED ST9 MFGR#1423 11378613 +1998 UNITED ST9 MFGR#1424 1904550 +1998 UNITED ST9 MFGR#1425 18831147 +1998 UNITED ST9 MFGR#1426 18562642 +1998 UNITED ST9 MFGR#1427 12565890 +1998 UNITED ST9 MFGR#1428 10720914 +1998 UNITED ST9 MFGR#1429 21940718 +1998 UNITED ST9 MFGR#143 21480722 +1998 UNITED ST9 MFGR#1430 30935173 +1998 UNITED ST9 MFGR#1431 7793012 +1998 UNITED ST9 MFGR#1432 10026104 +1998 UNITED ST9 MFGR#1433 17874435 +1998 UNITED ST9 MFGR#1434 5493785 +1998 UNITED ST9 MFGR#1435 10738769 +1998 UNITED ST9 MFGR#1436 43504 +1998 UNITED ST9 MFGR#1437 13559646 +1998 UNITED ST9 MFGR#1438 12654258 +1998 UNITED ST9 MFGR#1439 5972923 +1998 UNITED ST9 MFGR#144 15702123 +1998 UNITED ST9 MFGR#1440 6681029 +1998 UNITED ST9 MFGR#145 920718 +1998 UNITED ST9 MFGR#146 10435931 +1998 UNITED ST9 MFGR#147 4759326 +1998 UNITED ST9 MFGR#148 9970848 +1998 UNITED ST9 MFGR#149 7510940 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.1.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.1.out new file mode 100644 index 00000000000000..58a86b03f0c288 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.1.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q1.1 -- +446031203850 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.2.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.2.out new file mode 100644 index 00000000000000..7ca21d7cbbe869 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.2.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q1.2 -- +98714004603 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.3.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.3.out new file mode 100644 index 00000000000000..b6051a28437af7 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q1.3.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q1.3 -- +26110729246 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.1.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.1.out new file mode 100644 index 00000000000000..9b4932a76b0b93 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.1.out @@ -0,0 +1,283 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q2.1 -- +567838207 1992 MFGR#121 +610663790 1992 MFGR#1210 +550769662 1992 MFGR#1211 +649205856 1992 MFGR#1212 +624031241 1992 MFGR#1213 +670488468 1992 MFGR#1214 +633152470 1992 MFGR#1215 +674846781 1992 MFGR#1216 +675093435 1992 MFGR#1217 +600202070 1992 MFGR#1218 +538043594 1992 MFGR#1219 +655326672 1992 MFGR#122 +540262882 1992 MFGR#1220 +556120633 1992 MFGR#1221 +590762777 1992 MFGR#1222 +535448651 1992 MFGR#1223 +703752611 1992 MFGR#1224 +570832868 1992 MFGR#1225 +614061593 1992 MFGR#1226 +581759388 1992 MFGR#1227 +644642592 1992 MFGR#1228 +640858430 1992 MFGR#1229 +789755835 1992 MFGR#123 +468535087 1992 MFGR#1230 +592436656 1992 MFGR#1231 +664275152 1992 MFGR#1232 +613885100 1992 MFGR#1233 +667399281 1992 MFGR#1234 +640290070 1992 MFGR#1235 +501892561 1992 MFGR#1236 +591481503 1992 MFGR#1237 +477423770 1992 MFGR#1238 +638259374 1992 MFGR#1239 +572354196 1992 MFGR#124 +740479248 1992 MFGR#1240 +478777095 1992 MFGR#125 +592174616 1992 MFGR#126 +706151632 1992 MFGR#127 +542306646 1992 MFGR#128 +581987352 1992 MFGR#129 +823087702 1993 MFGR#121 +648160706 1993 MFGR#1210 +634743898 1993 MFGR#1211 +785639283 1993 MFGR#1212 +638255029 1993 MFGR#1213 +616837237 1993 MFGR#1214 +634687975 1993 MFGR#1215 +638353900 1993 MFGR#1216 +663372951 1993 MFGR#1217 +683985855 1993 MFGR#1218 +646950033 1993 MFGR#1219 +622532984 1993 MFGR#122 +530830127 1993 MFGR#1220 +543346337 1993 MFGR#1221 +756921203 1993 MFGR#1222 +533544350 1993 MFGR#1223 +915916085 1993 MFGR#1224 +473007381 1993 MFGR#1225 +739036124 1993 MFGR#1226 +592178887 1993 MFGR#1227 +583507058 1993 MFGR#1228 +617453491 1993 MFGR#1229 +637863868 1993 MFGR#123 +625534310 1993 MFGR#1230 +580327635 1993 MFGR#1231 +697373098 1993 MFGR#1232 +515571416 1993 MFGR#1233 +651935758 1993 MFGR#1234 +575779480 1993 MFGR#1235 +591878667 1993 MFGR#1236 +609618576 1993 MFGR#1237 +444614010 1993 MFGR#1238 +595256327 1993 MFGR#1239 +660586237 1993 MFGR#124 +788730059 1993 MFGR#1240 +616224539 1993 MFGR#125 +617126754 1993 MFGR#126 +654438324 1993 MFGR#127 +731657001 1993 MFGR#128 +548048395 1993 MFGR#129 +564405648 1994 MFGR#121 +645404849 1994 MFGR#1210 +631620635 1994 MFGR#1211 +568332348 1994 MFGR#1212 +678785857 1994 MFGR#1213 +534002330 1994 MFGR#1214 +654400242 1994 MFGR#1215 +558646341 1994 MFGR#1216 +687845641 1994 MFGR#1217 +546674347 1994 MFGR#1218 +567272942 1994 MFGR#1219 +659884062 1994 MFGR#122 +562582172 1994 MFGR#1220 +598618997 1994 MFGR#1221 +601016441 1994 MFGR#1222 +555134404 1994 MFGR#1223 +737422302 1994 MFGR#1224 +570745955 1994 MFGR#1225 +746302245 1994 MFGR#1226 +651707481 1994 MFGR#1227 +573693547 1994 MFGR#1228 +647918373 1994 MFGR#1229 +580449592 1994 MFGR#123 +493270412 1994 MFGR#1230 +603546148 1994 MFGR#1231 +719865331 1994 MFGR#1232 +638982238 1994 MFGR#1233 +743247677 1994 MFGR#1234 +598680959 1994 MFGR#1235 +615726097 1994 MFGR#1236 +542569815 1994 MFGR#1237 +573510781 1994 MFGR#1238 +579855853 1994 MFGR#1239 +684573322 1994 MFGR#124 +873735737 1994 MFGR#1240 +560488304 1994 MFGR#125 +657036514 1994 MFGR#126 +622571183 1994 MFGR#127 +586845664 1994 MFGR#128 +534541525 1994 MFGR#129 +706469511 1995 MFGR#121 +602892803 1995 MFGR#1210 +645166092 1995 MFGR#1211 +613289283 1995 MFGR#1212 +599586479 1995 MFGR#1213 +562570804 1995 MFGR#1214 +672528755 1995 MFGR#1215 +669000972 1995 MFGR#1216 +725362449 1995 MFGR#1217 +657026635 1995 MFGR#1218 +519659003 1995 MFGR#1219 +724727741 1995 MFGR#122 +517956131 1995 MFGR#1220 +635741351 1995 MFGR#1221 +564368410 1995 MFGR#1222 +600665149 1995 MFGR#1223 +762700351 1995 MFGR#1224 +671669586 1995 MFGR#1225 +572568748 1995 MFGR#1226 +530361300 1995 MFGR#1227 +633357085 1995 MFGR#1228 +547960244 1995 MFGR#1229 +660711077 1995 MFGR#123 +602735858 1995 MFGR#1230 +499852146 1995 MFGR#1231 +715300753 1995 MFGR#1232 +557149571 1995 MFGR#1233 +710023059 1995 MFGR#1234 +622425239 1995 MFGR#1235 +634565501 1995 MFGR#1236 +572847270 1995 MFGR#1237 +549318912 1995 MFGR#1238 +593851712 1995 MFGR#1239 +585421815 1995 MFGR#124 +707207888 1995 MFGR#1240 +538246872 1995 MFGR#125 +605799021 1995 MFGR#126 +665978112 1995 MFGR#127 +646960956 1995 MFGR#128 +508749401 1995 MFGR#129 +523879145 1996 MFGR#121 +643645053 1996 MFGR#1210 +595065339 1996 MFGR#1211 +674626440 1996 MFGR#1212 +496297087 1996 MFGR#1213 +583249505 1996 MFGR#1214 +702184857 1996 MFGR#1215 +601809334 1996 MFGR#1216 +704898387 1996 MFGR#1217 +528843086 1996 MFGR#1218 +586246330 1996 MFGR#1219 +712110492 1996 MFGR#122 +518444215 1996 MFGR#1220 +499319414 1996 MFGR#1221 +679469356 1996 MFGR#1222 +628762754 1996 MFGR#1223 +724844856 1996 MFGR#1224 +660620587 1996 MFGR#1225 +667674729 1996 MFGR#1226 +483838085 1996 MFGR#1227 +609855391 1996 MFGR#1228 +658959557 1996 MFGR#1229 +566217852 1996 MFGR#123 +528879998 1996 MFGR#1230 +589481194 1996 MFGR#1231 +702805896 1996 MFGR#1232 +663679947 1996 MFGR#1233 +571149450 1996 MFGR#1234 +478648074 1996 MFGR#1235 +568249365 1996 MFGR#1236 +592616167 1996 MFGR#1237 +466676148 1996 MFGR#1238 +670693719 1996 MFGR#1239 +560667719 1996 MFGR#124 +821167950 1996 MFGR#1240 +476864333 1996 MFGR#125 +558030884 1996 MFGR#126 +635873891 1996 MFGR#127 +551010618 1996 MFGR#128 +560570630 1996 MFGR#129 +587013207 1997 MFGR#121 +616287892 1997 MFGR#1210 +548588761 1997 MFGR#1211 +589593892 1997 MFGR#1212 +424306670 1997 MFGR#1213 +511971910 1997 MFGR#1214 +631772246 1997 MFGR#1215 +692135140 1997 MFGR#1216 +777994957 1997 MFGR#1217 +707053720 1997 MFGR#1218 +561169527 1997 MFGR#1219 +664916245 1997 MFGR#122 +594466157 1997 MFGR#1220 +588848171 1997 MFGR#1221 +528988960 1997 MFGR#1222 +537098211 1997 MFGR#1223 +674763166 1997 MFGR#1224 +450402292 1997 MFGR#1225 +701360722 1997 MFGR#1226 +506011570 1997 MFGR#1227 +585578737 1997 MFGR#1228 +622744016 1997 MFGR#1229 +646503168 1997 MFGR#123 +571800941 1997 MFGR#1230 +502601790 1997 MFGR#1231 +677924656 1997 MFGR#1232 +534455976 1997 MFGR#1233 +714934715 1997 MFGR#1234 +767151420 1997 MFGR#1235 +618877179 1997 MFGR#1236 +639638057 1997 MFGR#1237 +401953419 1997 MFGR#1238 +610756714 1997 MFGR#1239 +543248087 1997 MFGR#124 +675132692 1997 MFGR#1240 +479099365 1997 MFGR#125 +570696568 1997 MFGR#126 +583074592 1997 MFGR#127 +695133104 1997 MFGR#128 +655638776 1997 MFGR#129 +344575925 1998 MFGR#121 +417152416 1998 MFGR#1210 +317068168 1998 MFGR#1211 +374341516 1998 MFGR#1212 +332740903 1998 MFGR#1213 +304873002 1998 MFGR#1214 +366101132 1998 MFGR#1215 +379133898 1998 MFGR#1216 +359508497 1998 MFGR#1217 +320623334 1998 MFGR#1218 +346182862 1998 MFGR#1219 +312440027 1998 MFGR#122 +348123961 1998 MFGR#1220 +339845398 1998 MFGR#1221 +355416161 1998 MFGR#1222 +344889822 1998 MFGR#1223 +396906691 1998 MFGR#1224 +290208878 1998 MFGR#1225 +419415707 1998 MFGR#1226 +358466340 1998 MFGR#1227 +251549955 1998 MFGR#1228 +383138860 1998 MFGR#1229 +296330561 1998 MFGR#123 +437181243 1998 MFGR#1230 +398944492 1998 MFGR#1231 +424062455 1998 MFGR#1232 +406967188 1998 MFGR#1233 +428867240 1998 MFGR#1234 +352277781 1998 MFGR#1235 +361827086 1998 MFGR#1236 +341618569 1998 MFGR#1237 +244739231 1998 MFGR#1238 +414151803 1998 MFGR#1239 +330082371 1998 MFGR#124 +415312453 1998 MFGR#1240 +360289624 1998 MFGR#125 +341657580 1998 MFGR#126 +377507061 1998 MFGR#127 +361416497 1998 MFGR#128 +318769573 1998 MFGR#129 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.2.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.2.out new file mode 100644 index 00000000000000..4c53da2be7cfff --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.2.out @@ -0,0 +1,59 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q2.2 -- +709524929 1992 MFGR#2221 +783846394 1992 MFGR#2222 +765052002 1992 MFGR#2223 +651488962 1992 MFGR#2224 +646528589 1992 MFGR#2225 +709650548 1992 MFGR#2226 +745556316 1992 MFGR#2227 +756901875 1992 MFGR#2228 +766521103 1993 MFGR#2221 +691475597 1993 MFGR#2222 +758220752 1993 MFGR#2223 +669662707 1993 MFGR#2224 +773854228 1993 MFGR#2225 +737087518 1993 MFGR#2226 +781967766 1993 MFGR#2227 +680880216 1993 MFGR#2228 +685777518 1994 MFGR#2221 +666524807 1994 MFGR#2222 +733993590 1994 MFGR#2223 +707869040 1994 MFGR#2224 +721251967 1994 MFGR#2225 +822495919 1994 MFGR#2226 +720837128 1994 MFGR#2227 +826225350 1994 MFGR#2228 +775437074 1995 MFGR#2221 +761354792 1995 MFGR#2222 +637832575 1995 MFGR#2223 +589765707 1995 MFGR#2224 +708290039 1995 MFGR#2225 +747356383 1995 MFGR#2226 +802502540 1995 MFGR#2227 +895936786 1995 MFGR#2228 +700010008 1996 MFGR#2221 +777310085 1996 MFGR#2222 +656095314 1996 MFGR#2223 +656859917 1996 MFGR#2224 +765820896 1996 MFGR#2225 +808177734 1996 MFGR#2226 +729563303 1996 MFGR#2227 +819665874 1996 MFGR#2228 +727342382 1997 MFGR#2221 +664533779 1997 MFGR#2222 +748288392 1997 MFGR#2223 +630422081 1997 MFGR#2224 +757391203 1997 MFGR#2225 +747889257 1997 MFGR#2226 +728857899 1997 MFGR#2227 +775312985 1997 MFGR#2228 +335304504 1998 MFGR#2221 +409347137 1998 MFGR#2222 +459109577 1998 MFGR#2223 +413318072 1998 MFGR#2224 +410402095 1998 MFGR#2225 +453515044 1998 MFGR#2226 +390506405 1998 MFGR#2227 +397939103 1998 MFGR#2228 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.3.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.3.out new file mode 100644 index 00000000000000..f1a716cfd008b8 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q2.3.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q2.3 -- +726427486 1992 MFGR#2239 +705238959 1993 MFGR#2239 +664145134 1994 MFGR#2239 +713839336 1995 MFGR#2239 +728492865 1996 MFGR#2239 +733896532 1997 MFGR#2239 +380255731 1998 MFGR#2239 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.1.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.1.out new file mode 100644 index 00000000000000..9afae94c3fa253 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.1.out @@ -0,0 +1,153 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q3.1 -- +INDONESIA INDONESIA 1992 6905698988 +CHINA INDONESIA 1992 6616078869 +CHINA CHINA 1992 6456431477 +VIETNAM INDONESIA 1992 6340181707 +VIETNAM CHINA 1992 6332844390 +INDONESIA CHINA 1992 6310709533 +INDIA INDONESIA 1992 6238807354 +JAPAN INDONESIA 1992 6222565638 +CHINA INDIA 1992 6189831351 +CHINA JAPAN 1992 6157678893 +JAPAN CHINA 1992 6067100655 +INDIA CHINA 1992 6066461811 +VIETNAM JAPAN 1992 5863786349 +JAPAN INDIA 1992 5802749985 +INDONESIA INDIA 1992 5770203218 +VIETNAM INDIA 1992 5698511401 +INDONESIA JAPAN 1992 5663765610 +JAPAN JAPAN 1992 5648034711 +INDIA JAPAN 1992 5421959281 +CHINA VIETNAM 1992 5412322026 +JAPAN VIETNAM 1992 5401692555 +INDIA INDIA 1992 5361243755 +INDONESIA VIETNAM 1992 5302638372 +INDIA VIETNAM 1992 5245330621 +VIETNAM VIETNAM 1992 5111731778 +INDONESIA INDONESIA 1993 6931363262 +INDONESIA CHINA 1993 6612891249 +CHINA INDONESIA 1993 6581513366 +VIETNAM INDONESIA 1993 6511639352 +CHINA CHINA 1993 6444513787 +CHINA INDIA 1993 6421694121 +VIETNAM CHINA 1993 6413579999 +INDIA INDONESIA 1993 6331058594 +JAPAN CHINA 1993 6292293995 +INDIA CHINA 1993 6209353792 +CHINA JAPAN 1993 6195466884 +VIETNAM INDIA 1993 6161174477 +INDONESIA INDIA 1993 6151664306 +INDONESIA JAPAN 1993 6147605249 +JAPAN INDONESIA 1993 6053946313 +INDIA JAPAN 1993 5995208985 +CHINA VIETNAM 1993 5885023228 +VIETNAM JAPAN 1993 5874266867 +INDONESIA VIETNAM 1993 5840287722 +JAPAN INDIA 1993 5823343157 +INDIA INDIA 1993 5571575799 +VIETNAM VIETNAM 1993 5554161183 +JAPAN JAPAN 1993 5430318583 +JAPAN VIETNAM 1993 5406569653 +INDIA VIETNAM 1993 5233871487 +VIETNAM CHINA 1994 6709883442 +CHINA CHINA 1994 6648942965 +INDONESIA CHINA 1994 6483600910 +CHINA JAPAN 1994 6349037021 +VIETNAM INDONESIA 1994 6347441931 +JAPAN CHINA 1994 6320009218 +INDONESIA INDONESIA 1994 6315331086 +CHINA INDIA 1994 6297582811 +CHINA INDONESIA 1994 6234787896 +VIETNAM JAPAN 1994 6231973047 +INDONESIA INDIA 1994 6198412245 +INDIA INDONESIA 1994 6168189859 +INDONESIA JAPAN 1994 6141195969 +JAPAN INDONESIA 1994 6013034618 +CHINA VIETNAM 1994 5885318733 +INDIA CHINA 1994 5815022714 +VIETNAM INDIA 1994 5789398691 +JAPAN JAPAN 1994 5753821482 +JAPAN INDIA 1994 5645818872 +INDONESIA VIETNAM 1994 5624346368 +INDIA INDIA 1994 5584936515 +VIETNAM VIETNAM 1994 5418498159 +INDIA JAPAN 1994 5394134974 +JAPAN VIETNAM 1994 5275821637 +INDIA VIETNAM 1994 5251039815 +CHINA INDONESIA 1995 7074539444 +INDONESIA CHINA 1995 6928620620 +CHINA CHINA 1995 6624666612 +JAPAN CHINA 1995 6519889385 +VIETNAM CHINA 1995 6332731268 +INDONESIA INDONESIA 1995 6268531321 +VIETNAM JAPAN 1995 6263957020 +CHINA INDIA 1995 6246656374 +VIETNAM INDIA 1995 6198441957 +INDONESIA INDIA 1995 6173805183 +VIETNAM INDONESIA 1995 6057820148 +CHINA JAPAN 1995 6021854130 +INDONESIA JAPAN 1995 6019093871 +INDIA CHINA 1995 6010532793 +INDIA INDONESIA 1995 5975518597 +JAPAN JAPAN 1995 5952279129 +JAPAN INDONESIA 1995 5947000735 +VIETNAM VIETNAM 1995 5754727607 +JAPAN INDIA 1995 5730743126 +INDONESIA VIETNAM 1995 5574974066 +INDIA INDIA 1995 5565995550 +JAPAN VIETNAM 1995 5501313775 +CHINA VIETNAM 1995 5489936063 +INDIA JAPAN 1995 5469203427 +INDIA VIETNAM 1995 5207063284 +INDONESIA INDONESIA 1996 6750056283 +CHINA INDONESIA 1996 6657125075 +INDONESIA CHINA 1996 6613439112 +CHINA CHINA 1996 6591697915 +VIETNAM CHINA 1996 6428653890 +VIETNAM INDONESIA 1996 6295626732 +JAPAN INDONESIA 1996 6227447856 +INDIA CHINA 1996 6198567819 +INDIA INDONESIA 1996 6189242058 +CHINA INDIA 1996 6153787365 +INDONESIA INDIA 1996 6138810863 +CHINA JAPAN 1996 6105566324 +JAPAN CHINA 1996 6088985564 +INDONESIA JAPAN 1996 6055638222 +VIETNAM JAPAN 1996 5919997150 +VIETNAM VIETNAM 1996 5860842302 +INDIA JAPAN 1996 5841664805 +VIETNAM INDIA 1996 5807486983 +JAPAN INDIA 1996 5644579616 +JAPAN JAPAN 1996 5566204795 +INDIA INDIA 1996 5532073103 +INDONESIA VIETNAM 1996 5438514387 +CHINA VIETNAM 1996 5434772818 +JAPAN VIETNAM 1996 5334277549 +INDIA VIETNAM 1996 5293891531 +CHINA INDONESIA 1997 6653234696 +INDONESIA CHINA 1997 6577396314 +CHINA CHINA 1997 6463794795 +JAPAN INDONESIA 1997 6429270126 +INDONESIA INDONESIA 1997 6398427821 +VIETNAM INDONESIA 1997 6363583120 +CHINA JAPAN 1997 6284733518 +VIETNAM CHINA 1997 6164218723 +INDIA CHINA 1997 6153282214 +CHINA INDIA 1997 6084283983 +INDONESIA JAPAN 1997 6001427956 +INDIA INDONESIA 1997 5983439300 +JAPAN CHINA 1997 5973849687 +CHINA VIETNAM 1997 5908254176 +JAPAN INDIA 1997 5796921992 +JAPAN JAPAN 1997 5790450039 +INDONESIA INDIA 1997 5789367105 +VIETNAM INDIA 1997 5730121608 +INDIA INDIA 1997 5693028907 +VIETNAM JAPAN 1997 5689345230 +INDONESIA VIETNAM 1997 5665678132 +VIETNAM VIETNAM 1997 5620251324 +INDIA JAPAN 1997 5587827913 +JAPAN VIETNAM 1997 5249874614 +INDIA VIETNAM 1997 5003624837 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.2.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.2.out new file mode 100644 index 00000000000000..d0f90c01a154e2 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.2.out @@ -0,0 +1,602 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q3.2 -- +UNITED ST4 UNITED ST1 1992 102027455 +UNITED ST1 UNITED ST0 1992 96989491 +UNITED ST7 UNITED ST0 1992 96078010 +UNITED ST9 UNITED ST0 1992 94813294 +UNITED ST4 UNITED ST0 1992 94644242 +UNITED ST2 UNITED ST4 1992 91180500 +UNITED ST5 UNITED ST0 1992 90432300 +UNITED ST6 UNITED ST7 1992 87658267 +UNITED ST3 UNITED ST9 1992 86142048 +UNITED ST6 UNITED ST5 1992 85882966 +UNITED ST7 UNITED ST3 1992 83765666 +UNITED ST2 UNITED ST9 1992 83705618 +UNITED ST4 UNITED ST6 1992 81886374 +UNITED ST2 UNITED ST1 1992 81839165 +UNITED ST9 UNITED ST1 1992 80795302 +UNITED ST6 UNITED ST3 1992 78778218 +UNITED ST6 UNITED ST0 1992 78696956 +UNITED ST0 UNITED ST1 1992 77267396 +UNITED ST0 UNITED ST0 1992 75622122 +UNITED ST1 UNITED ST9 1992 75367059 +UNITED ST3 UNITED ST1 1992 73637490 +UNITED ST2 UNITED ST0 1992 72210218 +UNITED ST1 UNITED ST7 1992 71472973 +UNITED ST6 UNITED ST4 1992 71086944 +UNITED ST4 UNITED ST4 1992 70111335 +UNITED ST6 UNITED ST1 1992 69408688 +UNITED ST4 UNITED ST3 1992 69001787 +UNITED ST5 UNITED ST7 1992 68333651 +UNITED ST4 UNITED ST9 1992 67837970 +UNITED ST7 UNITED ST6 1992 65513205 +UNITED ST4 UNITED ST5 1992 65057872 +UNITED ST7 UNITED ST4 1992 64900888 +UNITED ST1 UNITED ST1 1992 64669070 +UNITED ST3 UNITED ST5 1992 64239048 +UNITED ST0 UNITED ST9 1992 63979996 +UNITED ST3 UNITED ST4 1992 63144772 +UNITED ST5 UNITED ST6 1992 62628093 +UNITED ST4 UNITED ST7 1992 62529376 +UNITED ST3 UNITED ST0 1992 62441656 +UNITED ST9 UNITED ST4 1992 61489513 +UNITED ST8 UNITED ST6 1992 60540440 +UNITED ST7 UNITED ST9 1992 60326042 +UNITED ST7 UNITED ST7 1992 60121386 +UNITED ST5 UNITED ST1 1992 59945287 +UNITED ST5 UNITED ST4 1992 57625627 +UNITED ST7 UNITED ST5 1992 57566802 +UNITED ST2 UNITED ST5 1992 57021365 +UNITED ST9 UNITED ST7 1992 56883359 +UNITED ST0 UNITED ST3 1992 56359317 +UNITED ST1 UNITED ST3 1992 55727474 +UNITED ST5 UNITED ST3 1992 53963553 +UNITED ST0 UNITED ST7 1992 50583409 +UNITED ST5 UNITED ST9 1992 50191091 +UNITED ST7 UNITED ST1 1992 50167208 +UNITED ST0 UNITED ST8 1992 49732640 +UNITED ST0 UNITED ST4 1992 49676807 +UNITED ST9 UNITED ST3 1992 47681165 +UNITED ST8 UNITED ST4 1992 46757019 +UNITED ST3 UNITED ST3 1992 45087216 +UNITED ST8 UNITED ST0 1992 44368839 +UNITED ST0 UNITED ST6 1992 42471806 +UNITED ST6 UNITED ST8 1992 42463690 +UNITED ST8 UNITED ST7 1992 41897901 +UNITED ST3 UNITED ST8 1992 41275764 +UNITED ST6 UNITED ST9 1992 40591721 +UNITED ST0 UNITED ST5 1992 40120886 +UNITED ST1 UNITED ST4 1992 39326346 +UNITED ST3 UNITED ST7 1992 39028579 +UNITED ST3 UNITED ST6 1992 38798715 +UNITED ST9 UNITED ST9 1992 36048343 +UNITED ST2 UNITED ST8 1992 36046449 +UNITED ST2 UNITED ST3 1992 35981963 +UNITED ST8 UNITED ST1 1992 35680752 +UNITED ST1 UNITED ST6 1992 35404990 +UNITED ST8 UNITED ST5 1992 35187610 +UNITED ST1 UNITED ST5 1992 33971251 +UNITED ST5 UNITED ST8 1992 33878053 +UNITED ST2 UNITED ST7 1992 33702779 +UNITED ST8 UNITED ST3 1992 30949324 +UNITED ST8 UNITED ST8 1992 29309108 +UNITED ST5 UNITED ST5 1992 29279568 +UNITED ST1 UNITED ST8 1992 28565579 +UNITED ST9 UNITED ST5 1992 28075004 +UNITED ST2 UNITED ST6 1992 27813739 +UNITED ST0 UNITED ST2 1992 27718733 +UNITED ST2 UNITED ST2 1992 25743654 +UNITED ST8 UNITED ST9 1992 22684471 +UNITED ST4 UNITED ST8 1992 21928442 +UNITED ST9 UNITED ST8 1992 21386100 +UNITED ST5 UNITED ST2 1992 20495817 +UNITED ST6 UNITED ST6 1992 18137105 +UNITED ST9 UNITED ST6 1992 15879568 +UNITED ST4 UNITED ST2 1992 12061845 +UNITED ST7 UNITED ST8 1992 11895702 +UNITED ST6 UNITED ST2 1992 11820698 +UNITED ST9 UNITED ST2 1992 11623177 +UNITED ST8 UNITED ST2 1992 10971561 +UNITED ST1 UNITED ST2 1992 7706728 +UNITED ST7 UNITED ST2 1992 4943204 +UNITED ST3 UNITED ST2 1992 1097208 +UNITED ST0 UNITED ST9 1993 109834040 +UNITED ST7 UNITED ST0 1993 109788024 +UNITED ST5 UNITED ST0 1993 106822597 +UNITED ST0 UNITED ST0 1993 106742548 +UNITED ST1 UNITED ST0 1993 99305952 +UNITED ST4 UNITED ST4 1993 98150465 +UNITED ST3 UNITED ST4 1993 92493920 +UNITED ST0 UNITED ST1 1993 91196593 +UNITED ST4 UNITED ST1 1993 88521423 +UNITED ST8 UNITED ST0 1993 88356371 +UNITED ST4 UNITED ST7 1993 88172198 +UNITED ST4 UNITED ST0 1993 86918458 +UNITED ST6 UNITED ST3 1993 83417161 +UNITED ST6 UNITED ST1 1993 83345939 +UNITED ST7 UNITED ST9 1993 80310701 +UNITED ST3 UNITED ST1 1993 78230278 +UNITED ST6 UNITED ST7 1993 78197294 +UNITED ST5 UNITED ST9 1993 76286539 +UNITED ST0 UNITED ST3 1993 76171283 +UNITED ST5 UNITED ST8 1993 74359279 +UNITED ST9 UNITED ST1 1993 74059419 +UNITED ST4 UNITED ST9 1993 73296959 +UNITED ST5 UNITED ST1 1993 71454623 +UNITED ST6 UNITED ST4 1993 69646913 +UNITED ST2 UNITED ST1 1993 69631701 +UNITED ST6 UNITED ST0 1993 68247539 +UNITED ST7 UNITED ST7 1993 68109820 +UNITED ST2 UNITED ST3 1993 66972438 +UNITED ST3 UNITED ST0 1993 66626926 +UNITED ST9 UNITED ST7 1993 66625483 +UNITED ST1 UNITED ST8 1993 66146198 +UNITED ST2 UNITED ST7 1993 64185014 +UNITED ST5 UNITED ST4 1993 63415639 +UNITED ST9 UNITED ST9 1993 63260763 +UNITED ST1 UNITED ST4 1993 62884347 +UNITED ST7 UNITED ST4 1993 61656613 +UNITED ST3 UNITED ST6 1993 58584808 +UNITED ST2 UNITED ST4 1993 56650391 +UNITED ST3 UNITED ST5 1993 55907305 +UNITED ST6 UNITED ST9 1993 54900942 +UNITED ST1 UNITED ST7 1993 54851183 +UNITED ST3 UNITED ST9 1993 54762596 +UNITED ST8 UNITED ST6 1993 54633062 +UNITED ST8 UNITED ST3 1993 54049874 +UNITED ST5 UNITED ST7 1993 52745538 +UNITED ST0 UNITED ST5 1993 52701052 +UNITED ST1 UNITED ST9 1993 52514902 +UNITED ST8 UNITED ST5 1993 52237837 +UNITED ST1 UNITED ST3 1993 52097946 +UNITED ST8 UNITED ST4 1993 51419356 +UNITED ST0 UNITED ST6 1993 50432282 +UNITED ST5 UNITED ST5 1993 50357189 +UNITED ST3 UNITED ST7 1993 50135448 +UNITED ST0 UNITED ST4 1993 49260067 +UNITED ST0 UNITED ST7 1993 48796360 +UNITED ST2 UNITED ST9 1993 48188507 +UNITED ST1 UNITED ST1 1993 47538610 +UNITED ST9 UNITED ST3 1993 46943647 +UNITED ST7 UNITED ST5 1993 44763692 +UNITED ST1 UNITED ST6 1993 44728540 +UNITED ST8 UNITED ST1 1993 44415434 +UNITED ST7 UNITED ST8 1993 43902628 +UNITED ST9 UNITED ST6 1993 43867160 +UNITED ST2 UNITED ST0 1993 42845485 +UNITED ST3 UNITED ST8 1993 42251848 +UNITED ST0 UNITED ST8 1993 42002682 +UNITED ST4 UNITED ST8 1993 41657582 +UNITED ST1 UNITED ST5 1993 40693513 +UNITED ST9 UNITED ST5 1993 39685269 +UNITED ST7 UNITED ST3 1993 39523861 +UNITED ST8 UNITED ST8 1993 38790235 +UNITED ST8 UNITED ST9 1993 38516361 +UNITED ST2 UNITED ST5 1993 37406845 +UNITED ST9 UNITED ST8 1993 37184696 +UNITED ST8 UNITED ST7 1993 36902218 +UNITED ST6 UNITED ST8 1993 36456741 +UNITED ST7 UNITED ST1 1993 34391159 +UNITED ST6 UNITED ST5 1993 34229082 +UNITED ST5 UNITED ST3 1993 34031811 +UNITED ST2 UNITED ST8 1993 33445446 +UNITED ST4 UNITED ST3 1993 33129412 +UNITED ST6 UNITED ST6 1993 33050663 +UNITED ST9 UNITED ST0 1993 32653305 +UNITED ST4 UNITED ST6 1993 30699255 +UNITED ST9 UNITED ST4 1993 30644687 +UNITED ST4 UNITED ST5 1993 29119594 +UNITED ST7 UNITED ST6 1993 27100502 +UNITED ST4 UNITED ST2 1993 27012678 +UNITED ST2 UNITED ST6 1993 26482226 +UNITED ST5 UNITED ST6 1993 25357679 +UNITED ST3 UNITED ST3 1993 21777144 +UNITED ST3 UNITED ST2 1993 21559073 +UNITED ST5 UNITED ST2 1993 20610242 +UNITED ST7 UNITED ST2 1993 20219304 +UNITED ST6 UNITED ST2 1993 18814367 +UNITED ST9 UNITED ST2 1993 17718390 +UNITED ST1 UNITED ST2 1993 16844538 +UNITED ST0 UNITED ST2 1993 15042145 +UNITED ST2 UNITED ST2 1993 14521995 +UNITED ST8 UNITED ST2 1993 9984366 +UNITED ST8 UNITED ST0 1994 99220789 +UNITED ST3 UNITED ST9 1994 97476185 +UNITED ST6 UNITED ST1 1994 96937147 +UNITED ST6 UNITED ST9 1994 94683309 +UNITED ST9 UNITED ST1 1994 90440948 +UNITED ST0 UNITED ST9 1994 89865202 +UNITED ST5 UNITED ST7 1994 89089961 +UNITED ST9 UNITED ST0 1994 87670573 +UNITED ST3 UNITED ST1 1994 85523653 +UNITED ST4 UNITED ST9 1994 83822393 +UNITED ST0 UNITED ST0 1994 83526877 +UNITED ST7 UNITED ST0 1994 82265536 +UNITED ST2 UNITED ST1 1994 81300089 +UNITED ST5 UNITED ST0 1994 78648057 +UNITED ST4 UNITED ST7 1994 76954140 +UNITED ST4 UNITED ST4 1994 76837381 +UNITED ST0 UNITED ST1 1994 76613379 +UNITED ST1 UNITED ST3 1994 75992459 +UNITED ST7 UNITED ST1 1994 75320799 +UNITED ST4 UNITED ST0 1994 73719340 +UNITED ST5 UNITED ST1 1994 73508418 +UNITED ST4 UNITED ST1 1994 72219557 +UNITED ST2 UNITED ST9 1994 69671054 +UNITED ST6 UNITED ST5 1994 66461534 +UNITED ST2 UNITED ST3 1994 65620760 +UNITED ST3 UNITED ST0 1994 65522727 +UNITED ST5 UNITED ST3 1994 65334911 +UNITED ST7 UNITED ST4 1994 64778715 +UNITED ST3 UNITED ST4 1994 63412365 +UNITED ST8 UNITED ST4 1994 62141681 +UNITED ST0 UNITED ST4 1994 61519744 +UNITED ST0 UNITED ST7 1994 61480820 +UNITED ST0 UNITED ST6 1994 61288778 +UNITED ST2 UNITED ST0 1994 60182153 +UNITED ST6 UNITED ST4 1994 59829989 +UNITED ST4 UNITED ST5 1994 59397028 +UNITED ST8 UNITED ST9 1994 58666906 +UNITED ST4 UNITED ST6 1994 58633482 +UNITED ST5 UNITED ST5 1994 56235213 +UNITED ST6 UNITED ST3 1994 56123238 +UNITED ST2 UNITED ST4 1994 55679377 +UNITED ST8 UNITED ST3 1994 55203841 +UNITED ST1 UNITED ST1 1994 54383174 +UNITED ST1 UNITED ST7 1994 53853106 +UNITED ST6 UNITED ST0 1994 53728853 +UNITED ST5 UNITED ST9 1994 53367331 +UNITED ST9 UNITED ST9 1994 51980849 +UNITED ST5 UNITED ST4 1994 51799593 +UNITED ST7 UNITED ST9 1994 50144085 +UNITED ST7 UNITED ST7 1994 46446442 +UNITED ST6 UNITED ST6 1994 46199722 +UNITED ST7 UNITED ST5 1994 45895364 +UNITED ST3 UNITED ST3 1994 45627153 +UNITED ST8 UNITED ST5 1994 44553056 +UNITED ST9 UNITED ST4 1994 43910761 +UNITED ST1 UNITED ST0 1994 43225201 +UNITED ST1 UNITED ST9 1994 43000037 +UNITED ST7 UNITED ST8 1994 42776467 +UNITED ST0 UNITED ST5 1994 41808301 +UNITED ST2 UNITED ST6 1994 41526105 +UNITED ST1 UNITED ST4 1994 41381558 +UNITED ST3 UNITED ST7 1994 40935131 +UNITED ST8 UNITED ST1 1994 40152096 +UNITED ST9 UNITED ST8 1994 39278808 +UNITED ST5 UNITED ST6 1994 38658451 +UNITED ST2 UNITED ST5 1994 37640317 +UNITED ST8 UNITED ST7 1994 37600687 +UNITED ST9 UNITED ST5 1994 37146726 +UNITED ST6 UNITED ST7 1994 37057808 +UNITED ST8 UNITED ST6 1994 36776569 +UNITED ST3 UNITED ST6 1994 36290257 +UNITED ST9 UNITED ST3 1994 35846500 +UNITED ST2 UNITED ST8 1994 33767774 +UNITED ST0 UNITED ST8 1994 31845433 +UNITED ST4 UNITED ST3 1994 31599433 +UNITED ST9 UNITED ST7 1994 31586173 +UNITED ST1 UNITED ST6 1994 31287326 +UNITED ST1 UNITED ST8 1994 30245153 +UNITED ST7 UNITED ST3 1994 29424840 +UNITED ST9 UNITED ST6 1994 29212927 +UNITED ST0 UNITED ST3 1994 27327829 +UNITED ST6 UNITED ST8 1994 26592996 +UNITED ST3 UNITED ST5 1994 26197875 +UNITED ST6 UNITED ST2 1994 25809000 +UNITED ST1 UNITED ST5 1994 24939138 +UNITED ST7 UNITED ST6 1994 24631937 +UNITED ST1 UNITED ST2 1994 23556586 +UNITED ST4 UNITED ST2 1994 23035892 +UNITED ST2 UNITED ST7 1994 22182758 +UNITED ST0 UNITED ST2 1994 22017954 +UNITED ST4 UNITED ST8 1994 20685352 +UNITED ST7 UNITED ST2 1994 19655081 +UNITED ST5 UNITED ST8 1994 18931891 +UNITED ST2 UNITED ST2 1994 18068657 +UNITED ST3 UNITED ST8 1994 15936051 +UNITED ST8 UNITED ST8 1994 10023412 +UNITED ST3 UNITED ST2 1994 9995234 +UNITED ST9 UNITED ST2 1994 9700989 +UNITED ST5 UNITED ST2 1994 7162796 +UNITED ST8 UNITED ST2 1994 3789626 +UNITED ST5 UNITED ST1 1995 119793669 +UNITED ST4 UNITED ST9 1995 99490068 +UNITED ST7 UNITED ST0 1995 98031295 +UNITED ST6 UNITED ST0 1995 91718471 +UNITED ST4 UNITED ST1 1995 90878653 +UNITED ST0 UNITED ST1 1995 90763599 +UNITED ST8 UNITED ST9 1995 88855089 +UNITED ST7 UNITED ST7 1995 86571624 +UNITED ST3 UNITED ST0 1995 84462733 +UNITED ST9 UNITED ST1 1995 82938967 +UNITED ST2 UNITED ST4 1995 82432305 +UNITED ST1 UNITED ST0 1995 81676623 +UNITED ST5 UNITED ST4 1995 81016761 +UNITED ST7 UNITED ST1 1995 79964362 +UNITED ST5 UNITED ST3 1995 78099130 +UNITED ST5 UNITED ST0 1995 77615746 +UNITED ST9 UNITED ST9 1995 76515958 +UNITED ST7 UNITED ST9 1995 75317709 +UNITED ST4 UNITED ST4 1995 74587071 +UNITED ST9 UNITED ST4 1995 72525686 +UNITED ST1 UNITED ST9 1995 72470870 +UNITED ST4 UNITED ST7 1995 69264407 +UNITED ST6 UNITED ST3 1995 67513062 +UNITED ST2 UNITED ST3 1995 65218129 +UNITED ST2 UNITED ST9 1995 65055178 +UNITED ST7 UNITED ST6 1995 65020671 +UNITED ST3 UNITED ST1 1995 64762815 +UNITED ST1 UNITED ST1 1995 64199332 +UNITED ST6 UNITED ST9 1995 63457105 +UNITED ST0 UNITED ST9 1995 63253499 +UNITED ST5 UNITED ST9 1995 62364897 +UNITED ST4 UNITED ST5 1995 62081505 +UNITED ST1 UNITED ST7 1995 61515741 +UNITED ST2 UNITED ST7 1995 60000208 +UNITED ST8 UNITED ST6 1995 58990404 +UNITED ST1 UNITED ST4 1995 57535599 +UNITED ST0 UNITED ST3 1995 56360708 +UNITED ST8 UNITED ST0 1995 55231664 +UNITED ST5 UNITED ST7 1995 53740759 +UNITED ST2 UNITED ST0 1995 52560838 +UNITED ST3 UNITED ST7 1995 51579548 +UNITED ST9 UNITED ST0 1995 51548621 +UNITED ST6 UNITED ST6 1995 50954677 +UNITED ST5 UNITED ST5 1995 50394007 +UNITED ST7 UNITED ST4 1995 49899545 +UNITED ST3 UNITED ST3 1995 48158089 +UNITED ST6 UNITED ST4 1995 47697241 +UNITED ST9 UNITED ST7 1995 46464589 +UNITED ST4 UNITED ST0 1995 46142899 +UNITED ST1 UNITED ST3 1995 45823056 +UNITED ST2 UNITED ST1 1995 45437340 +UNITED ST6 UNITED ST5 1995 45428152 +UNITED ST8 UNITED ST5 1995 44994863 +UNITED ST7 UNITED ST3 1995 43699734 +UNITED ST9 UNITED ST6 1995 43482494 +UNITED ST2 UNITED ST5 1995 43382417 +UNITED ST6 UNITED ST8 1995 41973920 +UNITED ST0 UNITED ST6 1995 40718942 +UNITED ST3 UNITED ST5 1995 40057815 +UNITED ST7 UNITED ST5 1995 39015293 +UNITED ST0 UNITED ST0 1995 38984502 +UNITED ST6 UNITED ST1 1995 38328352 +UNITED ST4 UNITED ST6 1995 38109524 +UNITED ST3 UNITED ST9 1995 37364623 +UNITED ST4 UNITED ST3 1995 37356396 +UNITED ST2 UNITED ST6 1995 37146288 +UNITED ST9 UNITED ST5 1995 36009924 +UNITED ST1 UNITED ST8 1995 34918793 +UNITED ST8 UNITED ST1 1995 34217780 +UNITED ST0 UNITED ST7 1995 33395313 +UNITED ST1 UNITED ST5 1995 31857452 +UNITED ST8 UNITED ST7 1995 30918202 +UNITED ST2 UNITED ST8 1995 30504189 +UNITED ST3 UNITED ST4 1995 30422346 +UNITED ST5 UNITED ST6 1995 30204737 +UNITED ST8 UNITED ST3 1995 29349938 +UNITED ST0 UNITED ST4 1995 29170038 +UNITED ST1 UNITED ST6 1995 27139403 +UNITED ST7 UNITED ST8 1995 26444490 +UNITED ST6 UNITED ST7 1995 23833977 +UNITED ST4 UNITED ST8 1995 23053236 +UNITED ST4 UNITED ST2 1995 22787003 +UNITED ST3 UNITED ST8 1995 22505239 +UNITED ST9 UNITED ST8 1995 21292527 +UNITED ST8 UNITED ST4 1995 19287311 +UNITED ST8 UNITED ST2 1995 18282990 +UNITED ST9 UNITED ST3 1995 17539102 +UNITED ST3 UNITED ST6 1995 16738530 +UNITED ST0 UNITED ST8 1995 16393249 +UNITED ST5 UNITED ST2 1995 14951023 +UNITED ST2 UNITED ST2 1995 13455031 +UNITED ST5 UNITED ST8 1995 13346932 +UNITED ST3 UNITED ST2 1995 12886829 +UNITED ST9 UNITED ST2 1995 12653362 +UNITED ST0 UNITED ST5 1995 11453709 +UNITED ST6 UNITED ST2 1995 11363551 +UNITED ST8 UNITED ST8 1995 11285867 +UNITED ST1 UNITED ST2 1995 7991676 +UNITED ST0 UNITED ST2 1995 4776460 +UNITED ST7 UNITED ST2 1995 3973565 +UNITED ST6 UNITED ST0 1996 132286763 +UNITED ST4 UNITED ST0 1996 106897563 +UNITED ST5 UNITED ST0 1996 104501979 +UNITED ST0 UNITED ST4 1996 103228749 +UNITED ST9 UNITED ST1 1996 101983827 +UNITED ST1 UNITED ST0 1996 94861554 +UNITED ST0 UNITED ST1 1996 91948777 +UNITED ST6 UNITED ST1 1996 89705870 +UNITED ST2 UNITED ST1 1996 88256155 +UNITED ST1 UNITED ST1 1996 87265848 +UNITED ST4 UNITED ST7 1996 83677814 +UNITED ST6 UNITED ST3 1996 82168229 +UNITED ST2 UNITED ST7 1996 80468477 +UNITED ST8 UNITED ST1 1996 78971756 +UNITED ST7 UNITED ST4 1996 77941011 +UNITED ST1 UNITED ST3 1996 77610905 +UNITED ST9 UNITED ST9 1996 77301740 +UNITED ST0 UNITED ST9 1996 75935209 +UNITED ST7 UNITED ST0 1996 75602445 +UNITED ST3 UNITED ST1 1996 74746699 +UNITED ST7 UNITED ST7 1996 74040644 +UNITED ST4 UNITED ST1 1996 72819867 +UNITED ST5 UNITED ST9 1996 72614114 +UNITED ST1 UNITED ST9 1996 69823769 +UNITED ST9 UNITED ST4 1996 69616614 +UNITED ST6 UNITED ST4 1996 69296005 +UNITED ST2 UNITED ST0 1996 67095122 +UNITED ST5 UNITED ST1 1996 65346389 +UNITED ST6 UNITED ST9 1996 63256182 +UNITED ST4 UNITED ST6 1996 62189328 +UNITED ST0 UNITED ST0 1996 61528855 +UNITED ST8 UNITED ST9 1996 60466691 +UNITED ST3 UNITED ST0 1996 60226840 +UNITED ST8 UNITED ST6 1996 59746655 +UNITED ST2 UNITED ST3 1996 59648598 +UNITED ST0 UNITED ST5 1996 57762895 +UNITED ST8 UNITED ST7 1996 57523925 +UNITED ST2 UNITED ST4 1996 57487057 +UNITED ST6 UNITED ST7 1996 57090619 +UNITED ST3 UNITED ST4 1996 54838259 +UNITED ST4 UNITED ST9 1996 54134840 +UNITED ST1 UNITED ST6 1996 54056366 +UNITED ST3 UNITED ST7 1996 53987218 +UNITED ST2 UNITED ST9 1996 53491415 +UNITED ST4 UNITED ST8 1996 53035662 +UNITED ST9 UNITED ST5 1996 52825922 +UNITED ST7 UNITED ST3 1996 52356886 +UNITED ST6 UNITED ST8 1996 52136784 +UNITED ST1 UNITED ST5 1996 51189649 +UNITED ST8 UNITED ST4 1996 51033054 +UNITED ST1 UNITED ST4 1996 50135547 +UNITED ST3 UNITED ST9 1996 49612304 +UNITED ST9 UNITED ST0 1996 49590701 +UNITED ST3 UNITED ST3 1996 49263796 +UNITED ST9 UNITED ST7 1996 48798759 +UNITED ST7 UNITED ST1 1996 48784175 +UNITED ST9 UNITED ST6 1996 48685063 +UNITED ST2 UNITED ST5 1996 47028976 +UNITED ST9 UNITED ST3 1996 47021018 +UNITED ST2 UNITED ST8 1996 46865113 +UNITED ST4 UNITED ST3 1996 46460940 +UNITED ST6 UNITED ST5 1996 46030104 +UNITED ST2 UNITED ST6 1996 45416649 +UNITED ST8 UNITED ST5 1996 43480473 +UNITED ST5 UNITED ST5 1996 43020722 +UNITED ST6 UNITED ST6 1996 42923032 +UNITED ST4 UNITED ST5 1996 42808412 +UNITED ST3 UNITED ST6 1996 41881628 +UNITED ST1 UNITED ST7 1996 41721506 +UNITED ST5 UNITED ST7 1996 40946330 +UNITED ST8 UNITED ST0 1996 39845427 +UNITED ST8 UNITED ST3 1996 39535940 +UNITED ST1 UNITED ST8 1996 39430882 +UNITED ST5 UNITED ST6 1996 38332044 +UNITED ST0 UNITED ST6 1996 37232062 +UNITED ST7 UNITED ST6 1996 36535628 +UNITED ST9 UNITED ST8 1996 36112301 +UNITED ST3 UNITED ST8 1996 33924732 +UNITED ST3 UNITED ST5 1996 33717439 +UNITED ST5 UNITED ST4 1996 33424859 +UNITED ST5 UNITED ST3 1996 32919926 +UNITED ST4 UNITED ST4 1996 32787995 +UNITED ST7 UNITED ST5 1996 32784224 +UNITED ST5 UNITED ST8 1996 32415682 +UNITED ST0 UNITED ST7 1996 31391181 +UNITED ST0 UNITED ST3 1996 29795665 +UNITED ST7 UNITED ST9 1996 25028091 +UNITED ST7 UNITED ST8 1996 24348851 +UNITED ST6 UNITED ST2 1996 20447847 +UNITED ST8 UNITED ST8 1996 16340603 +UNITED ST0 UNITED ST8 1996 15168262 +UNITED ST4 UNITED ST2 1996 12451867 +UNITED ST1 UNITED ST2 1996 10082536 +UNITED ST5 UNITED ST2 1996 8544233 +UNITED ST7 UNITED ST2 1996 8390470 +UNITED ST9 UNITED ST2 1996 8108035 +UNITED ST8 UNITED ST2 1996 7028334 +UNITED ST0 UNITED ST2 1996 6907199 +UNITED ST3 UNITED ST2 1996 4311800 +UNITED ST5 UNITED ST0 1997 121457766 +UNITED ST0 UNITED ST9 1997 119856268 +UNITED ST5 UNITED ST1 1997 106900161 +UNITED ST9 UNITED ST9 1997 106222795 +UNITED ST5 UNITED ST4 1997 103432927 +UNITED ST7 UNITED ST1 1997 101326940 +UNITED ST5 UNITED ST9 1997 97392640 +UNITED ST8 UNITED ST0 1997 89434845 +UNITED ST1 UNITED ST3 1997 85175638 +UNITED ST4 UNITED ST1 1997 84611188 +UNITED ST4 UNITED ST7 1997 84606996 +UNITED ST1 UNITED ST4 1997 83092569 +UNITED ST0 UNITED ST1 1997 80167139 +UNITED ST4 UNITED ST9 1997 79697927 +UNITED ST1 UNITED ST0 1997 77667866 +UNITED ST2 UNITED ST0 1997 77591470 +UNITED ST1 UNITED ST7 1997 77045722 +UNITED ST2 UNITED ST7 1997 76483802 +UNITED ST1 UNITED ST1 1997 76340444 +UNITED ST0 UNITED ST7 1997 72577490 +UNITED ST4 UNITED ST0 1997 69875804 +UNITED ST6 UNITED ST3 1997 69725506 +UNITED ST2 UNITED ST9 1997 69543984 +UNITED ST7 UNITED ST0 1997 69354312 +UNITED ST9 UNITED ST7 1997 69052630 +UNITED ST8 UNITED ST3 1997 66918394 +UNITED ST0 UNITED ST0 1997 66308516 +UNITED ST9 UNITED ST0 1997 66066791 +UNITED ST2 UNITED ST3 1997 65429453 +UNITED ST2 UNITED ST1 1997 65396135 +UNITED ST4 UNITED ST4 1997 62532346 +UNITED ST9 UNITED ST1 1997 62418406 +UNITED ST3 UNITED ST7 1997 61095300 +UNITED ST7 UNITED ST4 1997 60123494 +UNITED ST4 UNITED ST3 1997 59634153 +UNITED ST3 UNITED ST4 1997 58356141 +UNITED ST6 UNITED ST9 1997 58231263 +UNITED ST6 UNITED ST4 1997 57215022 +UNITED ST2 UNITED ST4 1997 57012611 +UNITED ST5 UNITED ST3 1997 56789932 +UNITED ST9 UNITED ST5 1997 56091920 +UNITED ST6 UNITED ST0 1997 55824919 +UNITED ST6 UNITED ST1 1997 55117709 +UNITED ST7 UNITED ST9 1997 55039970 +UNITED ST5 UNITED ST7 1997 54534315 +UNITED ST3 UNITED ST1 1997 54150683 +UNITED ST3 UNITED ST0 1997 54050172 +UNITED ST3 UNITED ST9 1997 51370308 +UNITED ST1 UNITED ST5 1997 51052110 +UNITED ST6 UNITED ST7 1997 49795849 +UNITED ST5 UNITED ST6 1997 49030016 +UNITED ST1 UNITED ST9 1997 48944111 +UNITED ST3 UNITED ST3 1997 48385233 +UNITED ST0 UNITED ST5 1997 47988418 +UNITED ST2 UNITED ST8 1997 46391909 +UNITED ST4 UNITED ST6 1997 46236849 +UNITED ST9 UNITED ST3 1997 46121724 +UNITED ST8 UNITED ST9 1997 45852796 +UNITED ST7 UNITED ST8 1997 45476266 +UNITED ST8 UNITED ST1 1997 43284139 +UNITED ST7 UNITED ST7 1997 42566603 +UNITED ST0 UNITED ST4 1997 41193803 +UNITED ST8 UNITED ST7 1997 40878429 +UNITED ST8 UNITED ST8 1997 40749400 +UNITED ST2 UNITED ST5 1997 40662886 +UNITED ST0 UNITED ST3 1997 40078508 +UNITED ST6 UNITED ST8 1997 37988445 +UNITED ST9 UNITED ST6 1997 37596882 +UNITED ST6 UNITED ST5 1997 37571788 +UNITED ST2 UNITED ST2 1997 37034333 +UNITED ST7 UNITED ST5 1997 36889736 +UNITED ST8 UNITED ST4 1997 36600584 +UNITED ST3 UNITED ST6 1997 36075844 +UNITED ST7 UNITED ST3 1997 35168922 +UNITED ST2 UNITED ST6 1997 34274467 +UNITED ST5 UNITED ST8 1997 32910946 +UNITED ST3 UNITED ST5 1997 32811963 +UNITED ST4 UNITED ST8 1997 32599736 +UNITED ST5 UNITED ST5 1997 32568888 +UNITED ST4 UNITED ST5 1997 31995868 +UNITED ST9 UNITED ST4 1997 31765478 +UNITED ST7 UNITED ST2 1997 31409590 +UNITED ST9 UNITED ST8 1997 31272385 +UNITED ST0 UNITED ST8 1997 30241370 +UNITED ST3 UNITED ST8 1997 29102220 +UNITED ST7 UNITED ST6 1997 27539931 +UNITED ST8 UNITED ST5 1997 26673743 +UNITED ST6 UNITED ST6 1997 24983291 +UNITED ST0 UNITED ST2 1997 23584229 +UNITED ST0 UNITED ST6 1997 22924046 +UNITED ST1 UNITED ST2 1997 20599130 +UNITED ST8 UNITED ST6 1997 20073000 +UNITED ST1 UNITED ST6 1997 18205326 +UNITED ST1 UNITED ST8 1997 15375258 +UNITED ST6 UNITED ST2 1997 14746680 +UNITED ST5 UNITED ST2 1997 13863438 +UNITED ST8 UNITED ST2 1997 12053706 +UNITED ST3 UNITED ST2 1997 7891878 +UNITED ST4 UNITED ST2 1997 2848157 +UNITED ST9 UNITED ST2 1997 2661652 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.3.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.3.out new file mode 100644 index 00000000000000..7a8708105dcc21 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.3.out @@ -0,0 +1,27 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q3.3 -- +UNITED KI1 UNITED KI1 1992 46735995 +UNITED KI5 UNITED KI1 1992 36277055 +UNITED KI5 UNITED KI5 1992 25355267 +UNITED KI1 UNITED KI5 1992 21917846 +UNITED KI5 UNITED KI1 1993 61017607 +UNITED KI1 UNITED KI1 1993 45669535 +UNITED KI5 UNITED KI5 1993 34099392 +UNITED KI1 UNITED KI5 1993 21444206 +UNITED KI5 UNITED KI1 1994 36282163 +UNITED KI1 UNITED KI1 1994 34868441 +UNITED KI5 UNITED KI5 1994 34507284 +UNITED KI1 UNITED KI5 1994 21221780 +UNITED KI5 UNITED KI1 1995 82955896 +UNITED KI1 UNITED KI1 1995 35881186 +UNITED KI5 UNITED KI5 1995 20539805 +UNITED KI1 UNITED KI5 1995 17176510 +UNITED KI5 UNITED KI1 1996 65767049 +UNITED KI1 UNITED KI1 1996 59923037 +UNITED KI5 UNITED KI5 1996 46077342 +UNITED KI1 UNITED KI5 1996 13700254 +UNITED KI1 UNITED KI1 1997 70343133 +UNITED KI5 UNITED KI1 1997 64978359 +UNITED KI5 UNITED KI5 1997 27332027 +UNITED KI1 UNITED KI5 1997 16410668 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.4.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.4.out new file mode 100644 index 00000000000000..f46fd55119065a --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q3.4.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q3.4 -- +UNITED KI5 UNITED KI1 1997 9117846 +UNITED KI5 UNITED KI5 1997 6203801 +UNITED KI1 UNITED KI5 1997 1870070 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.1.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.1.out new file mode 100644 index 00000000000000..9e7504904245dd --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.1.out @@ -0,0 +1,38 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q4.1 -- +1992 ARGENTINA 9658964469 +1992 BRAZIL 9226983055 +1992 CANADA 9643176787 +1992 PERU 9410676597 +1992 UNITED STATES 9849427653 +1993 ARGENTINA 9976332853 +1993 BRAZIL 9468799229 +1993 CANADA 9897302420 +1993 PERU 9309445836 +1993 UNITED STATES 10003985086 +1994 ARGENTINA 9940305215 +1994 BRAZIL 9348651677 +1994 CANADA 9582647596 +1994 PERU 9295265013 +1994 UNITED STATES 9519880425 +1995 ARGENTINA 10143841380 +1995 BRAZIL 9156077350 +1995 CANADA 9562612160 +1995 PERU 9778087211 +1995 UNITED STATES 9310565244 +1996 ARGENTINA 10001927895 +1996 BRAZIL 9168485151 +1996 CANADA 10061604203 +1996 PERU 9355135674 +1996 UNITED STATES 9769712174 +1997 ARGENTINA 9854560261 +1997 BRAZIL 9121571047 +1997 CANADA 10097371778 +1997 PERU 9315525917 +1997 UNITED STATES 10506723879 +1998 ARGENTINA 5834240407 +1998 BRAZIL 5356398095 +1998 CANADA 5423211196 +1998 PERU 5726185970 +1998 UNITED STATES 6009462019 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.2.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.2.out new file mode 100644 index 00000000000000..65b6d696df521b --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.2.out @@ -0,0 +1,103 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q4.2 -- +1997 ARGENTINA MFGR#11 907071566 +1997 ARGENTINA MFGR#12 924115562 +1997 ARGENTINA MFGR#13 972861821 +1997 ARGENTINA MFGR#14 975410345 +1997 ARGENTINA MFGR#15 938867375 +1997 ARGENTINA MFGR#21 1014782574 +1997 ARGENTINA MFGR#22 873016783 +1997 ARGENTINA MFGR#23 1030357302 +1997 ARGENTINA MFGR#24 893460579 +1997 ARGENTINA MFGR#25 1006311403 +1997 BRAZIL MFGR#11 1073219328 +1997 BRAZIL MFGR#12 989858833 +1997 BRAZIL MFGR#13 1128480379 +1997 BRAZIL MFGR#14 1194256722 +1997 BRAZIL MFGR#15 1094419124 +1997 BRAZIL MFGR#21 910026832 +1997 BRAZIL MFGR#22 993142048 +1997 BRAZIL MFGR#23 1107672874 +1997 BRAZIL MFGR#24 1058013649 +1997 BRAZIL MFGR#25 994733764 +1997 CANADA MFGR#11 854725020 +1997 CANADA MFGR#12 938718164 +1997 CANADA MFGR#13 959265890 +1997 CANADA MFGR#14 1002812450 +1997 CANADA MFGR#15 848183013 +1997 CANADA MFGR#21 999805272 +1997 CANADA MFGR#22 778419763 +1997 CANADA MFGR#23 928359645 +1997 CANADA MFGR#24 849895128 +1997 CANADA MFGR#25 904587965 +1997 PERU MFGR#11 1100242877 +1997 PERU MFGR#12 994365350 +1997 PERU MFGR#13 847486105 +1997 PERU MFGR#14 947769683 +1997 PERU MFGR#15 999395678 +1997 PERU MFGR#21 867923394 +1997 PERU MFGR#22 988747459 +1997 PERU MFGR#23 1066645086 +1997 PERU MFGR#24 935665725 +1997 PERU MFGR#25 981454129 +1997 UNITED STATES MFGR#11 1046706048 +1997 UNITED STATES MFGR#12 909213709 +1997 UNITED STATES MFGR#13 1096278906 +1997 UNITED STATES MFGR#14 934282111 +1997 UNITED STATES MFGR#15 962760843 +1997 UNITED STATES MFGR#21 1000676474 +1997 UNITED STATES MFGR#22 1076947615 +1997 UNITED STATES MFGR#23 937288102 +1997 UNITED STATES MFGR#24 1003386363 +1997 UNITED STATES MFGR#25 1053666052 +1998 ARGENTINA MFGR#11 567612227 +1998 ARGENTINA MFGR#12 527025042 +1998 ARGENTINA MFGR#13 582791792 +1998 ARGENTINA MFGR#14 523726368 +1998 ARGENTINA MFGR#15 522078267 +1998 ARGENTINA MFGR#21 504712685 +1998 ARGENTINA MFGR#22 506061736 +1998 ARGENTINA MFGR#23 560479801 +1998 ARGENTINA MFGR#24 524579118 +1998 ARGENTINA MFGR#25 547840211 +1998 BRAZIL MFGR#11 638578488 +1998 BRAZIL MFGR#12 646312681 +1998 BRAZIL MFGR#13 655161772 +1998 BRAZIL MFGR#14 552676170 +1998 BRAZIL MFGR#15 663812709 +1998 BRAZIL MFGR#21 668822448 +1998 BRAZIL MFGR#22 591791918 +1998 BRAZIL MFGR#23 690648877 +1998 BRAZIL MFGR#24 562362220 +1998 BRAZIL MFGR#25 704182376 +1998 CANADA MFGR#11 509086125 +1998 CANADA MFGR#12 488089772 +1998 CANADA MFGR#13 486533297 +1998 CANADA MFGR#14 527837227 +1998 CANADA MFGR#15 535869299 +1998 CANADA MFGR#21 455868651 +1998 CANADA MFGR#22 594277308 +1998 CANADA MFGR#23 574125070 +1998 CANADA MFGR#24 508530424 +1998 CANADA MFGR#25 547757992 +1998 PERU MFGR#11 567838547 +1998 PERU MFGR#12 540544757 +1998 PERU MFGR#13 591331883 +1998 PERU MFGR#14 481335064 +1998 PERU MFGR#15 570246138 +1998 PERU MFGR#21 533733330 +1998 PERU MFGR#22 527790656 +1998 PERU MFGR#23 636393221 +1998 PERU MFGR#24 589075262 +1998 PERU MFGR#25 543251115 +1998 UNITED STATES MFGR#11 556276232 +1998 UNITED STATES MFGR#12 612385982 +1998 UNITED STATES MFGR#13 622413927 +1998 UNITED STATES MFGR#14 555006887 +1998 UNITED STATES MFGR#15 525119569 +1998 UNITED STATES MFGR#21 563406836 +1998 UNITED STATES MFGR#22 580478735 +1998 UNITED STATES MFGR#23 656080465 +1998 UNITED STATES MFGR#24 538445058 +1998 UNITED STATES MFGR#25 589111952 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.3.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.3.out new file mode 100644 index 00000000000000..16b854fc0a88c9 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/flat_q4.3.out @@ -0,0 +1,697 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !flat_q4.3 -- +1997 UNITED ST0 MFGR#141 15456288 +1997 UNITED ST0 MFGR#1410 32961113 +1997 UNITED ST0 MFGR#1411 36166400 +1997 UNITED ST0 MFGR#1412 18237451 +1997 UNITED ST0 MFGR#1413 40752813 +1997 UNITED ST0 MFGR#1414 24237588 +1997 UNITED ST0 MFGR#1415 6419001 +1997 UNITED ST0 MFGR#1416 15639876 +1997 UNITED ST0 MFGR#1417 15476211 +1997 UNITED ST0 MFGR#1418 8169956 +1997 UNITED ST0 MFGR#1419 18217252 +1997 UNITED ST0 MFGR#142 29361222 +1997 UNITED ST0 MFGR#1420 34587603 +1997 UNITED ST0 MFGR#1421 28560921 +1997 UNITED ST0 MFGR#1422 25154181 +1997 UNITED ST0 MFGR#1423 18144008 +1997 UNITED ST0 MFGR#1424 28064267 +1997 UNITED ST0 MFGR#1425 15255845 +1997 UNITED ST0 MFGR#1426 5610353 +1997 UNITED ST0 MFGR#1427 5750693 +1997 UNITED ST0 MFGR#1428 14081707 +1997 UNITED ST0 MFGR#1429 9024512 +1997 UNITED ST0 MFGR#143 5325193 +1997 UNITED ST0 MFGR#1430 16961739 +1997 UNITED ST0 MFGR#1431 19015253 +1997 UNITED ST0 MFGR#1432 6324917 +1997 UNITED ST0 MFGR#1433 6734461 +1997 UNITED ST0 MFGR#1434 19206393 +1997 UNITED ST0 MFGR#1435 679925 +1997 UNITED ST0 MFGR#1436 13834012 +1997 UNITED ST0 MFGR#1437 23575407 +1997 UNITED ST0 MFGR#1438 24756074 +1997 UNITED ST0 MFGR#1439 19644072 +1997 UNITED ST0 MFGR#144 20285157 +1997 UNITED ST0 MFGR#1440 14068558 +1997 UNITED ST0 MFGR#145 22646138 +1997 UNITED ST0 MFGR#146 16381159 +1997 UNITED ST0 MFGR#147 12267827 +1997 UNITED ST0 MFGR#148 30968614 +1997 UNITED ST0 MFGR#149 8640669 +1997 UNITED ST1 MFGR#141 10152087 +1997 UNITED ST1 MFGR#1410 32583300 +1997 UNITED ST1 MFGR#1411 15690352 +1997 UNITED ST1 MFGR#1412 16671403 +1997 UNITED ST1 MFGR#1413 4942775 +1997 UNITED ST1 MFGR#1414 9391597 +1997 UNITED ST1 MFGR#1415 26937296 +1997 UNITED ST1 MFGR#1416 25782871 +1997 UNITED ST1 MFGR#1417 23349641 +1997 UNITED ST1 MFGR#1418 10934620 +1997 UNITED ST1 MFGR#1419 22673030 +1997 UNITED ST1 MFGR#142 14048276 +1997 UNITED ST1 MFGR#1420 18100795 +1997 UNITED ST1 MFGR#1421 25405678 +1997 UNITED ST1 MFGR#1423 5713800 +1997 UNITED ST1 MFGR#1424 4131923 +1997 UNITED ST1 MFGR#1425 17190425 +1997 UNITED ST1 MFGR#1426 22314044 +1997 UNITED ST1 MFGR#1427 13635057 +1997 UNITED ST1 MFGR#1428 17735811 +1997 UNITED ST1 MFGR#1429 28740911 +1997 UNITED ST1 MFGR#143 27418033 +1997 UNITED ST1 MFGR#1430 18655292 +1997 UNITED ST1 MFGR#1431 11074399 +1997 UNITED ST1 MFGR#1432 7490170 +1997 UNITED ST1 MFGR#1433 28997266 +1997 UNITED ST1 MFGR#1434 4612211 +1997 UNITED ST1 MFGR#1435 16327124 +1997 UNITED ST1 MFGR#1436 17409665 +1997 UNITED ST1 MFGR#1437 33391635 +1997 UNITED ST1 MFGR#1438 37321134 +1997 UNITED ST1 MFGR#1439 10279941 +1997 UNITED ST1 MFGR#144 12304723 +1997 UNITED ST1 MFGR#1440 28921230 +1997 UNITED ST1 MFGR#145 17933684 +1997 UNITED ST1 MFGR#146 8709103 +1997 UNITED ST1 MFGR#147 20145983 +1997 UNITED ST1 MFGR#148 16940400 +1997 UNITED ST1 MFGR#149 10284949 +1997 UNITED ST2 MFGR#1412 6328240 +1997 UNITED ST2 MFGR#1415 5022682 +1997 UNITED ST2 MFGR#1416 4102214 +1997 UNITED ST2 MFGR#1417 5284444 +1997 UNITED ST2 MFGR#1418 2649908 +1997 UNITED ST2 MFGR#1419 4664673 +1997 UNITED ST2 MFGR#142 10784835 +1997 UNITED ST2 MFGR#1423 4095190 +1997 UNITED ST2 MFGR#1424 5376355 +1997 UNITED ST2 MFGR#1425 3949957 +1997 UNITED ST2 MFGR#143 2859829 +1997 UNITED ST2 MFGR#1430 4618266 +1997 UNITED ST2 MFGR#1432 5183060 +1997 UNITED ST2 MFGR#1433 10531371 +1997 UNITED ST2 MFGR#1434 77646 +1997 UNITED ST2 MFGR#1436 6538508 +1997 UNITED ST2 MFGR#1438 12324747 +1997 UNITED ST2 MFGR#144 2002870 +1997 UNITED ST2 MFGR#1440 12247229 +1997 UNITED ST2 MFGR#145 202685 +1997 UNITED ST2 MFGR#148 5998932 +1997 UNITED ST3 MFGR#141 12866083 +1997 UNITED ST3 MFGR#1410 31051703 +1997 UNITED ST3 MFGR#1411 14099543 +1997 UNITED ST3 MFGR#1412 4970551 +1997 UNITED ST3 MFGR#1413 26129762 +1997 UNITED ST3 MFGR#1414 22364787 +1997 UNITED ST3 MFGR#1415 27883428 +1997 UNITED ST3 MFGR#1416 25289630 +1997 UNITED ST3 MFGR#1417 5524810 +1997 UNITED ST3 MFGR#1419 3616608 +1997 UNITED ST3 MFGR#142 10776059 +1997 UNITED ST3 MFGR#1420 12958851 +1997 UNITED ST3 MFGR#1421 5815494 +1997 UNITED ST3 MFGR#1422 17694270 +1997 UNITED ST3 MFGR#1423 10285142 +1997 UNITED ST3 MFGR#1424 21555317 +1997 UNITED ST3 MFGR#1425 5214517 +1997 UNITED ST3 MFGR#1426 16810700 +1997 UNITED ST3 MFGR#1427 10086430 +1997 UNITED ST3 MFGR#1428 7733716 +1997 UNITED ST3 MFGR#1429 11922430 +1997 UNITED ST3 MFGR#1430 20105292 +1997 UNITED ST3 MFGR#1431 14716979 +1997 UNITED ST3 MFGR#1432 15779247 +1997 UNITED ST3 MFGR#1433 15308773 +1997 UNITED ST3 MFGR#1434 8515037 +1997 UNITED ST3 MFGR#1435 25486899 +1997 UNITED ST3 MFGR#1436 12739448 +1997 UNITED ST3 MFGR#1437 11249180 +1997 UNITED ST3 MFGR#1438 4156824 +1997 UNITED ST3 MFGR#1439 18358277 +1997 UNITED ST3 MFGR#144 10097718 +1997 UNITED ST3 MFGR#1440 22920494 +1997 UNITED ST3 MFGR#145 20285859 +1997 UNITED ST3 MFGR#146 18406423 +1997 UNITED ST3 MFGR#147 17295990 +1997 UNITED ST3 MFGR#148 26127145 +1997 UNITED ST3 MFGR#149 19895538 +1997 UNITED ST4 MFGR#141 8135477 +1997 UNITED ST4 MFGR#1410 5503247 +1997 UNITED ST4 MFGR#1411 6096435 +1997 UNITED ST4 MFGR#1412 19976043 +1997 UNITED ST4 MFGR#1413 22581282 +1997 UNITED ST4 MFGR#1414 10101379 +1997 UNITED ST4 MFGR#1415 8877444 +1997 UNITED ST4 MFGR#1416 17943398 +1997 UNITED ST4 MFGR#1417 9224501 +1997 UNITED ST4 MFGR#1418 16255176 +1997 UNITED ST4 MFGR#1419 683069 +1997 UNITED ST4 MFGR#142 23512944 +1997 UNITED ST4 MFGR#1420 27954262 +1997 UNITED ST4 MFGR#1421 54209185 +1997 UNITED ST4 MFGR#1422 1173957 +1997 UNITED ST4 MFGR#1423 20410710 +1997 UNITED ST4 MFGR#1424 2889270 +1997 UNITED ST4 MFGR#1425 13361796 +1997 UNITED ST4 MFGR#1426 18678819 +1997 UNITED ST4 MFGR#1427 18206066 +1997 UNITED ST4 MFGR#1428 18153974 +1997 UNITED ST4 MFGR#1429 12545076 +1997 UNITED ST4 MFGR#143 6158834 +1997 UNITED ST4 MFGR#1430 16128370 +1997 UNITED ST4 MFGR#1431 11153717 +1997 UNITED ST4 MFGR#1432 7745636 +1997 UNITED ST4 MFGR#1433 15983634 +1997 UNITED ST4 MFGR#1434 3662246 +1997 UNITED ST4 MFGR#1435 3060180 +1997 UNITED ST4 MFGR#1436 11524592 +1997 UNITED ST4 MFGR#1437 36824174 +1997 UNITED ST4 MFGR#1438 8226666 +1997 UNITED ST4 MFGR#1439 9872470 +1997 UNITED ST4 MFGR#1440 401207 +1997 UNITED ST4 MFGR#145 11698312 +1997 UNITED ST4 MFGR#146 18259956 +1997 UNITED ST4 MFGR#147 10256110 +1997 UNITED ST4 MFGR#148 15892628 +1997 UNITED ST4 MFGR#149 4322491 +1997 UNITED ST5 MFGR#141 8534275 +1997 UNITED ST5 MFGR#1411 2427261 +1997 UNITED ST5 MFGR#1412 9032422 +1997 UNITED ST5 MFGR#1413 6343588 +1997 UNITED ST5 MFGR#1414 2961025 +1997 UNITED ST5 MFGR#1415 21186568 +1997 UNITED ST5 MFGR#1416 9354080 +1997 UNITED ST5 MFGR#1417 11786246 +1997 UNITED ST5 MFGR#1418 15297245 +1997 UNITED ST5 MFGR#1420 15787608 +1997 UNITED ST5 MFGR#1421 17858301 +1997 UNITED ST5 MFGR#1422 14080987 +1997 UNITED ST5 MFGR#1423 12535186 +1997 UNITED ST5 MFGR#1424 3221960 +1997 UNITED ST5 MFGR#1425 8950317 +1997 UNITED ST5 MFGR#1426 13161486 +1997 UNITED ST5 MFGR#1427 6149737 +1997 UNITED ST5 MFGR#1428 6796520 +1997 UNITED ST5 MFGR#1429 26026300 +1997 UNITED ST5 MFGR#143 6712572 +1997 UNITED ST5 MFGR#1430 16092131 +1997 UNITED ST5 MFGR#1431 3425291 +1997 UNITED ST5 MFGR#1432 14630175 +1997 UNITED ST5 MFGR#1433 4967585 +1997 UNITED ST5 MFGR#1434 6716261 +1997 UNITED ST5 MFGR#1435 6290790 +1997 UNITED ST5 MFGR#1436 6787699 +1997 UNITED ST5 MFGR#1437 6045168 +1997 UNITED ST5 MFGR#1438 1082877 +1997 UNITED ST5 MFGR#1439 5268611 +1997 UNITED ST5 MFGR#144 8900840 +1997 UNITED ST5 MFGR#1440 2644336 +1997 UNITED ST5 MFGR#145 18054844 +1997 UNITED ST5 MFGR#146 4893668 +1997 UNITED ST5 MFGR#147 9615891 +1997 UNITED ST5 MFGR#148 1426124 +1997 UNITED ST5 MFGR#149 7320726 +1997 UNITED ST6 MFGR#141 10051337 +1997 UNITED ST6 MFGR#1410 23352963 +1997 UNITED ST6 MFGR#1411 1537303 +1997 UNITED ST6 MFGR#1412 4332646 +1997 UNITED ST6 MFGR#1413 14686752 +1997 UNITED ST6 MFGR#1414 7226409 +1997 UNITED ST6 MFGR#1415 15062686 +1997 UNITED ST6 MFGR#1416 8606480 +1997 UNITED ST6 MFGR#1417 7662761 +1997 UNITED ST6 MFGR#1418 1810860 +1997 UNITED ST6 MFGR#1419 20179568 +1997 UNITED ST6 MFGR#142 15506072 +1997 UNITED ST6 MFGR#1420 3844271 +1997 UNITED ST6 MFGR#1421 10608709 +1997 UNITED ST6 MFGR#1422 16083699 +1997 UNITED ST6 MFGR#1423 6789571 +1997 UNITED ST6 MFGR#1424 17631950 +1997 UNITED ST6 MFGR#1425 8477221 +1997 UNITED ST6 MFGR#1426 16847029 +1997 UNITED ST6 MFGR#1428 17314796 +1997 UNITED ST6 MFGR#1429 19175871 +1997 UNITED ST6 MFGR#143 8775848 +1997 UNITED ST6 MFGR#1431 5319063 +1997 UNITED ST6 MFGR#1432 7875606 +1997 UNITED ST6 MFGR#1433 9305264 +1997 UNITED ST6 MFGR#1434 8050404 +1997 UNITED ST6 MFGR#1435 17585836 +1997 UNITED ST6 MFGR#1436 13745758 +1997 UNITED ST6 MFGR#1438 14536324 +1997 UNITED ST6 MFGR#1439 11451329 +1997 UNITED ST6 MFGR#144 10794837 +1997 UNITED ST6 MFGR#1440 5719238 +1997 UNITED ST6 MFGR#145 25449058 +1997 UNITED ST6 MFGR#146 17821996 +1997 UNITED ST6 MFGR#147 18922333 +1997 UNITED ST6 MFGR#148 10825718 +1997 UNITED ST6 MFGR#149 14950868 +1997 UNITED ST7 MFGR#141 8291476 +1997 UNITED ST7 MFGR#1410 4622953 +1997 UNITED ST7 MFGR#1411 14885507 +1997 UNITED ST7 MFGR#1412 19011375 +1997 UNITED ST7 MFGR#1413 31178370 +1997 UNITED ST7 MFGR#1414 16429613 +1997 UNITED ST7 MFGR#1415 3091832 +1997 UNITED ST7 MFGR#1416 17008243 +1997 UNITED ST7 MFGR#1417 15609283 +1997 UNITED ST7 MFGR#1418 5059394 +1997 UNITED ST7 MFGR#1419 7234721 +1997 UNITED ST7 MFGR#142 6898938 +1997 UNITED ST7 MFGR#1420 6426590 +1997 UNITED ST7 MFGR#1421 6043221 +1997 UNITED ST7 MFGR#1422 13273021 +1997 UNITED ST7 MFGR#1423 30537249 +1997 UNITED ST7 MFGR#1424 9224296 +1997 UNITED ST7 MFGR#1425 12799069 +1997 UNITED ST7 MFGR#1426 30277891 +1997 UNITED ST7 MFGR#1427 8331493 +1997 UNITED ST7 MFGR#1428 16205423 +1997 UNITED ST7 MFGR#1429 14396627 +1997 UNITED ST7 MFGR#143 10209623 +1997 UNITED ST7 MFGR#1430 16556758 +1997 UNITED ST7 MFGR#1431 15775327 +1997 UNITED ST7 MFGR#1432 24602989 +1997 UNITED ST7 MFGR#1433 22927717 +1997 UNITED ST7 MFGR#1434 9486462 +1997 UNITED ST7 MFGR#1435 14254290 +1997 UNITED ST7 MFGR#1436 8645365 +1997 UNITED ST7 MFGR#1437 19026166 +1997 UNITED ST7 MFGR#1438 11291550 +1997 UNITED ST7 MFGR#1439 9590292 +1997 UNITED ST7 MFGR#144 14110404 +1997 UNITED ST7 MFGR#1440 21877355 +1997 UNITED ST7 MFGR#145 8246717 +1997 UNITED ST7 MFGR#146 9785781 +1997 UNITED ST7 MFGR#147 13412691 +1997 UNITED ST7 MFGR#148 14235939 +1997 UNITED ST7 MFGR#149 23450309 +1997 UNITED ST8 MFGR#141 7323416 +1997 UNITED ST8 MFGR#1410 7191961 +1997 UNITED ST8 MFGR#1411 2425849 +1997 UNITED ST8 MFGR#1412 26725641 +1997 UNITED ST8 MFGR#1413 7915896 +1997 UNITED ST8 MFGR#1414 14359044 +1997 UNITED ST8 MFGR#1415 15834927 +1997 UNITED ST8 MFGR#1416 7464470 +1997 UNITED ST8 MFGR#1417 15274496 +1997 UNITED ST8 MFGR#1418 3261391 +1997 UNITED ST8 MFGR#1419 8020386 +1997 UNITED ST8 MFGR#142 5985550 +1997 UNITED ST8 MFGR#1420 9441461 +1997 UNITED ST8 MFGR#1421 2998735 +1997 UNITED ST8 MFGR#1423 1565415 +1997 UNITED ST8 MFGR#1424 5613094 +1997 UNITED ST8 MFGR#1425 7186734 +1997 UNITED ST8 MFGR#1426 3153117 +1997 UNITED ST8 MFGR#1427 11303043 +1997 UNITED ST8 MFGR#1428 3564003 +1997 UNITED ST8 MFGR#143 6357477 +1997 UNITED ST8 MFGR#1430 6633950 +1997 UNITED ST8 MFGR#1431 867285 +1997 UNITED ST8 MFGR#1432 9962884 +1997 UNITED ST8 MFGR#1433 13708011 +1997 UNITED ST8 MFGR#1434 7357067 +1997 UNITED ST8 MFGR#1435 11993453 +1997 UNITED ST8 MFGR#1436 7231938 +1997 UNITED ST8 MFGR#1437 18632056 +1997 UNITED ST8 MFGR#1438 3609384 +1997 UNITED ST8 MFGR#144 4822734 +1997 UNITED ST8 MFGR#1440 2521917 +1997 UNITED ST8 MFGR#145 11683385 +1997 UNITED ST8 MFGR#146 3350415 +1997 UNITED ST8 MFGR#147 8425393 +1997 UNITED ST8 MFGR#148 15005341 +1997 UNITED ST8 MFGR#149 9018101 +1997 UNITED ST9 MFGR#141 21838850 +1997 UNITED ST9 MFGR#1410 8455387 +1997 UNITED ST9 MFGR#1411 27984689 +1997 UNITED ST9 MFGR#1412 26491401 +1997 UNITED ST9 MFGR#1413 30237705 +1997 UNITED ST9 MFGR#1414 8957476 +1997 UNITED ST9 MFGR#1415 35391583 +1997 UNITED ST9 MFGR#1416 8923782 +1997 UNITED ST9 MFGR#1417 3210984 +1997 UNITED ST9 MFGR#1418 12102607 +1997 UNITED ST9 MFGR#1419 14468627 +1997 UNITED ST9 MFGR#142 33363582 +1997 UNITED ST9 MFGR#1420 13139731 +1997 UNITED ST9 MFGR#1421 8410650 +1997 UNITED ST9 MFGR#1422 11268410 +1997 UNITED ST9 MFGR#1423 2846614 +1997 UNITED ST9 MFGR#1424 2305872 +1997 UNITED ST9 MFGR#1425 4896060 +1997 UNITED ST9 MFGR#1426 31544772 +1997 UNITED ST9 MFGR#1427 4113085 +1997 UNITED ST9 MFGR#1428 11137081 +1997 UNITED ST9 MFGR#1429 10633230 +1997 UNITED ST9 MFGR#143 11438203 +1997 UNITED ST9 MFGR#1430 2727389 +1997 UNITED ST9 MFGR#1431 10548538 +1997 UNITED ST9 MFGR#1433 26564495 +1997 UNITED ST9 MFGR#1434 11948206 +1997 UNITED ST9 MFGR#1435 2201909 +1997 UNITED ST9 MFGR#1436 10347631 +1997 UNITED ST9 MFGR#1437 40051690 +1997 UNITED ST9 MFGR#1438 2662945 +1997 UNITED ST9 MFGR#1439 18011455 +1997 UNITED ST9 MFGR#144 10202784 +1997 UNITED ST9 MFGR#1440 11123196 +1997 UNITED ST9 MFGR#145 11326624 +1997 UNITED ST9 MFGR#146 27825456 +1997 UNITED ST9 MFGR#147 17579502 +1997 UNITED ST9 MFGR#148 17001116 +1998 UNITED ST0 MFGR#141 12484786 +1998 UNITED ST0 MFGR#1410 5225999 +1998 UNITED ST0 MFGR#1411 2633395 +1998 UNITED ST0 MFGR#1412 9479536 +1998 UNITED ST0 MFGR#1413 18468711 +1998 UNITED ST0 MFGR#1414 19440601 +1998 UNITED ST0 MFGR#1415 16894643 +1998 UNITED ST0 MFGR#1416 7520117 +1998 UNITED ST0 MFGR#1417 464576 +1998 UNITED ST0 MFGR#1418 5171747 +1998 UNITED ST0 MFGR#1419 5970985 +1998 UNITED ST0 MFGR#142 23830409 +1998 UNITED ST0 MFGR#1420 2568835 +1998 UNITED ST0 MFGR#1421 6096442 +1998 UNITED ST0 MFGR#1422 10099456 +1998 UNITED ST0 MFGR#1423 24617746 +1998 UNITED ST0 MFGR#1424 22853888 +1998 UNITED ST0 MFGR#1425 25622112 +1998 UNITED ST0 MFGR#1426 9630701 +1998 UNITED ST0 MFGR#1427 4952625 +1998 UNITED ST0 MFGR#1428 4675471 +1998 UNITED ST0 MFGR#1429 19713527 +1998 UNITED ST0 MFGR#143 4971265 +1998 UNITED ST0 MFGR#1430 13948705 +1998 UNITED ST0 MFGR#1431 2955181 +1998 UNITED ST0 MFGR#1432 17946759 +1998 UNITED ST0 MFGR#1433 23325027 +1998 UNITED ST0 MFGR#1434 8580738 +1998 UNITED ST0 MFGR#1435 6663914 +1998 UNITED ST0 MFGR#1436 13468582 +1998 UNITED ST0 MFGR#1437 9419979 +1998 UNITED ST0 MFGR#1438 16164522 +1998 UNITED ST0 MFGR#1439 8692866 +1998 UNITED ST0 MFGR#144 17338711 +1998 UNITED ST0 MFGR#1440 8513736 +1998 UNITED ST0 MFGR#145 6212569 +1998 UNITED ST0 MFGR#146 2702349 +1998 UNITED ST0 MFGR#149 8844540 +1998 UNITED ST1 MFGR#141 13449110 +1998 UNITED ST1 MFGR#1410 5224626 +1998 UNITED ST1 MFGR#1411 16046399 +1998 UNITED ST1 MFGR#1412 8424392 +1998 UNITED ST1 MFGR#1413 7634982 +1998 UNITED ST1 MFGR#1414 8711782 +1998 UNITED ST1 MFGR#1415 5991201 +1998 UNITED ST1 MFGR#1416 11059645 +1998 UNITED ST1 MFGR#1417 8108087 +1998 UNITED ST1 MFGR#1418 1710875 +1998 UNITED ST1 MFGR#1419 452298 +1998 UNITED ST1 MFGR#142 6186526 +1998 UNITED ST1 MFGR#1421 4766963 +1998 UNITED ST1 MFGR#1422 5556563 +1998 UNITED ST1 MFGR#1423 12110257 +1998 UNITED ST1 MFGR#1424 8924948 +1998 UNITED ST1 MFGR#1425 5684710 +1998 UNITED ST1 MFGR#1426 6844444 +1998 UNITED ST1 MFGR#1427 7625525 +1998 UNITED ST1 MFGR#1428 5227395 +1998 UNITED ST1 MFGR#1429 2855272 +1998 UNITED ST1 MFGR#143 6964811 +1998 UNITED ST1 MFGR#1430 16952380 +1998 UNITED ST1 MFGR#1431 7655883 +1998 UNITED ST1 MFGR#1432 2975960 +1998 UNITED ST1 MFGR#1433 10175594 +1998 UNITED ST1 MFGR#1434 26358848 +1998 UNITED ST1 MFGR#1435 11408623 +1998 UNITED ST1 MFGR#1436 4438242 +1998 UNITED ST1 MFGR#1437 10014906 +1998 UNITED ST1 MFGR#1438 18817171 +1998 UNITED ST1 MFGR#1439 2129973 +1998 UNITED ST1 MFGR#144 9512727 +1998 UNITED ST1 MFGR#1440 20185179 +1998 UNITED ST1 MFGR#145 823605 +1998 UNITED ST1 MFGR#146 9358026 +1998 UNITED ST1 MFGR#147 6552312 +1998 UNITED ST1 MFGR#148 9144461 +1998 UNITED ST1 MFGR#149 2149639 +1998 UNITED ST2 MFGR#1410 8044368 +1998 UNITED ST2 MFGR#1413 727343 +1998 UNITED ST2 MFGR#1414 251088 +1998 UNITED ST2 MFGR#1416 5224696 +1998 UNITED ST2 MFGR#1419 7917142 +1998 UNITED ST2 MFGR#1420 9848986 +1998 UNITED ST2 MFGR#1427 4816976 +1998 UNITED ST2 MFGR#1429 4196069 +1998 UNITED ST2 MFGR#143 10073044 +1998 UNITED ST2 MFGR#1431 877123 +1998 UNITED ST2 MFGR#1433 3740836 +1998 UNITED ST2 MFGR#1434 857691 +1998 UNITED ST2 MFGR#1435 2277870 +1998 UNITED ST2 MFGR#145 4906521 +1998 UNITED ST2 MFGR#149 842348 +1998 UNITED ST3 MFGR#141 4181582 +1998 UNITED ST3 MFGR#1410 3169761 +1998 UNITED ST3 MFGR#1412 6590440 +1998 UNITED ST3 MFGR#1413 2015369 +1998 UNITED ST3 MFGR#1414 4408466 +1998 UNITED ST3 MFGR#1415 3963826 +1998 UNITED ST3 MFGR#1416 3179540 +1998 UNITED ST3 MFGR#1417 11440999 +1998 UNITED ST3 MFGR#1418 8725751 +1998 UNITED ST3 MFGR#1419 10646990 +1998 UNITED ST3 MFGR#142 5056172 +1998 UNITED ST3 MFGR#1420 3583150 +1998 UNITED ST3 MFGR#1421 7085689 +1998 UNITED ST3 MFGR#1422 20858904 +1998 UNITED ST3 MFGR#1423 12825743 +1998 UNITED ST3 MFGR#1424 3533518 +1998 UNITED ST3 MFGR#1425 4938324 +1998 UNITED ST3 MFGR#1426 15164664 +1998 UNITED ST3 MFGR#1427 2619162 +1998 UNITED ST3 MFGR#1428 7723699 +1998 UNITED ST3 MFGR#143 24362893 +1998 UNITED ST3 MFGR#1430 7139650 +1998 UNITED ST3 MFGR#1431 10306394 +1998 UNITED ST3 MFGR#1432 14769004 +1998 UNITED ST3 MFGR#1433 9610545 +1998 UNITED ST3 MFGR#1435 6611760 +1998 UNITED ST3 MFGR#1436 941348 +1998 UNITED ST3 MFGR#1437 4802162 +1998 UNITED ST3 MFGR#1438 5079750 +1998 UNITED ST3 MFGR#1439 5905118 +1998 UNITED ST3 MFGR#144 5436839 +1998 UNITED ST3 MFGR#1440 9815517 +1998 UNITED ST3 MFGR#145 7580271 +1998 UNITED ST3 MFGR#146 14051404 +1998 UNITED ST3 MFGR#147 7786017 +1998 UNITED ST3 MFGR#149 1271813 +1998 UNITED ST4 MFGR#141 26703183 +1998 UNITED ST4 MFGR#1410 18094129 +1998 UNITED ST4 MFGR#1411 615999 +1998 UNITED ST4 MFGR#1412 13988709 +1998 UNITED ST4 MFGR#1413 10115647 +1998 UNITED ST4 MFGR#1414 1586551 +1998 UNITED ST4 MFGR#1415 14357244 +1998 UNITED ST4 MFGR#1416 9430036 +1998 UNITED ST4 MFGR#1417 17361773 +1998 UNITED ST4 MFGR#1418 10187338 +1998 UNITED ST4 MFGR#1419 7116094 +1998 UNITED ST4 MFGR#142 18454502 +1998 UNITED ST4 MFGR#1420 1621340 +1998 UNITED ST4 MFGR#1422 13861291 +1998 UNITED ST4 MFGR#1423 4208006 +1998 UNITED ST4 MFGR#1424 3242576 +1998 UNITED ST4 MFGR#1425 1470642 +1998 UNITED ST4 MFGR#1426 11987205 +1998 UNITED ST4 MFGR#1427 12975399 +1998 UNITED ST4 MFGR#1428 11756155 +1998 UNITED ST4 MFGR#1429 6956191 +1998 UNITED ST4 MFGR#143 11499476 +1998 UNITED ST4 MFGR#1430 4355096 +1998 UNITED ST4 MFGR#1431 20166612 +1998 UNITED ST4 MFGR#1432 8849789 +1998 UNITED ST4 MFGR#1433 10786037 +1998 UNITED ST4 MFGR#1435 13208151 +1998 UNITED ST4 MFGR#1436 4747808 +1998 UNITED ST4 MFGR#1437 12999351 +1998 UNITED ST4 MFGR#1438 17360787 +1998 UNITED ST4 MFGR#1439 2557841 +1998 UNITED ST4 MFGR#144 22030911 +1998 UNITED ST4 MFGR#145 5272280 +1998 UNITED ST4 MFGR#146 4307958 +1998 UNITED ST4 MFGR#147 7995050 +1998 UNITED ST4 MFGR#148 2348889 +1998 UNITED ST5 MFGR#141 8396190 +1998 UNITED ST5 MFGR#1410 17422889 +1998 UNITED ST5 MFGR#1411 764246 +1998 UNITED ST5 MFGR#1413 8051561 +1998 UNITED ST5 MFGR#1414 1500867 +1998 UNITED ST5 MFGR#1416 3874315 +1998 UNITED ST5 MFGR#1417 7550747 +1998 UNITED ST5 MFGR#1418 14241755 +1998 UNITED ST5 MFGR#1419 4893846 +1998 UNITED ST5 MFGR#142 5418299 +1998 UNITED ST5 MFGR#1420 3810727 +1998 UNITED ST5 MFGR#1421 3321336 +1998 UNITED ST5 MFGR#1423 5021100 +1998 UNITED ST5 MFGR#1424 6125645 +1998 UNITED ST5 MFGR#1426 880680 +1998 UNITED ST5 MFGR#1427 1284890 +1998 UNITED ST5 MFGR#1428 3813108 +1998 UNITED ST5 MFGR#1429 9429868 +1998 UNITED ST5 MFGR#143 15699981 +1998 UNITED ST5 MFGR#1430 7977654 +1998 UNITED ST5 MFGR#1431 5545969 +1998 UNITED ST5 MFGR#1432 1803434 +1998 UNITED ST5 MFGR#1435 12957056 +1998 UNITED ST5 MFGR#1436 20666755 +1998 UNITED ST5 MFGR#1437 9536733 +1998 UNITED ST5 MFGR#1439 11784214 +1998 UNITED ST5 MFGR#1440 613089 +1998 UNITED ST5 MFGR#145 8237944 +1998 UNITED ST5 MFGR#147 5370351 +1998 UNITED ST6 MFGR#141 6683977 +1998 UNITED ST6 MFGR#1411 11589040 +1998 UNITED ST6 MFGR#1412 4800954 +1998 UNITED ST6 MFGR#1416 5848432 +1998 UNITED ST6 MFGR#1417 1804324 +1998 UNITED ST6 MFGR#1418 4642727 +1998 UNITED ST6 MFGR#1419 10079621 +1998 UNITED ST6 MFGR#142 2533570 +1998 UNITED ST6 MFGR#1420 9016258 +1998 UNITED ST6 MFGR#1421 7636150 +1998 UNITED ST6 MFGR#1422 1272436 +1998 UNITED ST6 MFGR#1423 9600097 +1998 UNITED ST6 MFGR#1424 7036762 +1998 UNITED ST6 MFGR#1425 10867426 +1998 UNITED ST6 MFGR#1426 7033994 +1998 UNITED ST6 MFGR#1427 6481679 +1998 UNITED ST6 MFGR#1428 6269185 +1998 UNITED ST6 MFGR#1429 12055631 +1998 UNITED ST6 MFGR#143 6500459 +1998 UNITED ST6 MFGR#1430 6816243 +1998 UNITED ST6 MFGR#1432 7752973 +1998 UNITED ST6 MFGR#1436 6292813 +1998 UNITED ST6 MFGR#1437 2334894 +1998 UNITED ST6 MFGR#1438 4459455 +1998 UNITED ST6 MFGR#144 640220 +1998 UNITED ST6 MFGR#1440 748200 +1998 UNITED ST6 MFGR#145 7948794 +1998 UNITED ST6 MFGR#146 2734333 +1998 UNITED ST6 MFGR#147 2995632 +1998 UNITED ST6 MFGR#148 10695287 +1998 UNITED ST6 MFGR#149 1475864 +1998 UNITED ST7 MFGR#141 4816637 +1998 UNITED ST7 MFGR#1410 24393912 +1998 UNITED ST7 MFGR#1411 1283038 +1998 UNITED ST7 MFGR#1412 12607459 +1998 UNITED ST7 MFGR#1413 15421293 +1998 UNITED ST7 MFGR#1414 7681668 +1998 UNITED ST7 MFGR#1415 7138377 +1998 UNITED ST7 MFGR#1416 9246990 +1998 UNITED ST7 MFGR#1417 5242972 +1998 UNITED ST7 MFGR#1419 13548941 +1998 UNITED ST7 MFGR#142 17758762 +1998 UNITED ST7 MFGR#1420 11980437 +1998 UNITED ST7 MFGR#1421 11166952 +1998 UNITED ST7 MFGR#1422 9231883 +1998 UNITED ST7 MFGR#1424 5796389 +1998 UNITED ST7 MFGR#1425 4943582 +1998 UNITED ST7 MFGR#1426 16361100 +1998 UNITED ST7 MFGR#1427 9659296 +1998 UNITED ST7 MFGR#1428 2998020 +1998 UNITED ST7 MFGR#1429 5766768 +1998 UNITED ST7 MFGR#143 5526167 +1998 UNITED ST7 MFGR#1430 4191892 +1998 UNITED ST7 MFGR#1431 12979559 +1998 UNITED ST7 MFGR#1432 5832068 +1998 UNITED ST7 MFGR#1433 11410227 +1998 UNITED ST7 MFGR#1434 10525792 +1998 UNITED ST7 MFGR#1435 5982420 +1998 UNITED ST7 MFGR#1436 7142934 +1998 UNITED ST7 MFGR#1437 7499671 +1998 UNITED ST7 MFGR#1438 9177564 +1998 UNITED ST7 MFGR#1439 1974444 +1998 UNITED ST7 MFGR#144 2428463 +1998 UNITED ST7 MFGR#1440 3300838 +1998 UNITED ST7 MFGR#146 1771645 +1998 UNITED ST7 MFGR#147 4681835 +1998 UNITED ST7 MFGR#148 7124367 +1998 UNITED ST7 MFGR#149 11662970 +1998 UNITED ST8 MFGR#141 12753354 +1998 UNITED ST8 MFGR#1410 1097292 +1998 UNITED ST8 MFGR#1411 4069881 +1998 UNITED ST8 MFGR#1414 9474831 +1998 UNITED ST8 MFGR#1415 6406304 +1998 UNITED ST8 MFGR#1418 2392133 +1998 UNITED ST8 MFGR#1419 2953230 +1998 UNITED ST8 MFGR#142 10702563 +1998 UNITED ST8 MFGR#1420 3719432 +1998 UNITED ST8 MFGR#1421 3854657 +1998 UNITED ST8 MFGR#1422 9711968 +1998 UNITED ST8 MFGR#1424 1102516 +1998 UNITED ST8 MFGR#1426 8937532 +1998 UNITED ST8 MFGR#1427 5255637 +1998 UNITED ST8 MFGR#1428 1695112 +1998 UNITED ST8 MFGR#143 6231349 +1998 UNITED ST8 MFGR#1430 4678664 +1998 UNITED ST8 MFGR#1431 2459741 +1998 UNITED ST8 MFGR#1433 1724339 +1998 UNITED ST8 MFGR#1434 8222328 +1998 UNITED ST8 MFGR#1435 4087165 +1998 UNITED ST8 MFGR#1437 7379288 +1998 UNITED ST8 MFGR#1439 1947163 +1998 UNITED ST8 MFGR#144 21846266 +1998 UNITED ST8 MFGR#146 13768113 +1998 UNITED ST8 MFGR#147 5279643 +1998 UNITED ST8 MFGR#149 15221410 +1998 UNITED ST9 MFGR#141 5544000 +1998 UNITED ST9 MFGR#1410 7971104 +1998 UNITED ST9 MFGR#1411 26426609 +1998 UNITED ST9 MFGR#1412 1701777 +1998 UNITED ST9 MFGR#1413 18786672 +1998 UNITED ST9 MFGR#1414 15787780 +1998 UNITED ST9 MFGR#1415 11214818 +1998 UNITED ST9 MFGR#1416 2076120 +1998 UNITED ST9 MFGR#1417 14310334 +1998 UNITED ST9 MFGR#1418 3558355 +1998 UNITED ST9 MFGR#1419 6134867 +1998 UNITED ST9 MFGR#142 257897 +1998 UNITED ST9 MFGR#1420 7716338 +1998 UNITED ST9 MFGR#1421 10291775 +1998 UNITED ST9 MFGR#1422 6105716 +1998 UNITED ST9 MFGR#1423 11378613 +1998 UNITED ST9 MFGR#1424 1904550 +1998 UNITED ST9 MFGR#1425 18831147 +1998 UNITED ST9 MFGR#1426 18562642 +1998 UNITED ST9 MFGR#1427 12565890 +1998 UNITED ST9 MFGR#1428 10720914 +1998 UNITED ST9 MFGR#1429 21940718 +1998 UNITED ST9 MFGR#143 21480722 +1998 UNITED ST9 MFGR#1430 30935173 +1998 UNITED ST9 MFGR#1431 7793012 +1998 UNITED ST9 MFGR#1432 10026104 +1998 UNITED ST9 MFGR#1433 17874435 +1998 UNITED ST9 MFGR#1434 5493785 +1998 UNITED ST9 MFGR#1435 10738769 +1998 UNITED ST9 MFGR#1436 43504 +1998 UNITED ST9 MFGR#1437 13559646 +1998 UNITED ST9 MFGR#1438 12654258 +1998 UNITED ST9 MFGR#1439 5972923 +1998 UNITED ST9 MFGR#144 15702123 +1998 UNITED ST9 MFGR#1440 6681029 +1998 UNITED ST9 MFGR#145 920718 +1998 UNITED ST9 MFGR#146 10435931 +1998 UNITED ST9 MFGR#147 4759326 +1998 UNITED ST9 MFGR#148 9970848 +1998 UNITED ST9 MFGR#149 7510940 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.1.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.1.out new file mode 100644 index 00000000000000..ba450f16d5c36a --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.1.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1.1 -- +446031203850 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.2.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.2.out new file mode 100644 index 00000000000000..9e6e59831d352c --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.2.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1.2 -- +98714004603 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.3.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.3.out new file mode 100644 index 00000000000000..dd5cabbd7c348e --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q1.3.out @@ -0,0 +1,4 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q1.3 -- +26232701005 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.1.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.1.out new file mode 100644 index 00000000000000..e1ebddab4123d7 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.1.out @@ -0,0 +1,283 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q2.1 -- +567838207 1992 MFGR#121 +610663790 1992 MFGR#1210 +550769662 1992 MFGR#1211 +649205856 1992 MFGR#1212 +624031241 1992 MFGR#1213 +670488468 1992 MFGR#1214 +633152470 1992 MFGR#1215 +674846781 1992 MFGR#1216 +675093435 1992 MFGR#1217 +600202070 1992 MFGR#1218 +538043594 1992 MFGR#1219 +655326672 1992 MFGR#122 +540262882 1992 MFGR#1220 +556120633 1992 MFGR#1221 +590762777 1992 MFGR#1222 +535448651 1992 MFGR#1223 +703752611 1992 MFGR#1224 +570832868 1992 MFGR#1225 +614061593 1992 MFGR#1226 +581759388 1992 MFGR#1227 +644642592 1992 MFGR#1228 +640858430 1992 MFGR#1229 +789755835 1992 MFGR#123 +468535087 1992 MFGR#1230 +592436656 1992 MFGR#1231 +664275152 1992 MFGR#1232 +613885100 1992 MFGR#1233 +667399281 1992 MFGR#1234 +640290070 1992 MFGR#1235 +501892561 1992 MFGR#1236 +591481503 1992 MFGR#1237 +477423770 1992 MFGR#1238 +638259374 1992 MFGR#1239 +572354196 1992 MFGR#124 +740479248 1992 MFGR#1240 +478777095 1992 MFGR#125 +592174616 1992 MFGR#126 +706151632 1992 MFGR#127 +542306646 1992 MFGR#128 +581987352 1992 MFGR#129 +823087702 1993 MFGR#121 +648160706 1993 MFGR#1210 +634743898 1993 MFGR#1211 +785639283 1993 MFGR#1212 +638255029 1993 MFGR#1213 +616837237 1993 MFGR#1214 +634687975 1993 MFGR#1215 +638353900 1993 MFGR#1216 +663372951 1993 MFGR#1217 +683985855 1993 MFGR#1218 +646950033 1993 MFGR#1219 +622532984 1993 MFGR#122 +530830127 1993 MFGR#1220 +543346337 1993 MFGR#1221 +756921203 1993 MFGR#1222 +533544350 1993 MFGR#1223 +915916085 1993 MFGR#1224 +473007381 1993 MFGR#1225 +739036124 1993 MFGR#1226 +592178887 1993 MFGR#1227 +583507058 1993 MFGR#1228 +617453491 1993 MFGR#1229 +637863868 1993 MFGR#123 +625534310 1993 MFGR#1230 +580327635 1993 MFGR#1231 +697373098 1993 MFGR#1232 +515571416 1993 MFGR#1233 +651935758 1993 MFGR#1234 +575779480 1993 MFGR#1235 +591878667 1993 MFGR#1236 +609618576 1993 MFGR#1237 +444614010 1993 MFGR#1238 +595256327 1993 MFGR#1239 +660586237 1993 MFGR#124 +788730059 1993 MFGR#1240 +616224539 1993 MFGR#125 +617126754 1993 MFGR#126 +654438324 1993 MFGR#127 +731657001 1993 MFGR#128 +548048395 1993 MFGR#129 +564405648 1994 MFGR#121 +645404849 1994 MFGR#1210 +631620635 1994 MFGR#1211 +568332348 1994 MFGR#1212 +678785857 1994 MFGR#1213 +534002330 1994 MFGR#1214 +654400242 1994 MFGR#1215 +558646341 1994 MFGR#1216 +687845641 1994 MFGR#1217 +546674347 1994 MFGR#1218 +567272942 1994 MFGR#1219 +659884062 1994 MFGR#122 +562582172 1994 MFGR#1220 +598618997 1994 MFGR#1221 +601016441 1994 MFGR#1222 +555134404 1994 MFGR#1223 +737422302 1994 MFGR#1224 +570745955 1994 MFGR#1225 +746302245 1994 MFGR#1226 +651707481 1994 MFGR#1227 +573693547 1994 MFGR#1228 +647918373 1994 MFGR#1229 +580449592 1994 MFGR#123 +493270412 1994 MFGR#1230 +603546148 1994 MFGR#1231 +719865331 1994 MFGR#1232 +638982238 1994 MFGR#1233 +743247677 1994 MFGR#1234 +598680959 1994 MFGR#1235 +615726097 1994 MFGR#1236 +542569815 1994 MFGR#1237 +573510781 1994 MFGR#1238 +579855853 1994 MFGR#1239 +684573322 1994 MFGR#124 +873735737 1994 MFGR#1240 +560488304 1994 MFGR#125 +657036514 1994 MFGR#126 +622571183 1994 MFGR#127 +586845664 1994 MFGR#128 +534541525 1994 MFGR#129 +706469511 1995 MFGR#121 +602892803 1995 MFGR#1210 +645166092 1995 MFGR#1211 +613289283 1995 MFGR#1212 +599586479 1995 MFGR#1213 +562570804 1995 MFGR#1214 +672528755 1995 MFGR#1215 +669000972 1995 MFGR#1216 +725362449 1995 MFGR#1217 +657026635 1995 MFGR#1218 +519659003 1995 MFGR#1219 +724727741 1995 MFGR#122 +517956131 1995 MFGR#1220 +635741351 1995 MFGR#1221 +564368410 1995 MFGR#1222 +600665149 1995 MFGR#1223 +762700351 1995 MFGR#1224 +671669586 1995 MFGR#1225 +572568748 1995 MFGR#1226 +530361300 1995 MFGR#1227 +633357085 1995 MFGR#1228 +547960244 1995 MFGR#1229 +660711077 1995 MFGR#123 +602735858 1995 MFGR#1230 +499852146 1995 MFGR#1231 +715300753 1995 MFGR#1232 +557149571 1995 MFGR#1233 +710023059 1995 MFGR#1234 +622425239 1995 MFGR#1235 +634565501 1995 MFGR#1236 +572847270 1995 MFGR#1237 +549318912 1995 MFGR#1238 +593851712 1995 MFGR#1239 +585421815 1995 MFGR#124 +707207888 1995 MFGR#1240 +538246872 1995 MFGR#125 +605799021 1995 MFGR#126 +665978112 1995 MFGR#127 +646960956 1995 MFGR#128 +508749401 1995 MFGR#129 +523879145 1996 MFGR#121 +643645053 1996 MFGR#1210 +595065339 1996 MFGR#1211 +674626440 1996 MFGR#1212 +496297087 1996 MFGR#1213 +583249505 1996 MFGR#1214 +702184857 1996 MFGR#1215 +601809334 1996 MFGR#1216 +704898387 1996 MFGR#1217 +528843086 1996 MFGR#1218 +586246330 1996 MFGR#1219 +712110492 1996 MFGR#122 +518444215 1996 MFGR#1220 +499319414 1996 MFGR#1221 +679469356 1996 MFGR#1222 +628762754 1996 MFGR#1223 +724844856 1996 MFGR#1224 +660620587 1996 MFGR#1225 +667674729 1996 MFGR#1226 +483838085 1996 MFGR#1227 +609855391 1996 MFGR#1228 +658959557 1996 MFGR#1229 +566217852 1996 MFGR#123 +528879998 1996 MFGR#1230 +589481194 1996 MFGR#1231 +702805896 1996 MFGR#1232 +663679947 1996 MFGR#1233 +571149450 1996 MFGR#1234 +478648074 1996 MFGR#1235 +568249365 1996 MFGR#1236 +592616167 1996 MFGR#1237 +466676148 1996 MFGR#1238 +670693719 1996 MFGR#1239 +560667719 1996 MFGR#124 +821167950 1996 MFGR#1240 +476864333 1996 MFGR#125 +558030884 1996 MFGR#126 +635873891 1996 MFGR#127 +551010618 1996 MFGR#128 +560570630 1996 MFGR#129 +587013207 1997 MFGR#121 +616287892 1997 MFGR#1210 +548588761 1997 MFGR#1211 +589593892 1997 MFGR#1212 +424306670 1997 MFGR#1213 +511971910 1997 MFGR#1214 +631772246 1997 MFGR#1215 +692135140 1997 MFGR#1216 +777994957 1997 MFGR#1217 +707053720 1997 MFGR#1218 +561169527 1997 MFGR#1219 +664916245 1997 MFGR#122 +594466157 1997 MFGR#1220 +588848171 1997 MFGR#1221 +528988960 1997 MFGR#1222 +537098211 1997 MFGR#1223 +674763166 1997 MFGR#1224 +450402292 1997 MFGR#1225 +701360722 1997 MFGR#1226 +506011570 1997 MFGR#1227 +585578737 1997 MFGR#1228 +622744016 1997 MFGR#1229 +646503168 1997 MFGR#123 +571800941 1997 MFGR#1230 +502601790 1997 MFGR#1231 +677924656 1997 MFGR#1232 +534455976 1997 MFGR#1233 +714934715 1997 MFGR#1234 +767151420 1997 MFGR#1235 +618877179 1997 MFGR#1236 +639638057 1997 MFGR#1237 +401953419 1997 MFGR#1238 +610756714 1997 MFGR#1239 +543248087 1997 MFGR#124 +675132692 1997 MFGR#1240 +479099365 1997 MFGR#125 +570696568 1997 MFGR#126 +583074592 1997 MFGR#127 +695133104 1997 MFGR#128 +655638776 1997 MFGR#129 +344575925 1998 MFGR#121 +417152416 1998 MFGR#1210 +317068168 1998 MFGR#1211 +374341516 1998 MFGR#1212 +332740903 1998 MFGR#1213 +304873002 1998 MFGR#1214 +366101132 1998 MFGR#1215 +379133898 1998 MFGR#1216 +359508497 1998 MFGR#1217 +320623334 1998 MFGR#1218 +346182862 1998 MFGR#1219 +312440027 1998 MFGR#122 +348123961 1998 MFGR#1220 +339845398 1998 MFGR#1221 +355416161 1998 MFGR#1222 +344889822 1998 MFGR#1223 +396906691 1998 MFGR#1224 +290208878 1998 MFGR#1225 +419415707 1998 MFGR#1226 +358466340 1998 MFGR#1227 +251549955 1998 MFGR#1228 +383138860 1998 MFGR#1229 +296330561 1998 MFGR#123 +437181243 1998 MFGR#1230 +398944492 1998 MFGR#1231 +424062455 1998 MFGR#1232 +406967188 1998 MFGR#1233 +428867240 1998 MFGR#1234 +352277781 1998 MFGR#1235 +361827086 1998 MFGR#1236 +341618569 1998 MFGR#1237 +244739231 1998 MFGR#1238 +414151803 1998 MFGR#1239 +330082371 1998 MFGR#124 +415312453 1998 MFGR#1240 +360289624 1998 MFGR#125 +341657580 1998 MFGR#126 +377507061 1998 MFGR#127 +361416497 1998 MFGR#128 +318769573 1998 MFGR#129 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.2.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.2.out new file mode 100644 index 00000000000000..8677d312f47d44 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.2.out @@ -0,0 +1,59 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q2.2 -- +709524929 1992 MFGR#2221 +783846394 1992 MFGR#2222 +765052002 1992 MFGR#2223 +651488962 1992 MFGR#2224 +646528589 1992 MFGR#2225 +709650548 1992 MFGR#2226 +745556316 1992 MFGR#2227 +756901875 1992 MFGR#2228 +766521103 1993 MFGR#2221 +691475597 1993 MFGR#2222 +758220752 1993 MFGR#2223 +669662707 1993 MFGR#2224 +773854228 1993 MFGR#2225 +737087518 1993 MFGR#2226 +781967766 1993 MFGR#2227 +680880216 1993 MFGR#2228 +685777518 1994 MFGR#2221 +666524807 1994 MFGR#2222 +733993590 1994 MFGR#2223 +707869040 1994 MFGR#2224 +721251967 1994 MFGR#2225 +822495919 1994 MFGR#2226 +720837128 1994 MFGR#2227 +826225350 1994 MFGR#2228 +775437074 1995 MFGR#2221 +761354792 1995 MFGR#2222 +637832575 1995 MFGR#2223 +589765707 1995 MFGR#2224 +708290039 1995 MFGR#2225 +747356383 1995 MFGR#2226 +802502540 1995 MFGR#2227 +895936786 1995 MFGR#2228 +700010008 1996 MFGR#2221 +777310085 1996 MFGR#2222 +656095314 1996 MFGR#2223 +656859917 1996 MFGR#2224 +765820896 1996 MFGR#2225 +808177734 1996 MFGR#2226 +729563303 1996 MFGR#2227 +819665874 1996 MFGR#2228 +727342382 1997 MFGR#2221 +664533779 1997 MFGR#2222 +748288392 1997 MFGR#2223 +630422081 1997 MFGR#2224 +757391203 1997 MFGR#2225 +747889257 1997 MFGR#2226 +728857899 1997 MFGR#2227 +775312985 1997 MFGR#2228 +335304504 1998 MFGR#2221 +409347137 1998 MFGR#2222 +459109577 1998 MFGR#2223 +413318072 1998 MFGR#2224 +410402095 1998 MFGR#2225 +453515044 1998 MFGR#2226 +390506405 1998 MFGR#2227 +397939103 1998 MFGR#2228 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.3.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.3.out new file mode 100644 index 00000000000000..cb87e345040a0a --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q2.3.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q2.3 -- +726427486 1992 MFGR#2239 +705238959 1993 MFGR#2239 +664145134 1994 MFGR#2239 +713839336 1995 MFGR#2239 +728492865 1996 MFGR#2239 +733896532 1997 MFGR#2239 +380255731 1998 MFGR#2239 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.1.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.1.out new file mode 100644 index 00000000000000..c618f9495d97c2 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.1.out @@ -0,0 +1,153 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q3.1 -- +INDONESIA INDONESIA 1992 6905698988 +CHINA INDONESIA 1992 6616078869 +CHINA CHINA 1992 6456431477 +VIETNAM INDONESIA 1992 6340181707 +VIETNAM CHINA 1992 6332844390 +INDONESIA CHINA 1992 6310709533 +INDIA INDONESIA 1992 6238807354 +JAPAN INDONESIA 1992 6222565638 +CHINA INDIA 1992 6189831351 +CHINA JAPAN 1992 6157678893 +JAPAN CHINA 1992 6067100655 +INDIA CHINA 1992 6066461811 +VIETNAM JAPAN 1992 5863786349 +JAPAN INDIA 1992 5802749985 +INDONESIA INDIA 1992 5770203218 +VIETNAM INDIA 1992 5698511401 +INDONESIA JAPAN 1992 5663765610 +JAPAN JAPAN 1992 5648034711 +INDIA JAPAN 1992 5421959281 +CHINA VIETNAM 1992 5412322026 +JAPAN VIETNAM 1992 5401692555 +INDIA INDIA 1992 5361243755 +INDONESIA VIETNAM 1992 5302638372 +INDIA VIETNAM 1992 5245330621 +VIETNAM VIETNAM 1992 5111731778 +INDONESIA INDONESIA 1993 6931363262 +INDONESIA CHINA 1993 6612891249 +CHINA INDONESIA 1993 6581513366 +VIETNAM INDONESIA 1993 6511639352 +CHINA CHINA 1993 6444513787 +CHINA INDIA 1993 6421694121 +VIETNAM CHINA 1993 6413579999 +INDIA INDONESIA 1993 6331058594 +JAPAN CHINA 1993 6292293995 +INDIA CHINA 1993 6209353792 +CHINA JAPAN 1993 6195466884 +VIETNAM INDIA 1993 6161174477 +INDONESIA INDIA 1993 6151664306 +INDONESIA JAPAN 1993 6147605249 +JAPAN INDONESIA 1993 6053946313 +INDIA JAPAN 1993 5995208985 +CHINA VIETNAM 1993 5885023228 +VIETNAM JAPAN 1993 5874266867 +INDONESIA VIETNAM 1993 5840287722 +JAPAN INDIA 1993 5823343157 +INDIA INDIA 1993 5571575799 +VIETNAM VIETNAM 1993 5554161183 +JAPAN JAPAN 1993 5430318583 +JAPAN VIETNAM 1993 5406569653 +INDIA VIETNAM 1993 5233871487 +VIETNAM CHINA 1994 6709883442 +CHINA CHINA 1994 6648942965 +INDONESIA CHINA 1994 6483600910 +CHINA JAPAN 1994 6349037021 +VIETNAM INDONESIA 1994 6347441931 +JAPAN CHINA 1994 6320009218 +INDONESIA INDONESIA 1994 6315331086 +CHINA INDIA 1994 6297582811 +CHINA INDONESIA 1994 6234787896 +VIETNAM JAPAN 1994 6231973047 +INDONESIA INDIA 1994 6198412245 +INDIA INDONESIA 1994 6168189859 +INDONESIA JAPAN 1994 6141195969 +JAPAN INDONESIA 1994 6013034618 +CHINA VIETNAM 1994 5885318733 +INDIA CHINA 1994 5815022714 +VIETNAM INDIA 1994 5789398691 +JAPAN JAPAN 1994 5753821482 +JAPAN INDIA 1994 5645818872 +INDONESIA VIETNAM 1994 5624346368 +INDIA INDIA 1994 5584936515 +VIETNAM VIETNAM 1994 5418498159 +INDIA JAPAN 1994 5394134974 +JAPAN VIETNAM 1994 5275821637 +INDIA VIETNAM 1994 5251039815 +CHINA INDONESIA 1995 7074539444 +INDONESIA CHINA 1995 6928620620 +CHINA CHINA 1995 6624666612 +JAPAN CHINA 1995 6519889385 +VIETNAM CHINA 1995 6332731268 +INDONESIA INDONESIA 1995 6268531321 +VIETNAM JAPAN 1995 6263957020 +CHINA INDIA 1995 6246656374 +VIETNAM INDIA 1995 6198441957 +INDONESIA INDIA 1995 6173805183 +VIETNAM INDONESIA 1995 6057820148 +CHINA JAPAN 1995 6021854130 +INDONESIA JAPAN 1995 6019093871 +INDIA CHINA 1995 6010532793 +INDIA INDONESIA 1995 5975518597 +JAPAN JAPAN 1995 5952279129 +JAPAN INDONESIA 1995 5947000735 +VIETNAM VIETNAM 1995 5754727607 +JAPAN INDIA 1995 5730743126 +INDONESIA VIETNAM 1995 5574974066 +INDIA INDIA 1995 5565995550 +JAPAN VIETNAM 1995 5501313775 +CHINA VIETNAM 1995 5489936063 +INDIA JAPAN 1995 5469203427 +INDIA VIETNAM 1995 5207063284 +INDONESIA INDONESIA 1996 6750056283 +CHINA INDONESIA 1996 6657125075 +INDONESIA CHINA 1996 6613439112 +CHINA CHINA 1996 6591697915 +VIETNAM CHINA 1996 6428653890 +VIETNAM INDONESIA 1996 6295626732 +JAPAN INDONESIA 1996 6227447856 +INDIA CHINA 1996 6198567819 +INDIA INDONESIA 1996 6189242058 +CHINA INDIA 1996 6153787365 +INDONESIA INDIA 1996 6138810863 +CHINA JAPAN 1996 6105566324 +JAPAN CHINA 1996 6088985564 +INDONESIA JAPAN 1996 6055638222 +VIETNAM JAPAN 1996 5919997150 +VIETNAM VIETNAM 1996 5860842302 +INDIA JAPAN 1996 5841664805 +VIETNAM INDIA 1996 5807486983 +JAPAN INDIA 1996 5644579616 +JAPAN JAPAN 1996 5566204795 +INDIA INDIA 1996 5532073103 +INDONESIA VIETNAM 1996 5438514387 +CHINA VIETNAM 1996 5434772818 +JAPAN VIETNAM 1996 5334277549 +INDIA VIETNAM 1996 5293891531 +CHINA INDONESIA 1997 6653234696 +INDONESIA CHINA 1997 6577396314 +CHINA CHINA 1997 6463794795 +JAPAN INDONESIA 1997 6429270126 +INDONESIA INDONESIA 1997 6398427821 +VIETNAM INDONESIA 1997 6363583120 +CHINA JAPAN 1997 6284733518 +VIETNAM CHINA 1997 6164218723 +INDIA CHINA 1997 6153282214 +CHINA INDIA 1997 6084283983 +INDONESIA JAPAN 1997 6001427956 +INDIA INDONESIA 1997 5983439300 +JAPAN CHINA 1997 5973849687 +CHINA VIETNAM 1997 5908254176 +JAPAN INDIA 1997 5796921992 +JAPAN JAPAN 1997 5790450039 +INDONESIA INDIA 1997 5789367105 +VIETNAM INDIA 1997 5730121608 +INDIA INDIA 1997 5693028907 +VIETNAM JAPAN 1997 5689345230 +INDONESIA VIETNAM 1997 5665678132 +VIETNAM VIETNAM 1997 5620251324 +INDIA JAPAN 1997 5587827913 +JAPAN VIETNAM 1997 5249874614 +INDIA VIETNAM 1997 5003624837 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.2.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.2.out new file mode 100644 index 00000000000000..6fa0c9ba6d26f7 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.2.out @@ -0,0 +1,602 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q3.2 -- +UNITED ST4 UNITED ST1 1992 102027455 +UNITED ST1 UNITED ST0 1992 96989491 +UNITED ST7 UNITED ST0 1992 96078010 +UNITED ST9 UNITED ST0 1992 94813294 +UNITED ST4 UNITED ST0 1992 94644242 +UNITED ST2 UNITED ST4 1992 91180500 +UNITED ST5 UNITED ST0 1992 90432300 +UNITED ST6 UNITED ST7 1992 87658267 +UNITED ST3 UNITED ST9 1992 86142048 +UNITED ST6 UNITED ST5 1992 85882966 +UNITED ST7 UNITED ST3 1992 83765666 +UNITED ST2 UNITED ST9 1992 83705618 +UNITED ST4 UNITED ST6 1992 81886374 +UNITED ST2 UNITED ST1 1992 81839165 +UNITED ST9 UNITED ST1 1992 80795302 +UNITED ST6 UNITED ST3 1992 78778218 +UNITED ST6 UNITED ST0 1992 78696956 +UNITED ST0 UNITED ST1 1992 77267396 +UNITED ST0 UNITED ST0 1992 75622122 +UNITED ST1 UNITED ST9 1992 75367059 +UNITED ST3 UNITED ST1 1992 73637490 +UNITED ST2 UNITED ST0 1992 72210218 +UNITED ST1 UNITED ST7 1992 71472973 +UNITED ST6 UNITED ST4 1992 71086944 +UNITED ST4 UNITED ST4 1992 70111335 +UNITED ST6 UNITED ST1 1992 69408688 +UNITED ST4 UNITED ST3 1992 69001787 +UNITED ST5 UNITED ST7 1992 68333651 +UNITED ST4 UNITED ST9 1992 67837970 +UNITED ST7 UNITED ST6 1992 65513205 +UNITED ST4 UNITED ST5 1992 65057872 +UNITED ST7 UNITED ST4 1992 64900888 +UNITED ST1 UNITED ST1 1992 64669070 +UNITED ST3 UNITED ST5 1992 64239048 +UNITED ST0 UNITED ST9 1992 63979996 +UNITED ST3 UNITED ST4 1992 63144772 +UNITED ST5 UNITED ST6 1992 62628093 +UNITED ST4 UNITED ST7 1992 62529376 +UNITED ST3 UNITED ST0 1992 62441656 +UNITED ST9 UNITED ST4 1992 61489513 +UNITED ST8 UNITED ST6 1992 60540440 +UNITED ST7 UNITED ST9 1992 60326042 +UNITED ST7 UNITED ST7 1992 60121386 +UNITED ST5 UNITED ST1 1992 59945287 +UNITED ST5 UNITED ST4 1992 57625627 +UNITED ST7 UNITED ST5 1992 57566802 +UNITED ST2 UNITED ST5 1992 57021365 +UNITED ST9 UNITED ST7 1992 56883359 +UNITED ST0 UNITED ST3 1992 56359317 +UNITED ST1 UNITED ST3 1992 55727474 +UNITED ST5 UNITED ST3 1992 53963553 +UNITED ST0 UNITED ST7 1992 50583409 +UNITED ST5 UNITED ST9 1992 50191091 +UNITED ST7 UNITED ST1 1992 50167208 +UNITED ST0 UNITED ST8 1992 49732640 +UNITED ST0 UNITED ST4 1992 49676807 +UNITED ST9 UNITED ST3 1992 47681165 +UNITED ST8 UNITED ST4 1992 46757019 +UNITED ST3 UNITED ST3 1992 45087216 +UNITED ST8 UNITED ST0 1992 44368839 +UNITED ST0 UNITED ST6 1992 42471806 +UNITED ST6 UNITED ST8 1992 42463690 +UNITED ST8 UNITED ST7 1992 41897901 +UNITED ST3 UNITED ST8 1992 41275764 +UNITED ST6 UNITED ST9 1992 40591721 +UNITED ST0 UNITED ST5 1992 40120886 +UNITED ST1 UNITED ST4 1992 39326346 +UNITED ST3 UNITED ST7 1992 39028579 +UNITED ST3 UNITED ST6 1992 38798715 +UNITED ST9 UNITED ST9 1992 36048343 +UNITED ST2 UNITED ST8 1992 36046449 +UNITED ST2 UNITED ST3 1992 35981963 +UNITED ST8 UNITED ST1 1992 35680752 +UNITED ST1 UNITED ST6 1992 35404990 +UNITED ST8 UNITED ST5 1992 35187610 +UNITED ST1 UNITED ST5 1992 33971251 +UNITED ST5 UNITED ST8 1992 33878053 +UNITED ST2 UNITED ST7 1992 33702779 +UNITED ST8 UNITED ST3 1992 30949324 +UNITED ST8 UNITED ST8 1992 29309108 +UNITED ST5 UNITED ST5 1992 29279568 +UNITED ST1 UNITED ST8 1992 28565579 +UNITED ST9 UNITED ST5 1992 28075004 +UNITED ST2 UNITED ST6 1992 27813739 +UNITED ST0 UNITED ST2 1992 27718733 +UNITED ST2 UNITED ST2 1992 25743654 +UNITED ST8 UNITED ST9 1992 22684471 +UNITED ST4 UNITED ST8 1992 21928442 +UNITED ST9 UNITED ST8 1992 21386100 +UNITED ST5 UNITED ST2 1992 20495817 +UNITED ST6 UNITED ST6 1992 18137105 +UNITED ST9 UNITED ST6 1992 15879568 +UNITED ST4 UNITED ST2 1992 12061845 +UNITED ST7 UNITED ST8 1992 11895702 +UNITED ST6 UNITED ST2 1992 11820698 +UNITED ST9 UNITED ST2 1992 11623177 +UNITED ST8 UNITED ST2 1992 10971561 +UNITED ST1 UNITED ST2 1992 7706728 +UNITED ST7 UNITED ST2 1992 4943204 +UNITED ST3 UNITED ST2 1992 1097208 +UNITED ST0 UNITED ST9 1993 109834040 +UNITED ST7 UNITED ST0 1993 109788024 +UNITED ST5 UNITED ST0 1993 106822597 +UNITED ST0 UNITED ST0 1993 106742548 +UNITED ST1 UNITED ST0 1993 99305952 +UNITED ST4 UNITED ST4 1993 98150465 +UNITED ST3 UNITED ST4 1993 92493920 +UNITED ST0 UNITED ST1 1993 91196593 +UNITED ST4 UNITED ST1 1993 88521423 +UNITED ST8 UNITED ST0 1993 88356371 +UNITED ST4 UNITED ST7 1993 88172198 +UNITED ST4 UNITED ST0 1993 86918458 +UNITED ST6 UNITED ST3 1993 83417161 +UNITED ST6 UNITED ST1 1993 83345939 +UNITED ST7 UNITED ST9 1993 80310701 +UNITED ST3 UNITED ST1 1993 78230278 +UNITED ST6 UNITED ST7 1993 78197294 +UNITED ST5 UNITED ST9 1993 76286539 +UNITED ST0 UNITED ST3 1993 76171283 +UNITED ST5 UNITED ST8 1993 74359279 +UNITED ST9 UNITED ST1 1993 74059419 +UNITED ST4 UNITED ST9 1993 73296959 +UNITED ST5 UNITED ST1 1993 71454623 +UNITED ST6 UNITED ST4 1993 69646913 +UNITED ST2 UNITED ST1 1993 69631701 +UNITED ST6 UNITED ST0 1993 68247539 +UNITED ST7 UNITED ST7 1993 68109820 +UNITED ST2 UNITED ST3 1993 66972438 +UNITED ST3 UNITED ST0 1993 66626926 +UNITED ST9 UNITED ST7 1993 66625483 +UNITED ST1 UNITED ST8 1993 66146198 +UNITED ST2 UNITED ST7 1993 64185014 +UNITED ST5 UNITED ST4 1993 63415639 +UNITED ST9 UNITED ST9 1993 63260763 +UNITED ST1 UNITED ST4 1993 62884347 +UNITED ST7 UNITED ST4 1993 61656613 +UNITED ST3 UNITED ST6 1993 58584808 +UNITED ST2 UNITED ST4 1993 56650391 +UNITED ST3 UNITED ST5 1993 55907305 +UNITED ST6 UNITED ST9 1993 54900942 +UNITED ST1 UNITED ST7 1993 54851183 +UNITED ST3 UNITED ST9 1993 54762596 +UNITED ST8 UNITED ST6 1993 54633062 +UNITED ST8 UNITED ST3 1993 54049874 +UNITED ST5 UNITED ST7 1993 52745538 +UNITED ST0 UNITED ST5 1993 52701052 +UNITED ST1 UNITED ST9 1993 52514902 +UNITED ST8 UNITED ST5 1993 52237837 +UNITED ST1 UNITED ST3 1993 52097946 +UNITED ST8 UNITED ST4 1993 51419356 +UNITED ST0 UNITED ST6 1993 50432282 +UNITED ST5 UNITED ST5 1993 50357189 +UNITED ST3 UNITED ST7 1993 50135448 +UNITED ST0 UNITED ST4 1993 49260067 +UNITED ST0 UNITED ST7 1993 48796360 +UNITED ST2 UNITED ST9 1993 48188507 +UNITED ST1 UNITED ST1 1993 47538610 +UNITED ST9 UNITED ST3 1993 46943647 +UNITED ST7 UNITED ST5 1993 44763692 +UNITED ST1 UNITED ST6 1993 44728540 +UNITED ST8 UNITED ST1 1993 44415434 +UNITED ST7 UNITED ST8 1993 43902628 +UNITED ST9 UNITED ST6 1993 43867160 +UNITED ST2 UNITED ST0 1993 42845485 +UNITED ST3 UNITED ST8 1993 42251848 +UNITED ST0 UNITED ST8 1993 42002682 +UNITED ST4 UNITED ST8 1993 41657582 +UNITED ST1 UNITED ST5 1993 40693513 +UNITED ST9 UNITED ST5 1993 39685269 +UNITED ST7 UNITED ST3 1993 39523861 +UNITED ST8 UNITED ST8 1993 38790235 +UNITED ST8 UNITED ST9 1993 38516361 +UNITED ST2 UNITED ST5 1993 37406845 +UNITED ST9 UNITED ST8 1993 37184696 +UNITED ST8 UNITED ST7 1993 36902218 +UNITED ST6 UNITED ST8 1993 36456741 +UNITED ST7 UNITED ST1 1993 34391159 +UNITED ST6 UNITED ST5 1993 34229082 +UNITED ST5 UNITED ST3 1993 34031811 +UNITED ST2 UNITED ST8 1993 33445446 +UNITED ST4 UNITED ST3 1993 33129412 +UNITED ST6 UNITED ST6 1993 33050663 +UNITED ST9 UNITED ST0 1993 32653305 +UNITED ST4 UNITED ST6 1993 30699255 +UNITED ST9 UNITED ST4 1993 30644687 +UNITED ST4 UNITED ST5 1993 29119594 +UNITED ST7 UNITED ST6 1993 27100502 +UNITED ST4 UNITED ST2 1993 27012678 +UNITED ST2 UNITED ST6 1993 26482226 +UNITED ST5 UNITED ST6 1993 25357679 +UNITED ST3 UNITED ST3 1993 21777144 +UNITED ST3 UNITED ST2 1993 21559073 +UNITED ST5 UNITED ST2 1993 20610242 +UNITED ST7 UNITED ST2 1993 20219304 +UNITED ST6 UNITED ST2 1993 18814367 +UNITED ST9 UNITED ST2 1993 17718390 +UNITED ST1 UNITED ST2 1993 16844538 +UNITED ST0 UNITED ST2 1993 15042145 +UNITED ST2 UNITED ST2 1993 14521995 +UNITED ST8 UNITED ST2 1993 9984366 +UNITED ST8 UNITED ST0 1994 99220789 +UNITED ST3 UNITED ST9 1994 97476185 +UNITED ST6 UNITED ST1 1994 96937147 +UNITED ST6 UNITED ST9 1994 94683309 +UNITED ST9 UNITED ST1 1994 90440948 +UNITED ST0 UNITED ST9 1994 89865202 +UNITED ST5 UNITED ST7 1994 89089961 +UNITED ST9 UNITED ST0 1994 87670573 +UNITED ST3 UNITED ST1 1994 85523653 +UNITED ST4 UNITED ST9 1994 83822393 +UNITED ST0 UNITED ST0 1994 83526877 +UNITED ST7 UNITED ST0 1994 82265536 +UNITED ST2 UNITED ST1 1994 81300089 +UNITED ST5 UNITED ST0 1994 78648057 +UNITED ST4 UNITED ST7 1994 76954140 +UNITED ST4 UNITED ST4 1994 76837381 +UNITED ST0 UNITED ST1 1994 76613379 +UNITED ST1 UNITED ST3 1994 75992459 +UNITED ST7 UNITED ST1 1994 75320799 +UNITED ST4 UNITED ST0 1994 73719340 +UNITED ST5 UNITED ST1 1994 73508418 +UNITED ST4 UNITED ST1 1994 72219557 +UNITED ST2 UNITED ST9 1994 69671054 +UNITED ST6 UNITED ST5 1994 66461534 +UNITED ST2 UNITED ST3 1994 65620760 +UNITED ST3 UNITED ST0 1994 65522727 +UNITED ST5 UNITED ST3 1994 65334911 +UNITED ST7 UNITED ST4 1994 64778715 +UNITED ST3 UNITED ST4 1994 63412365 +UNITED ST8 UNITED ST4 1994 62141681 +UNITED ST0 UNITED ST4 1994 61519744 +UNITED ST0 UNITED ST7 1994 61480820 +UNITED ST0 UNITED ST6 1994 61288778 +UNITED ST2 UNITED ST0 1994 60182153 +UNITED ST6 UNITED ST4 1994 59829989 +UNITED ST4 UNITED ST5 1994 59397028 +UNITED ST8 UNITED ST9 1994 58666906 +UNITED ST4 UNITED ST6 1994 58633482 +UNITED ST5 UNITED ST5 1994 56235213 +UNITED ST6 UNITED ST3 1994 56123238 +UNITED ST2 UNITED ST4 1994 55679377 +UNITED ST8 UNITED ST3 1994 55203841 +UNITED ST1 UNITED ST1 1994 54383174 +UNITED ST1 UNITED ST7 1994 53853106 +UNITED ST6 UNITED ST0 1994 53728853 +UNITED ST5 UNITED ST9 1994 53367331 +UNITED ST9 UNITED ST9 1994 51980849 +UNITED ST5 UNITED ST4 1994 51799593 +UNITED ST7 UNITED ST9 1994 50144085 +UNITED ST7 UNITED ST7 1994 46446442 +UNITED ST6 UNITED ST6 1994 46199722 +UNITED ST7 UNITED ST5 1994 45895364 +UNITED ST3 UNITED ST3 1994 45627153 +UNITED ST8 UNITED ST5 1994 44553056 +UNITED ST9 UNITED ST4 1994 43910761 +UNITED ST1 UNITED ST0 1994 43225201 +UNITED ST1 UNITED ST9 1994 43000037 +UNITED ST7 UNITED ST8 1994 42776467 +UNITED ST0 UNITED ST5 1994 41808301 +UNITED ST2 UNITED ST6 1994 41526105 +UNITED ST1 UNITED ST4 1994 41381558 +UNITED ST3 UNITED ST7 1994 40935131 +UNITED ST8 UNITED ST1 1994 40152096 +UNITED ST9 UNITED ST8 1994 39278808 +UNITED ST5 UNITED ST6 1994 38658451 +UNITED ST2 UNITED ST5 1994 37640317 +UNITED ST8 UNITED ST7 1994 37600687 +UNITED ST9 UNITED ST5 1994 37146726 +UNITED ST6 UNITED ST7 1994 37057808 +UNITED ST8 UNITED ST6 1994 36776569 +UNITED ST3 UNITED ST6 1994 36290257 +UNITED ST9 UNITED ST3 1994 35846500 +UNITED ST2 UNITED ST8 1994 33767774 +UNITED ST0 UNITED ST8 1994 31845433 +UNITED ST4 UNITED ST3 1994 31599433 +UNITED ST9 UNITED ST7 1994 31586173 +UNITED ST1 UNITED ST6 1994 31287326 +UNITED ST1 UNITED ST8 1994 30245153 +UNITED ST7 UNITED ST3 1994 29424840 +UNITED ST9 UNITED ST6 1994 29212927 +UNITED ST0 UNITED ST3 1994 27327829 +UNITED ST6 UNITED ST8 1994 26592996 +UNITED ST3 UNITED ST5 1994 26197875 +UNITED ST6 UNITED ST2 1994 25809000 +UNITED ST1 UNITED ST5 1994 24939138 +UNITED ST7 UNITED ST6 1994 24631937 +UNITED ST1 UNITED ST2 1994 23556586 +UNITED ST4 UNITED ST2 1994 23035892 +UNITED ST2 UNITED ST7 1994 22182758 +UNITED ST0 UNITED ST2 1994 22017954 +UNITED ST4 UNITED ST8 1994 20685352 +UNITED ST7 UNITED ST2 1994 19655081 +UNITED ST5 UNITED ST8 1994 18931891 +UNITED ST2 UNITED ST2 1994 18068657 +UNITED ST3 UNITED ST8 1994 15936051 +UNITED ST8 UNITED ST8 1994 10023412 +UNITED ST3 UNITED ST2 1994 9995234 +UNITED ST9 UNITED ST2 1994 9700989 +UNITED ST5 UNITED ST2 1994 7162796 +UNITED ST8 UNITED ST2 1994 3789626 +UNITED ST5 UNITED ST1 1995 119793669 +UNITED ST4 UNITED ST9 1995 99490068 +UNITED ST7 UNITED ST0 1995 98031295 +UNITED ST6 UNITED ST0 1995 91718471 +UNITED ST4 UNITED ST1 1995 90878653 +UNITED ST0 UNITED ST1 1995 90763599 +UNITED ST8 UNITED ST9 1995 88855089 +UNITED ST7 UNITED ST7 1995 86571624 +UNITED ST3 UNITED ST0 1995 84462733 +UNITED ST9 UNITED ST1 1995 82938967 +UNITED ST2 UNITED ST4 1995 82432305 +UNITED ST1 UNITED ST0 1995 81676623 +UNITED ST5 UNITED ST4 1995 81016761 +UNITED ST7 UNITED ST1 1995 79964362 +UNITED ST5 UNITED ST3 1995 78099130 +UNITED ST5 UNITED ST0 1995 77615746 +UNITED ST9 UNITED ST9 1995 76515958 +UNITED ST7 UNITED ST9 1995 75317709 +UNITED ST4 UNITED ST4 1995 74587071 +UNITED ST9 UNITED ST4 1995 72525686 +UNITED ST1 UNITED ST9 1995 72470870 +UNITED ST4 UNITED ST7 1995 69264407 +UNITED ST6 UNITED ST3 1995 67513062 +UNITED ST2 UNITED ST3 1995 65218129 +UNITED ST2 UNITED ST9 1995 65055178 +UNITED ST7 UNITED ST6 1995 65020671 +UNITED ST3 UNITED ST1 1995 64762815 +UNITED ST1 UNITED ST1 1995 64199332 +UNITED ST6 UNITED ST9 1995 63457105 +UNITED ST0 UNITED ST9 1995 63253499 +UNITED ST5 UNITED ST9 1995 62364897 +UNITED ST4 UNITED ST5 1995 62081505 +UNITED ST1 UNITED ST7 1995 61515741 +UNITED ST2 UNITED ST7 1995 60000208 +UNITED ST8 UNITED ST6 1995 58990404 +UNITED ST1 UNITED ST4 1995 57535599 +UNITED ST0 UNITED ST3 1995 56360708 +UNITED ST8 UNITED ST0 1995 55231664 +UNITED ST5 UNITED ST7 1995 53740759 +UNITED ST2 UNITED ST0 1995 52560838 +UNITED ST3 UNITED ST7 1995 51579548 +UNITED ST9 UNITED ST0 1995 51548621 +UNITED ST6 UNITED ST6 1995 50954677 +UNITED ST5 UNITED ST5 1995 50394007 +UNITED ST7 UNITED ST4 1995 49899545 +UNITED ST3 UNITED ST3 1995 48158089 +UNITED ST6 UNITED ST4 1995 47697241 +UNITED ST9 UNITED ST7 1995 46464589 +UNITED ST4 UNITED ST0 1995 46142899 +UNITED ST1 UNITED ST3 1995 45823056 +UNITED ST2 UNITED ST1 1995 45437340 +UNITED ST6 UNITED ST5 1995 45428152 +UNITED ST8 UNITED ST5 1995 44994863 +UNITED ST7 UNITED ST3 1995 43699734 +UNITED ST9 UNITED ST6 1995 43482494 +UNITED ST2 UNITED ST5 1995 43382417 +UNITED ST6 UNITED ST8 1995 41973920 +UNITED ST0 UNITED ST6 1995 40718942 +UNITED ST3 UNITED ST5 1995 40057815 +UNITED ST7 UNITED ST5 1995 39015293 +UNITED ST0 UNITED ST0 1995 38984502 +UNITED ST6 UNITED ST1 1995 38328352 +UNITED ST4 UNITED ST6 1995 38109524 +UNITED ST3 UNITED ST9 1995 37364623 +UNITED ST4 UNITED ST3 1995 37356396 +UNITED ST2 UNITED ST6 1995 37146288 +UNITED ST9 UNITED ST5 1995 36009924 +UNITED ST1 UNITED ST8 1995 34918793 +UNITED ST8 UNITED ST1 1995 34217780 +UNITED ST0 UNITED ST7 1995 33395313 +UNITED ST1 UNITED ST5 1995 31857452 +UNITED ST8 UNITED ST7 1995 30918202 +UNITED ST2 UNITED ST8 1995 30504189 +UNITED ST3 UNITED ST4 1995 30422346 +UNITED ST5 UNITED ST6 1995 30204737 +UNITED ST8 UNITED ST3 1995 29349938 +UNITED ST0 UNITED ST4 1995 29170038 +UNITED ST1 UNITED ST6 1995 27139403 +UNITED ST7 UNITED ST8 1995 26444490 +UNITED ST6 UNITED ST7 1995 23833977 +UNITED ST4 UNITED ST8 1995 23053236 +UNITED ST4 UNITED ST2 1995 22787003 +UNITED ST3 UNITED ST8 1995 22505239 +UNITED ST9 UNITED ST8 1995 21292527 +UNITED ST8 UNITED ST4 1995 19287311 +UNITED ST8 UNITED ST2 1995 18282990 +UNITED ST9 UNITED ST3 1995 17539102 +UNITED ST3 UNITED ST6 1995 16738530 +UNITED ST0 UNITED ST8 1995 16393249 +UNITED ST5 UNITED ST2 1995 14951023 +UNITED ST2 UNITED ST2 1995 13455031 +UNITED ST5 UNITED ST8 1995 13346932 +UNITED ST3 UNITED ST2 1995 12886829 +UNITED ST9 UNITED ST2 1995 12653362 +UNITED ST0 UNITED ST5 1995 11453709 +UNITED ST6 UNITED ST2 1995 11363551 +UNITED ST8 UNITED ST8 1995 11285867 +UNITED ST1 UNITED ST2 1995 7991676 +UNITED ST0 UNITED ST2 1995 4776460 +UNITED ST7 UNITED ST2 1995 3973565 +UNITED ST6 UNITED ST0 1996 132286763 +UNITED ST4 UNITED ST0 1996 106897563 +UNITED ST5 UNITED ST0 1996 104501979 +UNITED ST0 UNITED ST4 1996 103228749 +UNITED ST9 UNITED ST1 1996 101983827 +UNITED ST1 UNITED ST0 1996 94861554 +UNITED ST0 UNITED ST1 1996 91948777 +UNITED ST6 UNITED ST1 1996 89705870 +UNITED ST2 UNITED ST1 1996 88256155 +UNITED ST1 UNITED ST1 1996 87265848 +UNITED ST4 UNITED ST7 1996 83677814 +UNITED ST6 UNITED ST3 1996 82168229 +UNITED ST2 UNITED ST7 1996 80468477 +UNITED ST8 UNITED ST1 1996 78971756 +UNITED ST7 UNITED ST4 1996 77941011 +UNITED ST1 UNITED ST3 1996 77610905 +UNITED ST9 UNITED ST9 1996 77301740 +UNITED ST0 UNITED ST9 1996 75935209 +UNITED ST7 UNITED ST0 1996 75602445 +UNITED ST3 UNITED ST1 1996 74746699 +UNITED ST7 UNITED ST7 1996 74040644 +UNITED ST4 UNITED ST1 1996 72819867 +UNITED ST5 UNITED ST9 1996 72614114 +UNITED ST1 UNITED ST9 1996 69823769 +UNITED ST9 UNITED ST4 1996 69616614 +UNITED ST6 UNITED ST4 1996 69296005 +UNITED ST2 UNITED ST0 1996 67095122 +UNITED ST5 UNITED ST1 1996 65346389 +UNITED ST6 UNITED ST9 1996 63256182 +UNITED ST4 UNITED ST6 1996 62189328 +UNITED ST0 UNITED ST0 1996 61528855 +UNITED ST8 UNITED ST9 1996 60466691 +UNITED ST3 UNITED ST0 1996 60226840 +UNITED ST8 UNITED ST6 1996 59746655 +UNITED ST2 UNITED ST3 1996 59648598 +UNITED ST0 UNITED ST5 1996 57762895 +UNITED ST8 UNITED ST7 1996 57523925 +UNITED ST2 UNITED ST4 1996 57487057 +UNITED ST6 UNITED ST7 1996 57090619 +UNITED ST3 UNITED ST4 1996 54838259 +UNITED ST4 UNITED ST9 1996 54134840 +UNITED ST1 UNITED ST6 1996 54056366 +UNITED ST3 UNITED ST7 1996 53987218 +UNITED ST2 UNITED ST9 1996 53491415 +UNITED ST4 UNITED ST8 1996 53035662 +UNITED ST9 UNITED ST5 1996 52825922 +UNITED ST7 UNITED ST3 1996 52356886 +UNITED ST6 UNITED ST8 1996 52136784 +UNITED ST1 UNITED ST5 1996 51189649 +UNITED ST8 UNITED ST4 1996 51033054 +UNITED ST1 UNITED ST4 1996 50135547 +UNITED ST3 UNITED ST9 1996 49612304 +UNITED ST9 UNITED ST0 1996 49590701 +UNITED ST3 UNITED ST3 1996 49263796 +UNITED ST9 UNITED ST7 1996 48798759 +UNITED ST7 UNITED ST1 1996 48784175 +UNITED ST9 UNITED ST6 1996 48685063 +UNITED ST2 UNITED ST5 1996 47028976 +UNITED ST9 UNITED ST3 1996 47021018 +UNITED ST2 UNITED ST8 1996 46865113 +UNITED ST4 UNITED ST3 1996 46460940 +UNITED ST6 UNITED ST5 1996 46030104 +UNITED ST2 UNITED ST6 1996 45416649 +UNITED ST8 UNITED ST5 1996 43480473 +UNITED ST5 UNITED ST5 1996 43020722 +UNITED ST6 UNITED ST6 1996 42923032 +UNITED ST4 UNITED ST5 1996 42808412 +UNITED ST3 UNITED ST6 1996 41881628 +UNITED ST1 UNITED ST7 1996 41721506 +UNITED ST5 UNITED ST7 1996 40946330 +UNITED ST8 UNITED ST0 1996 39845427 +UNITED ST8 UNITED ST3 1996 39535940 +UNITED ST1 UNITED ST8 1996 39430882 +UNITED ST5 UNITED ST6 1996 38332044 +UNITED ST0 UNITED ST6 1996 37232062 +UNITED ST7 UNITED ST6 1996 36535628 +UNITED ST9 UNITED ST8 1996 36112301 +UNITED ST3 UNITED ST8 1996 33924732 +UNITED ST3 UNITED ST5 1996 33717439 +UNITED ST5 UNITED ST4 1996 33424859 +UNITED ST5 UNITED ST3 1996 32919926 +UNITED ST4 UNITED ST4 1996 32787995 +UNITED ST7 UNITED ST5 1996 32784224 +UNITED ST5 UNITED ST8 1996 32415682 +UNITED ST0 UNITED ST7 1996 31391181 +UNITED ST0 UNITED ST3 1996 29795665 +UNITED ST7 UNITED ST9 1996 25028091 +UNITED ST7 UNITED ST8 1996 24348851 +UNITED ST6 UNITED ST2 1996 20447847 +UNITED ST8 UNITED ST8 1996 16340603 +UNITED ST0 UNITED ST8 1996 15168262 +UNITED ST4 UNITED ST2 1996 12451867 +UNITED ST1 UNITED ST2 1996 10082536 +UNITED ST5 UNITED ST2 1996 8544233 +UNITED ST7 UNITED ST2 1996 8390470 +UNITED ST9 UNITED ST2 1996 8108035 +UNITED ST8 UNITED ST2 1996 7028334 +UNITED ST0 UNITED ST2 1996 6907199 +UNITED ST3 UNITED ST2 1996 4311800 +UNITED ST5 UNITED ST0 1997 121457766 +UNITED ST0 UNITED ST9 1997 119856268 +UNITED ST5 UNITED ST1 1997 106900161 +UNITED ST9 UNITED ST9 1997 106222795 +UNITED ST5 UNITED ST4 1997 103432927 +UNITED ST7 UNITED ST1 1997 101326940 +UNITED ST5 UNITED ST9 1997 97392640 +UNITED ST8 UNITED ST0 1997 89434845 +UNITED ST1 UNITED ST3 1997 85175638 +UNITED ST4 UNITED ST1 1997 84611188 +UNITED ST4 UNITED ST7 1997 84606996 +UNITED ST1 UNITED ST4 1997 83092569 +UNITED ST0 UNITED ST1 1997 80167139 +UNITED ST4 UNITED ST9 1997 79697927 +UNITED ST1 UNITED ST0 1997 77667866 +UNITED ST2 UNITED ST0 1997 77591470 +UNITED ST1 UNITED ST7 1997 77045722 +UNITED ST2 UNITED ST7 1997 76483802 +UNITED ST1 UNITED ST1 1997 76340444 +UNITED ST0 UNITED ST7 1997 72577490 +UNITED ST4 UNITED ST0 1997 69875804 +UNITED ST6 UNITED ST3 1997 69725506 +UNITED ST2 UNITED ST9 1997 69543984 +UNITED ST7 UNITED ST0 1997 69354312 +UNITED ST9 UNITED ST7 1997 69052630 +UNITED ST8 UNITED ST3 1997 66918394 +UNITED ST0 UNITED ST0 1997 66308516 +UNITED ST9 UNITED ST0 1997 66066791 +UNITED ST2 UNITED ST3 1997 65429453 +UNITED ST2 UNITED ST1 1997 65396135 +UNITED ST4 UNITED ST4 1997 62532346 +UNITED ST9 UNITED ST1 1997 62418406 +UNITED ST3 UNITED ST7 1997 61095300 +UNITED ST7 UNITED ST4 1997 60123494 +UNITED ST4 UNITED ST3 1997 59634153 +UNITED ST3 UNITED ST4 1997 58356141 +UNITED ST6 UNITED ST9 1997 58231263 +UNITED ST6 UNITED ST4 1997 57215022 +UNITED ST2 UNITED ST4 1997 57012611 +UNITED ST5 UNITED ST3 1997 56789932 +UNITED ST9 UNITED ST5 1997 56091920 +UNITED ST6 UNITED ST0 1997 55824919 +UNITED ST6 UNITED ST1 1997 55117709 +UNITED ST7 UNITED ST9 1997 55039970 +UNITED ST5 UNITED ST7 1997 54534315 +UNITED ST3 UNITED ST1 1997 54150683 +UNITED ST3 UNITED ST0 1997 54050172 +UNITED ST3 UNITED ST9 1997 51370308 +UNITED ST1 UNITED ST5 1997 51052110 +UNITED ST6 UNITED ST7 1997 49795849 +UNITED ST5 UNITED ST6 1997 49030016 +UNITED ST1 UNITED ST9 1997 48944111 +UNITED ST3 UNITED ST3 1997 48385233 +UNITED ST0 UNITED ST5 1997 47988418 +UNITED ST2 UNITED ST8 1997 46391909 +UNITED ST4 UNITED ST6 1997 46236849 +UNITED ST9 UNITED ST3 1997 46121724 +UNITED ST8 UNITED ST9 1997 45852796 +UNITED ST7 UNITED ST8 1997 45476266 +UNITED ST8 UNITED ST1 1997 43284139 +UNITED ST7 UNITED ST7 1997 42566603 +UNITED ST0 UNITED ST4 1997 41193803 +UNITED ST8 UNITED ST7 1997 40878429 +UNITED ST8 UNITED ST8 1997 40749400 +UNITED ST2 UNITED ST5 1997 40662886 +UNITED ST0 UNITED ST3 1997 40078508 +UNITED ST6 UNITED ST8 1997 37988445 +UNITED ST9 UNITED ST6 1997 37596882 +UNITED ST6 UNITED ST5 1997 37571788 +UNITED ST2 UNITED ST2 1997 37034333 +UNITED ST7 UNITED ST5 1997 36889736 +UNITED ST8 UNITED ST4 1997 36600584 +UNITED ST3 UNITED ST6 1997 36075844 +UNITED ST7 UNITED ST3 1997 35168922 +UNITED ST2 UNITED ST6 1997 34274467 +UNITED ST5 UNITED ST8 1997 32910946 +UNITED ST3 UNITED ST5 1997 32811963 +UNITED ST4 UNITED ST8 1997 32599736 +UNITED ST5 UNITED ST5 1997 32568888 +UNITED ST4 UNITED ST5 1997 31995868 +UNITED ST9 UNITED ST4 1997 31765478 +UNITED ST7 UNITED ST2 1997 31409590 +UNITED ST9 UNITED ST8 1997 31272385 +UNITED ST0 UNITED ST8 1997 30241370 +UNITED ST3 UNITED ST8 1997 29102220 +UNITED ST7 UNITED ST6 1997 27539931 +UNITED ST8 UNITED ST5 1997 26673743 +UNITED ST6 UNITED ST6 1997 24983291 +UNITED ST0 UNITED ST2 1997 23584229 +UNITED ST0 UNITED ST6 1997 22924046 +UNITED ST1 UNITED ST2 1997 20599130 +UNITED ST8 UNITED ST6 1997 20073000 +UNITED ST1 UNITED ST6 1997 18205326 +UNITED ST1 UNITED ST8 1997 15375258 +UNITED ST6 UNITED ST2 1997 14746680 +UNITED ST5 UNITED ST2 1997 13863438 +UNITED ST8 UNITED ST2 1997 12053706 +UNITED ST3 UNITED ST2 1997 7891878 +UNITED ST4 UNITED ST2 1997 2848157 +UNITED ST9 UNITED ST2 1997 2661652 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.3.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.3.out new file mode 100644 index 00000000000000..96695e04c80746 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.3.out @@ -0,0 +1,27 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q3.3 -- +UNITED KI1 UNITED KI1 1992 46735995 +UNITED KI5 UNITED KI1 1992 36277055 +UNITED KI5 UNITED KI5 1992 25355267 +UNITED KI1 UNITED KI5 1992 21917846 +UNITED KI5 UNITED KI1 1993 61017607 +UNITED KI1 UNITED KI1 1993 45669535 +UNITED KI5 UNITED KI5 1993 34099392 +UNITED KI1 UNITED KI5 1993 21444206 +UNITED KI5 UNITED KI1 1994 36282163 +UNITED KI1 UNITED KI1 1994 34868441 +UNITED KI5 UNITED KI5 1994 34507284 +UNITED KI1 UNITED KI5 1994 21221780 +UNITED KI5 UNITED KI1 1995 82955896 +UNITED KI1 UNITED KI1 1995 35881186 +UNITED KI5 UNITED KI5 1995 20539805 +UNITED KI1 UNITED KI5 1995 17176510 +UNITED KI5 UNITED KI1 1996 65767049 +UNITED KI1 UNITED KI1 1996 59923037 +UNITED KI5 UNITED KI5 1996 46077342 +UNITED KI1 UNITED KI5 1996 13700254 +UNITED KI1 UNITED KI1 1997 70343133 +UNITED KI5 UNITED KI1 1997 64978359 +UNITED KI5 UNITED KI5 1997 27332027 +UNITED KI1 UNITED KI5 1997 16410668 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.4.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.4.out new file mode 100644 index 00000000000000..136aefa3508328 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q3.4.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q3.4 -- +UNITED KI5 UNITED KI1 1997 9117846 +UNITED KI5 UNITED KI5 1997 6203801 +UNITED KI1 UNITED KI5 1997 1870070 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.1.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.1.out new file mode 100644 index 00000000000000..485ce1aa922775 --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.1.out @@ -0,0 +1,38 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q4.1 -- +1992 ARGENTINA 9658964469 +1992 BRAZIL 9226983055 +1992 CANADA 9643176787 +1992 PERU 9410676597 +1992 UNITED STATES 9849427653 +1993 ARGENTINA 9976332853 +1993 BRAZIL 9468799229 +1993 CANADA 9897302420 +1993 PERU 9309445836 +1993 UNITED STATES 10003985086 +1994 ARGENTINA 9940305215 +1994 BRAZIL 9348651677 +1994 CANADA 9582647596 +1994 PERU 9295265013 +1994 UNITED STATES 9519880425 +1995 ARGENTINA 10143841380 +1995 BRAZIL 9156077350 +1995 CANADA 9562612160 +1995 PERU 9778087211 +1995 UNITED STATES 9310565244 +1996 ARGENTINA 10001927895 +1996 BRAZIL 9168485151 +1996 CANADA 10061604203 +1996 PERU 9355135674 +1996 UNITED STATES 9769712174 +1997 ARGENTINA 9854560261 +1997 BRAZIL 9121571047 +1997 CANADA 10097371778 +1997 PERU 9315525917 +1997 UNITED STATES 10506723879 +1998 ARGENTINA 5834240407 +1998 BRAZIL 5356398095 +1998 CANADA 5423211196 +1998 PERU 5726185970 +1998 UNITED STATES 6009462019 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.2.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.2.out new file mode 100644 index 00000000000000..600a018fb40d8f --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.2.out @@ -0,0 +1,103 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q4.2 -- +1997 ARGENTINA MFGR#11 907071566 +1997 ARGENTINA MFGR#12 924115562 +1997 ARGENTINA MFGR#13 972861821 +1997 ARGENTINA MFGR#14 975410345 +1997 ARGENTINA MFGR#15 938867375 +1997 ARGENTINA MFGR#21 1014782574 +1997 ARGENTINA MFGR#22 873016783 +1997 ARGENTINA MFGR#23 1030357302 +1997 ARGENTINA MFGR#24 893460579 +1997 ARGENTINA MFGR#25 1006311403 +1997 BRAZIL MFGR#11 1073219328 +1997 BRAZIL MFGR#12 989858833 +1997 BRAZIL MFGR#13 1128480379 +1997 BRAZIL MFGR#14 1194256722 +1997 BRAZIL MFGR#15 1094419124 +1997 BRAZIL MFGR#21 910026832 +1997 BRAZIL MFGR#22 993142048 +1997 BRAZIL MFGR#23 1107672874 +1997 BRAZIL MFGR#24 1058013649 +1997 BRAZIL MFGR#25 994733764 +1997 CANADA MFGR#11 854725020 +1997 CANADA MFGR#12 938718164 +1997 CANADA MFGR#13 959265890 +1997 CANADA MFGR#14 1002812450 +1997 CANADA MFGR#15 848183013 +1997 CANADA MFGR#21 999805272 +1997 CANADA MFGR#22 778419763 +1997 CANADA MFGR#23 928359645 +1997 CANADA MFGR#24 849895128 +1997 CANADA MFGR#25 904587965 +1997 PERU MFGR#11 1100242877 +1997 PERU MFGR#12 994365350 +1997 PERU MFGR#13 847486105 +1997 PERU MFGR#14 947769683 +1997 PERU MFGR#15 999395678 +1997 PERU MFGR#21 867923394 +1997 PERU MFGR#22 988747459 +1997 PERU MFGR#23 1066645086 +1997 PERU MFGR#24 935665725 +1997 PERU MFGR#25 981454129 +1997 UNITED STATES MFGR#11 1046706048 +1997 UNITED STATES MFGR#12 909213709 +1997 UNITED STATES MFGR#13 1096278906 +1997 UNITED STATES MFGR#14 934282111 +1997 UNITED STATES MFGR#15 962760843 +1997 UNITED STATES MFGR#21 1000676474 +1997 UNITED STATES MFGR#22 1076947615 +1997 UNITED STATES MFGR#23 937288102 +1997 UNITED STATES MFGR#24 1003386363 +1997 UNITED STATES MFGR#25 1053666052 +1998 ARGENTINA MFGR#11 567612227 +1998 ARGENTINA MFGR#12 527025042 +1998 ARGENTINA MFGR#13 582791792 +1998 ARGENTINA MFGR#14 523726368 +1998 ARGENTINA MFGR#15 522078267 +1998 ARGENTINA MFGR#21 504712685 +1998 ARGENTINA MFGR#22 506061736 +1998 ARGENTINA MFGR#23 560479801 +1998 ARGENTINA MFGR#24 524579118 +1998 ARGENTINA MFGR#25 547840211 +1998 BRAZIL MFGR#11 638578488 +1998 BRAZIL MFGR#12 646312681 +1998 BRAZIL MFGR#13 655161772 +1998 BRAZIL MFGR#14 552676170 +1998 BRAZIL MFGR#15 663812709 +1998 BRAZIL MFGR#21 668822448 +1998 BRAZIL MFGR#22 591791918 +1998 BRAZIL MFGR#23 690648877 +1998 BRAZIL MFGR#24 562362220 +1998 BRAZIL MFGR#25 704182376 +1998 CANADA MFGR#11 509086125 +1998 CANADA MFGR#12 488089772 +1998 CANADA MFGR#13 486533297 +1998 CANADA MFGR#14 527837227 +1998 CANADA MFGR#15 535869299 +1998 CANADA MFGR#21 455868651 +1998 CANADA MFGR#22 594277308 +1998 CANADA MFGR#23 574125070 +1998 CANADA MFGR#24 508530424 +1998 CANADA MFGR#25 547757992 +1998 PERU MFGR#11 567838547 +1998 PERU MFGR#12 540544757 +1998 PERU MFGR#13 591331883 +1998 PERU MFGR#14 481335064 +1998 PERU MFGR#15 570246138 +1998 PERU MFGR#21 533733330 +1998 PERU MFGR#22 527790656 +1998 PERU MFGR#23 636393221 +1998 PERU MFGR#24 589075262 +1998 PERU MFGR#25 543251115 +1998 UNITED STATES MFGR#11 556276232 +1998 UNITED STATES MFGR#12 612385982 +1998 UNITED STATES MFGR#13 622413927 +1998 UNITED STATES MFGR#14 555006887 +1998 UNITED STATES MFGR#15 525119569 +1998 UNITED STATES MFGR#21 563406836 +1998 UNITED STATES MFGR#22 580478735 +1998 UNITED STATES MFGR#23 656080465 +1998 UNITED STATES MFGR#24 538445058 +1998 UNITED STATES MFGR#25 589111952 + diff --git a/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.3.out b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.3.out new file mode 100644 index 00000000000000..09c8399d008f6b --- /dev/null +++ b/regression-test/data/s3_vault/ssb_sf1_p2/sql/q4.3.out @@ -0,0 +1,697 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q4.3 -- +1997 UNITED ST0 MFGR#141 15456288 +1997 UNITED ST0 MFGR#1410 32961113 +1997 UNITED ST0 MFGR#1411 36166400 +1997 UNITED ST0 MFGR#1412 18237451 +1997 UNITED ST0 MFGR#1413 40752813 +1997 UNITED ST0 MFGR#1414 24237588 +1997 UNITED ST0 MFGR#1415 6419001 +1997 UNITED ST0 MFGR#1416 15639876 +1997 UNITED ST0 MFGR#1417 15476211 +1997 UNITED ST0 MFGR#1418 8169956 +1997 UNITED ST0 MFGR#1419 18217252 +1997 UNITED ST0 MFGR#142 29361222 +1997 UNITED ST0 MFGR#1420 34587603 +1997 UNITED ST0 MFGR#1421 28560921 +1997 UNITED ST0 MFGR#1422 25154181 +1997 UNITED ST0 MFGR#1423 18144008 +1997 UNITED ST0 MFGR#1424 28064267 +1997 UNITED ST0 MFGR#1425 15255845 +1997 UNITED ST0 MFGR#1426 5610353 +1997 UNITED ST0 MFGR#1427 5750693 +1997 UNITED ST0 MFGR#1428 14081707 +1997 UNITED ST0 MFGR#1429 9024512 +1997 UNITED ST0 MFGR#143 5325193 +1997 UNITED ST0 MFGR#1430 16961739 +1997 UNITED ST0 MFGR#1431 19015253 +1997 UNITED ST0 MFGR#1432 6324917 +1997 UNITED ST0 MFGR#1433 6734461 +1997 UNITED ST0 MFGR#1434 19206393 +1997 UNITED ST0 MFGR#1435 679925 +1997 UNITED ST0 MFGR#1436 13834012 +1997 UNITED ST0 MFGR#1437 23575407 +1997 UNITED ST0 MFGR#1438 24756074 +1997 UNITED ST0 MFGR#1439 19644072 +1997 UNITED ST0 MFGR#144 20285157 +1997 UNITED ST0 MFGR#1440 14068558 +1997 UNITED ST0 MFGR#145 22646138 +1997 UNITED ST0 MFGR#146 16381159 +1997 UNITED ST0 MFGR#147 12267827 +1997 UNITED ST0 MFGR#148 30968614 +1997 UNITED ST0 MFGR#149 8640669 +1997 UNITED ST1 MFGR#141 10152087 +1997 UNITED ST1 MFGR#1410 32583300 +1997 UNITED ST1 MFGR#1411 15690352 +1997 UNITED ST1 MFGR#1412 16671403 +1997 UNITED ST1 MFGR#1413 4942775 +1997 UNITED ST1 MFGR#1414 9391597 +1997 UNITED ST1 MFGR#1415 26937296 +1997 UNITED ST1 MFGR#1416 25782871 +1997 UNITED ST1 MFGR#1417 23349641 +1997 UNITED ST1 MFGR#1418 10934620 +1997 UNITED ST1 MFGR#1419 22673030 +1997 UNITED ST1 MFGR#142 14048276 +1997 UNITED ST1 MFGR#1420 18100795 +1997 UNITED ST1 MFGR#1421 25405678 +1997 UNITED ST1 MFGR#1423 5713800 +1997 UNITED ST1 MFGR#1424 4131923 +1997 UNITED ST1 MFGR#1425 17190425 +1997 UNITED ST1 MFGR#1426 22314044 +1997 UNITED ST1 MFGR#1427 13635057 +1997 UNITED ST1 MFGR#1428 17735811 +1997 UNITED ST1 MFGR#1429 28740911 +1997 UNITED ST1 MFGR#143 27418033 +1997 UNITED ST1 MFGR#1430 18655292 +1997 UNITED ST1 MFGR#1431 11074399 +1997 UNITED ST1 MFGR#1432 7490170 +1997 UNITED ST1 MFGR#1433 28997266 +1997 UNITED ST1 MFGR#1434 4612211 +1997 UNITED ST1 MFGR#1435 16327124 +1997 UNITED ST1 MFGR#1436 17409665 +1997 UNITED ST1 MFGR#1437 33391635 +1997 UNITED ST1 MFGR#1438 37321134 +1997 UNITED ST1 MFGR#1439 10279941 +1997 UNITED ST1 MFGR#144 12304723 +1997 UNITED ST1 MFGR#1440 28921230 +1997 UNITED ST1 MFGR#145 17933684 +1997 UNITED ST1 MFGR#146 8709103 +1997 UNITED ST1 MFGR#147 20145983 +1997 UNITED ST1 MFGR#148 16940400 +1997 UNITED ST1 MFGR#149 10284949 +1997 UNITED ST2 MFGR#1412 6328240 +1997 UNITED ST2 MFGR#1415 5022682 +1997 UNITED ST2 MFGR#1416 4102214 +1997 UNITED ST2 MFGR#1417 5284444 +1997 UNITED ST2 MFGR#1418 2649908 +1997 UNITED ST2 MFGR#1419 4664673 +1997 UNITED ST2 MFGR#142 10784835 +1997 UNITED ST2 MFGR#1423 4095190 +1997 UNITED ST2 MFGR#1424 5376355 +1997 UNITED ST2 MFGR#1425 3949957 +1997 UNITED ST2 MFGR#143 2859829 +1997 UNITED ST2 MFGR#1430 4618266 +1997 UNITED ST2 MFGR#1432 5183060 +1997 UNITED ST2 MFGR#1433 10531371 +1997 UNITED ST2 MFGR#1434 77646 +1997 UNITED ST2 MFGR#1436 6538508 +1997 UNITED ST2 MFGR#1438 12324747 +1997 UNITED ST2 MFGR#144 2002870 +1997 UNITED ST2 MFGR#1440 12247229 +1997 UNITED ST2 MFGR#145 202685 +1997 UNITED ST2 MFGR#148 5998932 +1997 UNITED ST3 MFGR#141 12866083 +1997 UNITED ST3 MFGR#1410 31051703 +1997 UNITED ST3 MFGR#1411 14099543 +1997 UNITED ST3 MFGR#1412 4970551 +1997 UNITED ST3 MFGR#1413 26129762 +1997 UNITED ST3 MFGR#1414 22364787 +1997 UNITED ST3 MFGR#1415 27883428 +1997 UNITED ST3 MFGR#1416 25289630 +1997 UNITED ST3 MFGR#1417 5524810 +1997 UNITED ST3 MFGR#1419 3616608 +1997 UNITED ST3 MFGR#142 10776059 +1997 UNITED ST3 MFGR#1420 12958851 +1997 UNITED ST3 MFGR#1421 5815494 +1997 UNITED ST3 MFGR#1422 17694270 +1997 UNITED ST3 MFGR#1423 10285142 +1997 UNITED ST3 MFGR#1424 21555317 +1997 UNITED ST3 MFGR#1425 5214517 +1997 UNITED ST3 MFGR#1426 16810700 +1997 UNITED ST3 MFGR#1427 10086430 +1997 UNITED ST3 MFGR#1428 7733716 +1997 UNITED ST3 MFGR#1429 11922430 +1997 UNITED ST3 MFGR#1430 20105292 +1997 UNITED ST3 MFGR#1431 14716979 +1997 UNITED ST3 MFGR#1432 15779247 +1997 UNITED ST3 MFGR#1433 15308773 +1997 UNITED ST3 MFGR#1434 8515037 +1997 UNITED ST3 MFGR#1435 25486899 +1997 UNITED ST3 MFGR#1436 12739448 +1997 UNITED ST3 MFGR#1437 11249180 +1997 UNITED ST3 MFGR#1438 4156824 +1997 UNITED ST3 MFGR#1439 18358277 +1997 UNITED ST3 MFGR#144 10097718 +1997 UNITED ST3 MFGR#1440 22920494 +1997 UNITED ST3 MFGR#145 20285859 +1997 UNITED ST3 MFGR#146 18406423 +1997 UNITED ST3 MFGR#147 17295990 +1997 UNITED ST3 MFGR#148 26127145 +1997 UNITED ST3 MFGR#149 19895538 +1997 UNITED ST4 MFGR#141 8135477 +1997 UNITED ST4 MFGR#1410 5503247 +1997 UNITED ST4 MFGR#1411 6096435 +1997 UNITED ST4 MFGR#1412 19976043 +1997 UNITED ST4 MFGR#1413 22581282 +1997 UNITED ST4 MFGR#1414 10101379 +1997 UNITED ST4 MFGR#1415 8877444 +1997 UNITED ST4 MFGR#1416 17943398 +1997 UNITED ST4 MFGR#1417 9224501 +1997 UNITED ST4 MFGR#1418 16255176 +1997 UNITED ST4 MFGR#1419 683069 +1997 UNITED ST4 MFGR#142 23512944 +1997 UNITED ST4 MFGR#1420 27954262 +1997 UNITED ST4 MFGR#1421 54209185 +1997 UNITED ST4 MFGR#1422 1173957 +1997 UNITED ST4 MFGR#1423 20410710 +1997 UNITED ST4 MFGR#1424 2889270 +1997 UNITED ST4 MFGR#1425 13361796 +1997 UNITED ST4 MFGR#1426 18678819 +1997 UNITED ST4 MFGR#1427 18206066 +1997 UNITED ST4 MFGR#1428 18153974 +1997 UNITED ST4 MFGR#1429 12545076 +1997 UNITED ST4 MFGR#143 6158834 +1997 UNITED ST4 MFGR#1430 16128370 +1997 UNITED ST4 MFGR#1431 11153717 +1997 UNITED ST4 MFGR#1432 7745636 +1997 UNITED ST4 MFGR#1433 15983634 +1997 UNITED ST4 MFGR#1434 3662246 +1997 UNITED ST4 MFGR#1435 3060180 +1997 UNITED ST4 MFGR#1436 11524592 +1997 UNITED ST4 MFGR#1437 36824174 +1997 UNITED ST4 MFGR#1438 8226666 +1997 UNITED ST4 MFGR#1439 9872470 +1997 UNITED ST4 MFGR#1440 401207 +1997 UNITED ST4 MFGR#145 11698312 +1997 UNITED ST4 MFGR#146 18259956 +1997 UNITED ST4 MFGR#147 10256110 +1997 UNITED ST4 MFGR#148 15892628 +1997 UNITED ST4 MFGR#149 4322491 +1997 UNITED ST5 MFGR#141 8534275 +1997 UNITED ST5 MFGR#1411 2427261 +1997 UNITED ST5 MFGR#1412 9032422 +1997 UNITED ST5 MFGR#1413 6343588 +1997 UNITED ST5 MFGR#1414 2961025 +1997 UNITED ST5 MFGR#1415 21186568 +1997 UNITED ST5 MFGR#1416 9354080 +1997 UNITED ST5 MFGR#1417 11786246 +1997 UNITED ST5 MFGR#1418 15297245 +1997 UNITED ST5 MFGR#1420 15787608 +1997 UNITED ST5 MFGR#1421 17858301 +1997 UNITED ST5 MFGR#1422 14080987 +1997 UNITED ST5 MFGR#1423 12535186 +1997 UNITED ST5 MFGR#1424 3221960 +1997 UNITED ST5 MFGR#1425 8950317 +1997 UNITED ST5 MFGR#1426 13161486 +1997 UNITED ST5 MFGR#1427 6149737 +1997 UNITED ST5 MFGR#1428 6796520 +1997 UNITED ST5 MFGR#1429 26026300 +1997 UNITED ST5 MFGR#143 6712572 +1997 UNITED ST5 MFGR#1430 16092131 +1997 UNITED ST5 MFGR#1431 3425291 +1997 UNITED ST5 MFGR#1432 14630175 +1997 UNITED ST5 MFGR#1433 4967585 +1997 UNITED ST5 MFGR#1434 6716261 +1997 UNITED ST5 MFGR#1435 6290790 +1997 UNITED ST5 MFGR#1436 6787699 +1997 UNITED ST5 MFGR#1437 6045168 +1997 UNITED ST5 MFGR#1438 1082877 +1997 UNITED ST5 MFGR#1439 5268611 +1997 UNITED ST5 MFGR#144 8900840 +1997 UNITED ST5 MFGR#1440 2644336 +1997 UNITED ST5 MFGR#145 18054844 +1997 UNITED ST5 MFGR#146 4893668 +1997 UNITED ST5 MFGR#147 9615891 +1997 UNITED ST5 MFGR#148 1426124 +1997 UNITED ST5 MFGR#149 7320726 +1997 UNITED ST6 MFGR#141 10051337 +1997 UNITED ST6 MFGR#1410 23352963 +1997 UNITED ST6 MFGR#1411 1537303 +1997 UNITED ST6 MFGR#1412 4332646 +1997 UNITED ST6 MFGR#1413 14686752 +1997 UNITED ST6 MFGR#1414 7226409 +1997 UNITED ST6 MFGR#1415 15062686 +1997 UNITED ST6 MFGR#1416 8606480 +1997 UNITED ST6 MFGR#1417 7662761 +1997 UNITED ST6 MFGR#1418 1810860 +1997 UNITED ST6 MFGR#1419 20179568 +1997 UNITED ST6 MFGR#142 15506072 +1997 UNITED ST6 MFGR#1420 3844271 +1997 UNITED ST6 MFGR#1421 10608709 +1997 UNITED ST6 MFGR#1422 16083699 +1997 UNITED ST6 MFGR#1423 6789571 +1997 UNITED ST6 MFGR#1424 17631950 +1997 UNITED ST6 MFGR#1425 8477221 +1997 UNITED ST6 MFGR#1426 16847029 +1997 UNITED ST6 MFGR#1428 17314796 +1997 UNITED ST6 MFGR#1429 19175871 +1997 UNITED ST6 MFGR#143 8775848 +1997 UNITED ST6 MFGR#1431 5319063 +1997 UNITED ST6 MFGR#1432 7875606 +1997 UNITED ST6 MFGR#1433 9305264 +1997 UNITED ST6 MFGR#1434 8050404 +1997 UNITED ST6 MFGR#1435 17585836 +1997 UNITED ST6 MFGR#1436 13745758 +1997 UNITED ST6 MFGR#1438 14536324 +1997 UNITED ST6 MFGR#1439 11451329 +1997 UNITED ST6 MFGR#144 10794837 +1997 UNITED ST6 MFGR#1440 5719238 +1997 UNITED ST6 MFGR#145 25449058 +1997 UNITED ST6 MFGR#146 17821996 +1997 UNITED ST6 MFGR#147 18922333 +1997 UNITED ST6 MFGR#148 10825718 +1997 UNITED ST6 MFGR#149 14950868 +1997 UNITED ST7 MFGR#141 8291476 +1997 UNITED ST7 MFGR#1410 4622953 +1997 UNITED ST7 MFGR#1411 14885507 +1997 UNITED ST7 MFGR#1412 19011375 +1997 UNITED ST7 MFGR#1413 31178370 +1997 UNITED ST7 MFGR#1414 16429613 +1997 UNITED ST7 MFGR#1415 3091832 +1997 UNITED ST7 MFGR#1416 17008243 +1997 UNITED ST7 MFGR#1417 15609283 +1997 UNITED ST7 MFGR#1418 5059394 +1997 UNITED ST7 MFGR#1419 7234721 +1997 UNITED ST7 MFGR#142 6898938 +1997 UNITED ST7 MFGR#1420 6426590 +1997 UNITED ST7 MFGR#1421 6043221 +1997 UNITED ST7 MFGR#1422 13273021 +1997 UNITED ST7 MFGR#1423 30537249 +1997 UNITED ST7 MFGR#1424 9224296 +1997 UNITED ST7 MFGR#1425 12799069 +1997 UNITED ST7 MFGR#1426 30277891 +1997 UNITED ST7 MFGR#1427 8331493 +1997 UNITED ST7 MFGR#1428 16205423 +1997 UNITED ST7 MFGR#1429 14396627 +1997 UNITED ST7 MFGR#143 10209623 +1997 UNITED ST7 MFGR#1430 16556758 +1997 UNITED ST7 MFGR#1431 15775327 +1997 UNITED ST7 MFGR#1432 24602989 +1997 UNITED ST7 MFGR#1433 22927717 +1997 UNITED ST7 MFGR#1434 9486462 +1997 UNITED ST7 MFGR#1435 14254290 +1997 UNITED ST7 MFGR#1436 8645365 +1997 UNITED ST7 MFGR#1437 19026166 +1997 UNITED ST7 MFGR#1438 11291550 +1997 UNITED ST7 MFGR#1439 9590292 +1997 UNITED ST7 MFGR#144 14110404 +1997 UNITED ST7 MFGR#1440 21877355 +1997 UNITED ST7 MFGR#145 8246717 +1997 UNITED ST7 MFGR#146 9785781 +1997 UNITED ST7 MFGR#147 13412691 +1997 UNITED ST7 MFGR#148 14235939 +1997 UNITED ST7 MFGR#149 23450309 +1997 UNITED ST8 MFGR#141 7323416 +1997 UNITED ST8 MFGR#1410 7191961 +1997 UNITED ST8 MFGR#1411 2425849 +1997 UNITED ST8 MFGR#1412 26725641 +1997 UNITED ST8 MFGR#1413 7915896 +1997 UNITED ST8 MFGR#1414 14359044 +1997 UNITED ST8 MFGR#1415 15834927 +1997 UNITED ST8 MFGR#1416 7464470 +1997 UNITED ST8 MFGR#1417 15274496 +1997 UNITED ST8 MFGR#1418 3261391 +1997 UNITED ST8 MFGR#1419 8020386 +1997 UNITED ST8 MFGR#142 5985550 +1997 UNITED ST8 MFGR#1420 9441461 +1997 UNITED ST8 MFGR#1421 2998735 +1997 UNITED ST8 MFGR#1423 1565415 +1997 UNITED ST8 MFGR#1424 5613094 +1997 UNITED ST8 MFGR#1425 7186734 +1997 UNITED ST8 MFGR#1426 3153117 +1997 UNITED ST8 MFGR#1427 11303043 +1997 UNITED ST8 MFGR#1428 3564003 +1997 UNITED ST8 MFGR#143 6357477 +1997 UNITED ST8 MFGR#1430 6633950 +1997 UNITED ST8 MFGR#1431 867285 +1997 UNITED ST8 MFGR#1432 9962884 +1997 UNITED ST8 MFGR#1433 13708011 +1997 UNITED ST8 MFGR#1434 7357067 +1997 UNITED ST8 MFGR#1435 11993453 +1997 UNITED ST8 MFGR#1436 7231938 +1997 UNITED ST8 MFGR#1437 18632056 +1997 UNITED ST8 MFGR#1438 3609384 +1997 UNITED ST8 MFGR#144 4822734 +1997 UNITED ST8 MFGR#1440 2521917 +1997 UNITED ST8 MFGR#145 11683385 +1997 UNITED ST8 MFGR#146 3350415 +1997 UNITED ST8 MFGR#147 8425393 +1997 UNITED ST8 MFGR#148 15005341 +1997 UNITED ST8 MFGR#149 9018101 +1997 UNITED ST9 MFGR#141 21838850 +1997 UNITED ST9 MFGR#1410 8455387 +1997 UNITED ST9 MFGR#1411 27984689 +1997 UNITED ST9 MFGR#1412 26491401 +1997 UNITED ST9 MFGR#1413 30237705 +1997 UNITED ST9 MFGR#1414 8957476 +1997 UNITED ST9 MFGR#1415 35391583 +1997 UNITED ST9 MFGR#1416 8923782 +1997 UNITED ST9 MFGR#1417 3210984 +1997 UNITED ST9 MFGR#1418 12102607 +1997 UNITED ST9 MFGR#1419 14468627 +1997 UNITED ST9 MFGR#142 33363582 +1997 UNITED ST9 MFGR#1420 13139731 +1997 UNITED ST9 MFGR#1421 8410650 +1997 UNITED ST9 MFGR#1422 11268410 +1997 UNITED ST9 MFGR#1423 2846614 +1997 UNITED ST9 MFGR#1424 2305872 +1997 UNITED ST9 MFGR#1425 4896060 +1997 UNITED ST9 MFGR#1426 31544772 +1997 UNITED ST9 MFGR#1427 4113085 +1997 UNITED ST9 MFGR#1428 11137081 +1997 UNITED ST9 MFGR#1429 10633230 +1997 UNITED ST9 MFGR#143 11438203 +1997 UNITED ST9 MFGR#1430 2727389 +1997 UNITED ST9 MFGR#1431 10548538 +1997 UNITED ST9 MFGR#1433 26564495 +1997 UNITED ST9 MFGR#1434 11948206 +1997 UNITED ST9 MFGR#1435 2201909 +1997 UNITED ST9 MFGR#1436 10347631 +1997 UNITED ST9 MFGR#1437 40051690 +1997 UNITED ST9 MFGR#1438 2662945 +1997 UNITED ST9 MFGR#1439 18011455 +1997 UNITED ST9 MFGR#144 10202784 +1997 UNITED ST9 MFGR#1440 11123196 +1997 UNITED ST9 MFGR#145 11326624 +1997 UNITED ST9 MFGR#146 27825456 +1997 UNITED ST9 MFGR#147 17579502 +1997 UNITED ST9 MFGR#148 17001116 +1998 UNITED ST0 MFGR#141 12484786 +1998 UNITED ST0 MFGR#1410 5225999 +1998 UNITED ST0 MFGR#1411 2633395 +1998 UNITED ST0 MFGR#1412 9479536 +1998 UNITED ST0 MFGR#1413 18468711 +1998 UNITED ST0 MFGR#1414 19440601 +1998 UNITED ST0 MFGR#1415 16894643 +1998 UNITED ST0 MFGR#1416 7520117 +1998 UNITED ST0 MFGR#1417 464576 +1998 UNITED ST0 MFGR#1418 5171747 +1998 UNITED ST0 MFGR#1419 5970985 +1998 UNITED ST0 MFGR#142 23830409 +1998 UNITED ST0 MFGR#1420 2568835 +1998 UNITED ST0 MFGR#1421 6096442 +1998 UNITED ST0 MFGR#1422 10099456 +1998 UNITED ST0 MFGR#1423 24617746 +1998 UNITED ST0 MFGR#1424 22853888 +1998 UNITED ST0 MFGR#1425 25622112 +1998 UNITED ST0 MFGR#1426 9630701 +1998 UNITED ST0 MFGR#1427 4952625 +1998 UNITED ST0 MFGR#1428 4675471 +1998 UNITED ST0 MFGR#1429 19713527 +1998 UNITED ST0 MFGR#143 4971265 +1998 UNITED ST0 MFGR#1430 13948705 +1998 UNITED ST0 MFGR#1431 2955181 +1998 UNITED ST0 MFGR#1432 17946759 +1998 UNITED ST0 MFGR#1433 23325027 +1998 UNITED ST0 MFGR#1434 8580738 +1998 UNITED ST0 MFGR#1435 6663914 +1998 UNITED ST0 MFGR#1436 13468582 +1998 UNITED ST0 MFGR#1437 9419979 +1998 UNITED ST0 MFGR#1438 16164522 +1998 UNITED ST0 MFGR#1439 8692866 +1998 UNITED ST0 MFGR#144 17338711 +1998 UNITED ST0 MFGR#1440 8513736 +1998 UNITED ST0 MFGR#145 6212569 +1998 UNITED ST0 MFGR#146 2702349 +1998 UNITED ST0 MFGR#149 8844540 +1998 UNITED ST1 MFGR#141 13449110 +1998 UNITED ST1 MFGR#1410 5224626 +1998 UNITED ST1 MFGR#1411 16046399 +1998 UNITED ST1 MFGR#1412 8424392 +1998 UNITED ST1 MFGR#1413 7634982 +1998 UNITED ST1 MFGR#1414 8711782 +1998 UNITED ST1 MFGR#1415 5991201 +1998 UNITED ST1 MFGR#1416 11059645 +1998 UNITED ST1 MFGR#1417 8108087 +1998 UNITED ST1 MFGR#1418 1710875 +1998 UNITED ST1 MFGR#1419 452298 +1998 UNITED ST1 MFGR#142 6186526 +1998 UNITED ST1 MFGR#1421 4766963 +1998 UNITED ST1 MFGR#1422 5556563 +1998 UNITED ST1 MFGR#1423 12110257 +1998 UNITED ST1 MFGR#1424 8924948 +1998 UNITED ST1 MFGR#1425 5684710 +1998 UNITED ST1 MFGR#1426 6844444 +1998 UNITED ST1 MFGR#1427 7625525 +1998 UNITED ST1 MFGR#1428 5227395 +1998 UNITED ST1 MFGR#1429 2855272 +1998 UNITED ST1 MFGR#143 6964811 +1998 UNITED ST1 MFGR#1430 16952380 +1998 UNITED ST1 MFGR#1431 7655883 +1998 UNITED ST1 MFGR#1432 2975960 +1998 UNITED ST1 MFGR#1433 10175594 +1998 UNITED ST1 MFGR#1434 26358848 +1998 UNITED ST1 MFGR#1435 11408623 +1998 UNITED ST1 MFGR#1436 4438242 +1998 UNITED ST1 MFGR#1437 10014906 +1998 UNITED ST1 MFGR#1438 18817171 +1998 UNITED ST1 MFGR#1439 2129973 +1998 UNITED ST1 MFGR#144 9512727 +1998 UNITED ST1 MFGR#1440 20185179 +1998 UNITED ST1 MFGR#145 823605 +1998 UNITED ST1 MFGR#146 9358026 +1998 UNITED ST1 MFGR#147 6552312 +1998 UNITED ST1 MFGR#148 9144461 +1998 UNITED ST1 MFGR#149 2149639 +1998 UNITED ST2 MFGR#1410 8044368 +1998 UNITED ST2 MFGR#1413 727343 +1998 UNITED ST2 MFGR#1414 251088 +1998 UNITED ST2 MFGR#1416 5224696 +1998 UNITED ST2 MFGR#1419 7917142 +1998 UNITED ST2 MFGR#1420 9848986 +1998 UNITED ST2 MFGR#1427 4816976 +1998 UNITED ST2 MFGR#1429 4196069 +1998 UNITED ST2 MFGR#143 10073044 +1998 UNITED ST2 MFGR#1431 877123 +1998 UNITED ST2 MFGR#1433 3740836 +1998 UNITED ST2 MFGR#1434 857691 +1998 UNITED ST2 MFGR#1435 2277870 +1998 UNITED ST2 MFGR#145 4906521 +1998 UNITED ST2 MFGR#149 842348 +1998 UNITED ST3 MFGR#141 4181582 +1998 UNITED ST3 MFGR#1410 3169761 +1998 UNITED ST3 MFGR#1412 6590440 +1998 UNITED ST3 MFGR#1413 2015369 +1998 UNITED ST3 MFGR#1414 4408466 +1998 UNITED ST3 MFGR#1415 3963826 +1998 UNITED ST3 MFGR#1416 3179540 +1998 UNITED ST3 MFGR#1417 11440999 +1998 UNITED ST3 MFGR#1418 8725751 +1998 UNITED ST3 MFGR#1419 10646990 +1998 UNITED ST3 MFGR#142 5056172 +1998 UNITED ST3 MFGR#1420 3583150 +1998 UNITED ST3 MFGR#1421 7085689 +1998 UNITED ST3 MFGR#1422 20858904 +1998 UNITED ST3 MFGR#1423 12825743 +1998 UNITED ST3 MFGR#1424 3533518 +1998 UNITED ST3 MFGR#1425 4938324 +1998 UNITED ST3 MFGR#1426 15164664 +1998 UNITED ST3 MFGR#1427 2619162 +1998 UNITED ST3 MFGR#1428 7723699 +1998 UNITED ST3 MFGR#143 24362893 +1998 UNITED ST3 MFGR#1430 7139650 +1998 UNITED ST3 MFGR#1431 10306394 +1998 UNITED ST3 MFGR#1432 14769004 +1998 UNITED ST3 MFGR#1433 9610545 +1998 UNITED ST3 MFGR#1435 6611760 +1998 UNITED ST3 MFGR#1436 941348 +1998 UNITED ST3 MFGR#1437 4802162 +1998 UNITED ST3 MFGR#1438 5079750 +1998 UNITED ST3 MFGR#1439 5905118 +1998 UNITED ST3 MFGR#144 5436839 +1998 UNITED ST3 MFGR#1440 9815517 +1998 UNITED ST3 MFGR#145 7580271 +1998 UNITED ST3 MFGR#146 14051404 +1998 UNITED ST3 MFGR#147 7786017 +1998 UNITED ST3 MFGR#149 1271813 +1998 UNITED ST4 MFGR#141 26703183 +1998 UNITED ST4 MFGR#1410 18094129 +1998 UNITED ST4 MFGR#1411 615999 +1998 UNITED ST4 MFGR#1412 13988709 +1998 UNITED ST4 MFGR#1413 10115647 +1998 UNITED ST4 MFGR#1414 1586551 +1998 UNITED ST4 MFGR#1415 14357244 +1998 UNITED ST4 MFGR#1416 9430036 +1998 UNITED ST4 MFGR#1417 17361773 +1998 UNITED ST4 MFGR#1418 10187338 +1998 UNITED ST4 MFGR#1419 7116094 +1998 UNITED ST4 MFGR#142 18454502 +1998 UNITED ST4 MFGR#1420 1621340 +1998 UNITED ST4 MFGR#1422 13861291 +1998 UNITED ST4 MFGR#1423 4208006 +1998 UNITED ST4 MFGR#1424 3242576 +1998 UNITED ST4 MFGR#1425 1470642 +1998 UNITED ST4 MFGR#1426 11987205 +1998 UNITED ST4 MFGR#1427 12975399 +1998 UNITED ST4 MFGR#1428 11756155 +1998 UNITED ST4 MFGR#1429 6956191 +1998 UNITED ST4 MFGR#143 11499476 +1998 UNITED ST4 MFGR#1430 4355096 +1998 UNITED ST4 MFGR#1431 20166612 +1998 UNITED ST4 MFGR#1432 8849789 +1998 UNITED ST4 MFGR#1433 10786037 +1998 UNITED ST4 MFGR#1435 13208151 +1998 UNITED ST4 MFGR#1436 4747808 +1998 UNITED ST4 MFGR#1437 12999351 +1998 UNITED ST4 MFGR#1438 17360787 +1998 UNITED ST4 MFGR#1439 2557841 +1998 UNITED ST4 MFGR#144 22030911 +1998 UNITED ST4 MFGR#145 5272280 +1998 UNITED ST4 MFGR#146 4307958 +1998 UNITED ST4 MFGR#147 7995050 +1998 UNITED ST4 MFGR#148 2348889 +1998 UNITED ST5 MFGR#141 8396190 +1998 UNITED ST5 MFGR#1410 17422889 +1998 UNITED ST5 MFGR#1411 764246 +1998 UNITED ST5 MFGR#1413 8051561 +1998 UNITED ST5 MFGR#1414 1500867 +1998 UNITED ST5 MFGR#1416 3874315 +1998 UNITED ST5 MFGR#1417 7550747 +1998 UNITED ST5 MFGR#1418 14241755 +1998 UNITED ST5 MFGR#1419 4893846 +1998 UNITED ST5 MFGR#142 5418299 +1998 UNITED ST5 MFGR#1420 3810727 +1998 UNITED ST5 MFGR#1421 3321336 +1998 UNITED ST5 MFGR#1423 5021100 +1998 UNITED ST5 MFGR#1424 6125645 +1998 UNITED ST5 MFGR#1426 880680 +1998 UNITED ST5 MFGR#1427 1284890 +1998 UNITED ST5 MFGR#1428 3813108 +1998 UNITED ST5 MFGR#1429 9429868 +1998 UNITED ST5 MFGR#143 15699981 +1998 UNITED ST5 MFGR#1430 7977654 +1998 UNITED ST5 MFGR#1431 5545969 +1998 UNITED ST5 MFGR#1432 1803434 +1998 UNITED ST5 MFGR#1435 12957056 +1998 UNITED ST5 MFGR#1436 20666755 +1998 UNITED ST5 MFGR#1437 9536733 +1998 UNITED ST5 MFGR#1439 11784214 +1998 UNITED ST5 MFGR#1440 613089 +1998 UNITED ST5 MFGR#145 8237944 +1998 UNITED ST5 MFGR#147 5370351 +1998 UNITED ST6 MFGR#141 6683977 +1998 UNITED ST6 MFGR#1411 11589040 +1998 UNITED ST6 MFGR#1412 4800954 +1998 UNITED ST6 MFGR#1416 5848432 +1998 UNITED ST6 MFGR#1417 1804324 +1998 UNITED ST6 MFGR#1418 4642727 +1998 UNITED ST6 MFGR#1419 10079621 +1998 UNITED ST6 MFGR#142 2533570 +1998 UNITED ST6 MFGR#1420 9016258 +1998 UNITED ST6 MFGR#1421 7636150 +1998 UNITED ST6 MFGR#1422 1272436 +1998 UNITED ST6 MFGR#1423 9600097 +1998 UNITED ST6 MFGR#1424 7036762 +1998 UNITED ST6 MFGR#1425 10867426 +1998 UNITED ST6 MFGR#1426 7033994 +1998 UNITED ST6 MFGR#1427 6481679 +1998 UNITED ST6 MFGR#1428 6269185 +1998 UNITED ST6 MFGR#1429 12055631 +1998 UNITED ST6 MFGR#143 6500459 +1998 UNITED ST6 MFGR#1430 6816243 +1998 UNITED ST6 MFGR#1432 7752973 +1998 UNITED ST6 MFGR#1436 6292813 +1998 UNITED ST6 MFGR#1437 2334894 +1998 UNITED ST6 MFGR#1438 4459455 +1998 UNITED ST6 MFGR#144 640220 +1998 UNITED ST6 MFGR#1440 748200 +1998 UNITED ST6 MFGR#145 7948794 +1998 UNITED ST6 MFGR#146 2734333 +1998 UNITED ST6 MFGR#147 2995632 +1998 UNITED ST6 MFGR#148 10695287 +1998 UNITED ST6 MFGR#149 1475864 +1998 UNITED ST7 MFGR#141 4816637 +1998 UNITED ST7 MFGR#1410 24393912 +1998 UNITED ST7 MFGR#1411 1283038 +1998 UNITED ST7 MFGR#1412 12607459 +1998 UNITED ST7 MFGR#1413 15421293 +1998 UNITED ST7 MFGR#1414 7681668 +1998 UNITED ST7 MFGR#1415 7138377 +1998 UNITED ST7 MFGR#1416 9246990 +1998 UNITED ST7 MFGR#1417 5242972 +1998 UNITED ST7 MFGR#1419 13548941 +1998 UNITED ST7 MFGR#142 17758762 +1998 UNITED ST7 MFGR#1420 11980437 +1998 UNITED ST7 MFGR#1421 11166952 +1998 UNITED ST7 MFGR#1422 9231883 +1998 UNITED ST7 MFGR#1424 5796389 +1998 UNITED ST7 MFGR#1425 4943582 +1998 UNITED ST7 MFGR#1426 16361100 +1998 UNITED ST7 MFGR#1427 9659296 +1998 UNITED ST7 MFGR#1428 2998020 +1998 UNITED ST7 MFGR#1429 5766768 +1998 UNITED ST7 MFGR#143 5526167 +1998 UNITED ST7 MFGR#1430 4191892 +1998 UNITED ST7 MFGR#1431 12979559 +1998 UNITED ST7 MFGR#1432 5832068 +1998 UNITED ST7 MFGR#1433 11410227 +1998 UNITED ST7 MFGR#1434 10525792 +1998 UNITED ST7 MFGR#1435 5982420 +1998 UNITED ST7 MFGR#1436 7142934 +1998 UNITED ST7 MFGR#1437 7499671 +1998 UNITED ST7 MFGR#1438 9177564 +1998 UNITED ST7 MFGR#1439 1974444 +1998 UNITED ST7 MFGR#144 2428463 +1998 UNITED ST7 MFGR#1440 3300838 +1998 UNITED ST7 MFGR#146 1771645 +1998 UNITED ST7 MFGR#147 4681835 +1998 UNITED ST7 MFGR#148 7124367 +1998 UNITED ST7 MFGR#149 11662970 +1998 UNITED ST8 MFGR#141 12753354 +1998 UNITED ST8 MFGR#1410 1097292 +1998 UNITED ST8 MFGR#1411 4069881 +1998 UNITED ST8 MFGR#1414 9474831 +1998 UNITED ST8 MFGR#1415 6406304 +1998 UNITED ST8 MFGR#1418 2392133 +1998 UNITED ST8 MFGR#1419 2953230 +1998 UNITED ST8 MFGR#142 10702563 +1998 UNITED ST8 MFGR#1420 3719432 +1998 UNITED ST8 MFGR#1421 3854657 +1998 UNITED ST8 MFGR#1422 9711968 +1998 UNITED ST8 MFGR#1424 1102516 +1998 UNITED ST8 MFGR#1426 8937532 +1998 UNITED ST8 MFGR#1427 5255637 +1998 UNITED ST8 MFGR#1428 1695112 +1998 UNITED ST8 MFGR#143 6231349 +1998 UNITED ST8 MFGR#1430 4678664 +1998 UNITED ST8 MFGR#1431 2459741 +1998 UNITED ST8 MFGR#1433 1724339 +1998 UNITED ST8 MFGR#1434 8222328 +1998 UNITED ST8 MFGR#1435 4087165 +1998 UNITED ST8 MFGR#1437 7379288 +1998 UNITED ST8 MFGR#1439 1947163 +1998 UNITED ST8 MFGR#144 21846266 +1998 UNITED ST8 MFGR#146 13768113 +1998 UNITED ST8 MFGR#147 5279643 +1998 UNITED ST8 MFGR#149 15221410 +1998 UNITED ST9 MFGR#141 5544000 +1998 UNITED ST9 MFGR#1410 7971104 +1998 UNITED ST9 MFGR#1411 26426609 +1998 UNITED ST9 MFGR#1412 1701777 +1998 UNITED ST9 MFGR#1413 18786672 +1998 UNITED ST9 MFGR#1414 15787780 +1998 UNITED ST9 MFGR#1415 11214818 +1998 UNITED ST9 MFGR#1416 2076120 +1998 UNITED ST9 MFGR#1417 14310334 +1998 UNITED ST9 MFGR#1418 3558355 +1998 UNITED ST9 MFGR#1419 6134867 +1998 UNITED ST9 MFGR#142 257897 +1998 UNITED ST9 MFGR#1420 7716338 +1998 UNITED ST9 MFGR#1421 10291775 +1998 UNITED ST9 MFGR#1422 6105716 +1998 UNITED ST9 MFGR#1423 11378613 +1998 UNITED ST9 MFGR#1424 1904550 +1998 UNITED ST9 MFGR#1425 18831147 +1998 UNITED ST9 MFGR#1426 18562642 +1998 UNITED ST9 MFGR#1427 12565890 +1998 UNITED ST9 MFGR#1428 10720914 +1998 UNITED ST9 MFGR#1429 21940718 +1998 UNITED ST9 MFGR#143 21480722 +1998 UNITED ST9 MFGR#1430 30935173 +1998 UNITED ST9 MFGR#1431 7793012 +1998 UNITED ST9 MFGR#1432 10026104 +1998 UNITED ST9 MFGR#1433 17874435 +1998 UNITED ST9 MFGR#1434 5493785 +1998 UNITED ST9 MFGR#1435 10738769 +1998 UNITED ST9 MFGR#1436 43504 +1998 UNITED ST9 MFGR#1437 13559646 +1998 UNITED ST9 MFGR#1438 12654258 +1998 UNITED ST9 MFGR#1439 5972923 +1998 UNITED ST9 MFGR#144 15702123 +1998 UNITED ST9 MFGR#1440 6681029 +1998 UNITED ST9 MFGR#145 920718 +1998 UNITED ST9 MFGR#146 10435931 +1998 UNITED ST9 MFGR#147 4759326 +1998 UNITED ST9 MFGR#148 9970848 +1998 UNITED ST9 MFGR#149 7510940 + diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy index f780cf30615e37..8cbb02421162a2 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy @@ -664,6 +664,11 @@ class Suite implements GroovyInterceptable { return (enableBrokerLoad != null && enableBrokerLoad.equals("true")); } + String getS3Provider() { + String s3Provider = context.config.otherConfigs.get("s3Provider"); + return s3Provider + } + String getS3Region() { String s3Region = context.config.otherConfigs.get("s3Region"); return s3Region diff --git a/regression-test/suites/hdfs_vault/default_vault_p2/load.groovy b/regression-test/suites/hdfs_vault/default_vault_p2/load.groovy index 2c627d07153615..bdef1c9f82cf10 100644 --- a/regression-test/suites/hdfs_vault/default_vault_p2/load.groovy +++ b/regression-test/suites/hdfs_vault/default_vault_p2/load.groovy @@ -45,8 +45,8 @@ suite("load") { SET default_vault_ssb_flat_hdfs_vault AS DEFAULT STORAGE VAULT """ - // sleep 2 seconds to wait for BE sync the vault info from MS - sleep(2000) + // sleep 3 minutes to wait for BE sync the vault info from MS + sleep(180000) // ssb_sf1_p1 is writted to test unique key table merge correctly. // It creates unique key table and sets bucket num to 1 in order to make sure that diff --git a/regression-test/suites/hdfs_vault/multi_vault_p2/load.groovy b/regression-test/suites/hdfs_vault/multi_vault_p2/load.groovy index e7f9ad8e4a3ba2..c0ff09cf06fb1a 100644 --- a/regression-test/suites/hdfs_vault/multi_vault_p2/load.groovy +++ b/regression-test/suites/hdfs_vault/multi_vault_p2/load.groovy @@ -41,8 +41,8 @@ suite("load") { ); """ - // sleep 2 seconds to wait for BE sync the vault info from MS - sleep(2000) + // sleep 3 minutes to wait for BE sync the vault info from MS + sleep(180000) // ssb_sf1_p1 is writted to test unique key table merge correctly. // It creates unique key table and sets bucket num to 1 in order to make sure that diff --git a/regression-test/suites/hdfs_vault/ssb_sf1_p2/load.groovy b/regression-test/suites/hdfs_vault/ssb_sf1_p2/load.groovy index f7ac5a59a1f965..eb4c59baa63eb2 100644 --- a/regression-test/suites/hdfs_vault/ssb_sf1_p2/load.groovy +++ b/regression-test/suites/hdfs_vault/ssb_sf1_p2/load.groovy @@ -32,8 +32,8 @@ suite("load") { ); """ - // sleep 2 seconds to wait for BE sync the vault info from MS - sleep(2000) + // sleep 3 minutes to wait for BE sync the vault info from MS + sleep(180000) // ssb_sf1_p1 is writted to test unique key table merge correctly. // It creates unique key table and sets bucket num to 1 in order to make sure that diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/customer_create.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/customer_create.sql new file mode 100644 index 00000000000000..d13101b70e01bf --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/customer_create.sql @@ -0,0 +1,15 @@ +CREATE TABLE IF NOT EXISTS `customer` ( + `c_custkey` int(11) NOT NULL COMMENT "", + `c_name` varchar(26) NOT NULL COMMENT "", + `c_address` varchar(41) NOT NULL COMMENT "", + `c_city` varchar(11) NOT NULL COMMENT "", + `c_nation` varchar(16) NOT NULL COMMENT "", + `c_region` varchar(13) NOT NULL COMMENT "", + `c_phone` varchar(16) NOT NULL COMMENT "", + `c_mktsegment` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`c_custkey`) +DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/customer_delete.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/customer_delete.sql new file mode 100644 index 00000000000000..fe22a226fedf85 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/customer_delete.sql @@ -0,0 +1 @@ +truncate table customer; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/date_create.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/date_create.sql new file mode 100644 index 00000000000000..cfab87b1d8bc08 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/date_create.sql @@ -0,0 +1,24 @@ +CREATE TABLE IF NOT EXISTS `date` ( + `d_datekey` int(11) NOT NULL COMMENT "", + `d_date` varchar(20) NOT NULL COMMENT "", + `d_dayofweek` varchar(10) NOT NULL COMMENT "", + `d_month` varchar(11) NOT NULL COMMENT "", + `d_year` int(11) NOT NULL COMMENT "", + `d_yearmonthnum` int(11) NOT NULL COMMENT "", + `d_yearmonth` varchar(9) NOT NULL COMMENT "", + `d_daynuminweek` int(11) NOT NULL COMMENT "", + `d_daynuminmonth` int(11) NOT NULL COMMENT "", + `d_daynuminyear` int(11) NOT NULL COMMENT "", + `d_monthnuminyear` int(11) NOT NULL COMMENT "", + `d_weeknuminyear` int(11) NOT NULL COMMENT "", + `d_sellingseason` varchar(14) NOT NULL COMMENT "", + `d_lastdayinweekfl` int(11) NOT NULL COMMENT "", + `d_lastdayinmonthfl` int(11) NOT NULL COMMENT "", + `d_holidayfl` int(11) NOT NULL COMMENT "", + `d_weekdayfl` int(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`d_datekey`) +DISTRIBUTED BY HASH(`d_datekey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/date_delete.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/date_delete.sql new file mode 100644 index 00000000000000..12933cbbad92da --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/date_delete.sql @@ -0,0 +1 @@ +truncate table `date`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_create.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_create.sql new file mode 100644 index 00000000000000..c066ea8a0d70c8 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_create.sql @@ -0,0 +1,24 @@ +CREATE TABLE IF NOT EXISTS `lineorder` ( + `lo_orderkey` bigint(20) NOT NULL COMMENT "", + `lo_linenumber` bigint(20) NOT NULL COMMENT "", + `lo_custkey` int(11) NOT NULL COMMENT "", + `lo_partkey` int(11) NOT NULL COMMENT "", + `lo_suppkey` int(11) NOT NULL COMMENT "", + `lo_orderdate` int(11) NOT NULL COMMENT "", + `lo_orderpriority` varchar(16) NOT NULL COMMENT "", + `lo_shippriority` int(11) NOT NULL COMMENT "", + `lo_quantity` bigint(20) NOT NULL COMMENT "", + `lo_extendedprice` bigint(20) NOT NULL COMMENT "", + `lo_ordtotalprice` bigint(20) NOT NULL COMMENT "", + `lo_discount` bigint(20) NOT NULL COMMENT "", + `lo_revenue` bigint(20) NOT NULL COMMENT "", + `lo_supplycost` bigint(20) NOT NULL COMMENT "", + `lo_tax` bigint(20) NOT NULL COMMENT "", + `lo_commitdate` bigint(20) NOT NULL COMMENT "", + `lo_shipmode` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`lo_orderkey`, `lo_linenumber`) +DISTRIBUTED BY HASH(`lo_orderkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_delete.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_delete.sql new file mode 100644 index 00000000000000..329e040060edc6 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_delete.sql @@ -0,0 +1 @@ +truncate table lineorder; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_flat_create.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_flat_create.sql new file mode 100644 index 00000000000000..35faee0786cb89 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_flat_create.sql @@ -0,0 +1,45 @@ +CREATE TABLE IF NOT EXISTS `lineorder_flat` ( + `LO_ORDERDATE` date NOT NULL COMMENT "", + `LO_ORDERKEY` int(11) NOT NULL COMMENT "", + `LO_LINENUMBER` tinyint(4) NOT NULL COMMENT "", + `LO_CUSTKEY` int(11) NOT NULL COMMENT "", + `LO_PARTKEY` int(11) NOT NULL COMMENT "", + `LO_SUPPKEY` int(11) NOT NULL COMMENT "", + `LO_ORDERPRIORITY` varchar(100) NOT NULL COMMENT "", + `LO_SHIPPRIORITY` tinyint(4) NOT NULL COMMENT "", + `LO_QUANTITY` tinyint(4) NOT NULL COMMENT "", + `LO_EXTENDEDPRICE` int(11) NOT NULL COMMENT "", + `LO_ORDTOTALPRICE` int(11) NOT NULL COMMENT "", + `LO_DISCOUNT` tinyint(4) NOT NULL COMMENT "", + `LO_REVENUE` int(11) NOT NULL COMMENT "", + `LO_SUPPLYCOST` int(11) NOT NULL COMMENT "", + `LO_TAX` tinyint(4) NOT NULL COMMENT "", + `LO_COMMITDATE` date NOT NULL COMMENT "", + `LO_SHIPMODE` varchar(100) NOT NULL COMMENT "", + `C_NAME` varchar(100) NOT NULL COMMENT "", + `C_ADDRESS` varchar(100) NOT NULL COMMENT "", + `C_CITY` varchar(100) NOT NULL COMMENT "", + `C_NATION` varchar(100) NOT NULL COMMENT "", + `C_REGION` varchar(100) NOT NULL COMMENT "", + `C_PHONE` varchar(100) NOT NULL COMMENT "", + `C_MKTSEGMENT` varchar(100) NOT NULL COMMENT "", + `S_NAME` varchar(100) NOT NULL COMMENT "", + `S_ADDRESS` varchar(100) NOT NULL COMMENT "", + `S_CITY` varchar(100) NOT NULL COMMENT "", + `S_NATION` varchar(100) NOT NULL COMMENT "", + `S_REGION` varchar(100) NOT NULL COMMENT "", + `S_PHONE` varchar(100) NOT NULL COMMENT "", + `P_NAME` varchar(100) NOT NULL COMMENT "", + `P_MFGR` varchar(100) NOT NULL COMMENT "", + `P_CATEGORY` varchar(100) NOT NULL COMMENT "", + `P_BRAND` varchar(100) NOT NULL COMMENT "", + `P_COLOR` varchar(100) NOT NULL COMMENT "", + `P_TYPE` varchar(100) NOT NULL COMMENT "", + `P_SIZE` tinyint(4) NOT NULL COMMENT "", + `P_CONTAINER` varchar(100) NOT NULL COMMENT "" +) ENGINE=OLAP +UNIQUE KEY(`LO_ORDERDATE`, `LO_ORDERKEY`, `LO_LINENUMBER`) +DISTRIBUTED BY HASH(`LO_ORDERKEY`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_flat_delete.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_flat_delete.sql new file mode 100644 index 00000000000000..13f6c32bb65a60 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/lineorder_flat_delete.sql @@ -0,0 +1 @@ +truncate table lineorder_flat; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/part_create.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/part_create.sql new file mode 100644 index 00000000000000..2405026a3cd152 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/part_create.sql @@ -0,0 +1,16 @@ +CREATE TABLE IF NOT EXISTS `part` ( + `p_partkey` int(11) NOT NULL COMMENT "", + `p_name` varchar(23) NOT NULL COMMENT "", + `p_mfgr` varchar(7) NOT NULL COMMENT "", + `p_category` varchar(8) NOT NULL COMMENT "", + `p_brand` varchar(10) NOT NULL COMMENT "", + `p_color` varchar(12) NOT NULL COMMENT "", + `p_type` varchar(26) NOT NULL COMMENT "", + `p_size` int(11) NOT NULL COMMENT "", + `p_container` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`p_partkey`) +DISTRIBUTED BY HASH(`p_partkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/part_delete.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/part_delete.sql new file mode 100644 index 00000000000000..02c6abd2539add --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/part_delete.sql @@ -0,0 +1 @@ +truncate table `part`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/supplier_create.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/supplier_create.sql new file mode 100644 index 00000000000000..1580a8fcca4084 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/supplier_create.sql @@ -0,0 +1,14 @@ +CREATE TABLE IF NOT EXISTS `supplier` ( + `s_suppkey` int(11) NOT NULL COMMENT "", + `s_name` varchar(26) NOT NULL COMMENT "", + `s_address` varchar(26) NOT NULL COMMENT "", + `s_city` varchar(11) NOT NULL COMMENT "", + `s_nation` varchar(16) NOT NULL COMMENT "", + `s_region` varchar(13) NOT NULL COMMENT "", + `s_phone` varchar(16) NOT NULL COMMENT "" +) +UNIQUE KEY (`s_suppkey`) +DISTRIBUTED BY HASH(`s_suppkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/ddl/supplier_delete.sql b/regression-test/suites/s3_vault/default_vault_p2/ddl/supplier_delete.sql new file mode 100644 index 00000000000000..39e663134cabd0 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/ddl/supplier_delete.sql @@ -0,0 +1 @@ +truncate table `supplier`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/load.groovy b/regression-test/suites/s3_vault/default_vault_p2/load.groovy new file mode 100644 index 00000000000000..62edda98e65430 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/load.groovy @@ -0,0 +1,147 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq +suite("load") { + + sql """ + CREATE STORAGE VAULT IF NOT EXISTS default_vault_ssb_flat_hdfs_vault_s3 + PROPERTIES ( + "type"="S3", + "s3.endpoint"="${getS3Endpoint()}", + "s3.region" = "${getS3Region()}", + "s3.access_key" = "${getS3AK()}", + "s3.secret_key" = "${getS3SK()}", + "s3.root.path" = "ssb_sf1_p2_s3", + "s3.bucket" = "${getS3BucketName()}", + "s3.external_endpoint" = "", + "provider" = "${getS3Provider()}" + ); + """ + + sql """ + SET default_vault_ssb_flat_hdfs_vault_s3 AS DEFAULT STORAGE VAULT + """ + + // sleep 3 minutes to wait for BE sync the vault info from MS + sleep(180000) + + // ssb_sf1_p1 is writted to test unique key table merge correctly. + // It creates unique key table and sets bucket num to 1 in order to make sure that + // many rowsets will be created during loading and then the merge process will be triggered. + + def tables = ["customer", "lineorder", "part", "date", "supplier"] + def columns = ["""c_custkey,c_name,c_address,c_city,c_nation,c_region,c_phone,c_mktsegment,no_use""", + """lo_orderkey,lo_linenumber,lo_custkey,lo_partkey,lo_suppkey,lo_orderdate,lo_orderpriority, + lo_shippriority,lo_quantity,lo_extendedprice,lo_ordtotalprice,lo_discount, + lo_revenue,lo_supplycost,lo_tax,lo_commitdate,lo_shipmode,lo_dummy""", + """p_partkey,p_name,p_mfgr,p_category,p_brand,p_color,p_type,p_size,p_container,p_dummy""", + """d_datekey,d_date,d_dayofweek,d_month,d_year,d_yearmonthnum,d_yearmonth, + d_daynuminweek,d_daynuminmonth,d_daynuminyear,d_monthnuminyear,d_weeknuminyear, + d_sellingseason,d_lastdayinweekfl,d_lastdayinmonthfl,d_holidayfl,d_weekdayfl,d_dummy""", + """s_suppkey,s_name,s_address,s_city,s_nation,s_region,s_phone,s_dummy"""] + + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_create.sql""").text + sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text + } + def i = 0 + for (String tableName in tables) { + streamLoad { + // a default db 'regression_test' is specified in + // ${DORIS_HOME}/conf/regression-conf.groovy + table tableName + + // default label is UUID: + // set 'label' UUID.randomUUID().toString() + + // default column_separator is specify in doris fe config, usually is '\t'. + // this line change to ',' + set 'column_separator', '|' + set 'compress_type', 'GZ' + set 'columns', columns[i] + + + // relate to ${DORIS_HOME}/regression-test/data/demo/streamload_input.csv. + // also, you can stream load a http stream, e.g. http://xxx/some.csv + file """${getS3Url()}/regression/ssb/sf1/${tableName}.tbl.gz""" + + time 10000 // limit inflight 10s + + // stream load action will check result, include Success status, and NumberTotalRows == NumberLoadedRows + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(json.NumberTotalRows, json.NumberLoadedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + i++ + } + + def table = "lineorder_flat" + def table_rows = 6001215 + sql new File("""${context.file.parent}/ddl/${table}_create.sql""").text + def rowCount = sql "select count(*) from ${table}" + if (rowCount[0][0] != table_rows) { + sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text + sql "set insert_timeout=3600" + def r = sql "select @@insert_timeout" + assertEquals(3600, r[0][0]) + year_cons = [ + 'lo_orderdate<19930101', + 'lo_orderdate>=19930101 and lo_orderdate<19940101', + 'lo_orderdate>=19940101 and lo_orderdate<19950101', + 'lo_orderdate>=19950101 and lo_orderdate<19960101', + 'lo_orderdate>=19960101 and lo_orderdate<19970101', + 'lo_orderdate>=19970101 and lo_orderdate<19980101', + 'lo_orderdate>=19980101' + ] + for (String con in year_cons){ + sql """ + INSERT INTO lineorder_flat + SELECT 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 + FROM ( SELECT lo_orderkey, lo_linenumber, lo_custkey, lo_partkey, lo_suppkey, + lo_orderdate, lo_orderpriority, lo_shippriority, lo_quantity, + lo_extendedprice, lo_ordtotalprice, lo_discount, lo_revenue, + lo_supplycost, lo_tax, lo_commitdate, lo_shipmode FROM lineorder WHERE ${con} ) l + INNER JOIN customer c ON (c.c_custkey = l.lo_custkey) + INNER JOIN supplier s ON (s.s_suppkey = l.lo_suppkey) + INNER JOIN part p ON (p.p_partkey = l.lo_partkey);""" + } + rowCount = sql "select count(*) from ${table}" + assertEquals(table_rows, rowCount[0][0]) + } +} diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.1.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.1.sql new file mode 100644 index 00000000000000..eae02823a08dac --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.1.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + LO_ORDERDATE >= 19930101 + AND LO_ORDERDATE <= 19931231 + AND LO_DISCOUNT BETWEEN 1 AND 3 + AND LO_QUANTITY < 25; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.2.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.2.sql new file mode 100644 index 00000000000000..3a899c93442a20 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.2.sql @@ -0,0 +1,24 @@ +-- 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. +--Q1.2 +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + LO_ORDERDATE >= 19940101 + AND LO_ORDERDATE <= 19940131 + AND LO_DISCOUNT BETWEEN 4 AND 6 + AND LO_QUANTITY BETWEEN 26 AND 35; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.3.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.3.sql new file mode 100644 index 00000000000000..5aaeff83a7e8bb --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q1.3.sql @@ -0,0 +1,25 @@ +-- 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. +--Q1.3 +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + weekofyear(LO_ORDERDATE) = 6 + AND LO_ORDERDATE >= 19940101 + AND LO_ORDERDATE <= 19941231 + AND LO_DISCOUNT BETWEEN 5 AND 7 + AND LO_QUANTITY BETWEEN 26 AND 35; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.1.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.1.sql new file mode 100644 index 00000000000000..254ea6481ac128 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.1.sql @@ -0,0 +1,24 @@ +-- 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. +--Q2.1 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE P_CATEGORY = 'MFGR#12' AND S_REGION = 'AMERICA' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.2.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.2.sql new file mode 100644 index 00000000000000..6a636f3a9e6f88 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.2.sql @@ -0,0 +1,27 @@ +-- 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. +--Q2.2 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE + P_BRAND >= 'MFGR#2221' + AND P_BRAND <= 'MFGR#2228' + AND S_REGION = 'ASIA' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.3.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.3.sql new file mode 100644 index 00000000000000..a2ef0c6df3789f --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q2.3.sql @@ -0,0 +1,26 @@ +-- 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. +--Q2.3 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE + P_BRAND = 'MFGR#2239' + AND S_REGION = 'EUROPE' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.1.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.1.sql new file mode 100644 index 00000000000000..8df98222c42b85 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.1.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.1 +SELECT + C_NATION, + S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_REGION = 'ASIA' + AND S_REGION = 'ASIA' + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_NATION, S_NATION, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.2.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.2.sql new file mode 100644 index 00000000000000..c588b5bbce66e4 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.2.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.2 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_NATION = 'UNITED STATES' + AND S_NATION = 'UNITED STATES' + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.3.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.3.sql new file mode 100644 index 00000000000000..9a099d1732170b --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.3.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.3 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_CITY IN ('UNITED KI1', 'UNITED KI5') + AND S_CITY IN ('UNITED KI1', 'UNITED KI5') + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.4.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.4.sql new file mode 100644 index 00000000000000..6bd71b5891a1a6 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q3.4.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.4 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_CITY IN ('UNITED KI1', 'UNITED KI5') + AND S_CITY IN ('UNITED KI1', 'UNITED KI5') + AND LO_ORDERDATE >= 19971201 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.1.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.1.sql new file mode 100644 index 00000000000000..aedd0e047e67fb --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.1.sql @@ -0,0 +1,27 @@ +-- 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. +--Q4.1 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + C_NATION, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + C_REGION = 'AMERICA' + AND S_REGION = 'AMERICA' + AND P_MFGR IN ('MFGR#1', 'MFGR#2') +GROUP BY YEAR, C_NATION +ORDER BY YEAR ASC, C_NATION ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.2.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.2.sql new file mode 100644 index 00000000000000..b9891ee4087b2f --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.2.sql @@ -0,0 +1,33 @@ +-- 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. +--Q4.2 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + S_NATION, + P_CATEGORY, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + C_REGION = 'AMERICA' + AND S_REGION = 'AMERICA' + AND LO_ORDERDATE >= 19970101 + AND LO_ORDERDATE <= 19981231 + AND P_MFGR IN ('MFGR#1', 'MFGR#2') +GROUP BY YEAR, S_NATION, P_CATEGORY +ORDER BY + YEAR ASC, + S_NATION ASC, + P_CATEGORY ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.3.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.3.sql new file mode 100644 index 00000000000000..6871023137ac95 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/flat_q4.3.sql @@ -0,0 +1,29 @@ +-- 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. +--Q4.3 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + S_CITY, + P_BRAND, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + S_NATION = 'UNITED STATES' + AND LO_ORDERDATE >= 19970101 + AND LO_ORDERDATE <= 19981231 + AND P_CATEGORY = 'MFGR#14' +GROUP BY YEAR, S_CITY, P_BRAND +ORDER BY YEAR ASC, S_CITY ASC, P_BRAND ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q1.1.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q1.1.sql new file mode 100644 index 00000000000000..4ef15e93ea2f5f --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q1.1.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_year = 1993 +AND lo_discount BETWEEN 1 AND 3 +AND lo_quantity < 25; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q1.2.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q1.2.sql new file mode 100644 index 00000000000000..1b8442bd939454 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q1.2.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_yearmonth = 'Jan1994' +AND lo_discount BETWEEN 4 AND 6 +AND lo_quantity BETWEEN 26 AND 35; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q1.3.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q1.3.sql new file mode 100644 index 00000000000000..ed6e51b1cfd264 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q1.3.sql @@ -0,0 +1,25 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_weeknuminyear= 6 +AND d_year = 1994 +AND lo_discount BETWEEN 5 AND 7 +AND lo_quantity BETWEEN 26 AND 35; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q2.1.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q2.1.sql new file mode 100644 index 00000000000000..e1a1f52d189e4e --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q2.1.sql @@ -0,0 +1,26 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_category = 'MFGR#12' +AND s_region = 'AMERICA' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q2.2.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q2.2.sql new file mode 100644 index 00000000000000..3db617011947ef --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q2.2.sql @@ -0,0 +1,27 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_brand BETWEEN 'MFGR#2221' +AND 'MFGR#2228' +AND s_region = 'ASIA' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q2.3.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q2.3.sql new file mode 100644 index 00000000000000..b70ca90666b8fe --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q2.3.sql @@ -0,0 +1,26 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_brand = 'MFGR#2239' +AND s_region = 'EUROPE' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q3.1.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q3.1.sql new file mode 100644 index 00000000000000..70f17d789b45a2 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q3.1.sql @@ -0,0 +1,28 @@ +-- 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. + +SELECT c_nation, s_nation, d_year, +SUM(lo_revenue) AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND c_region = 'ASIA' +AND s_region = 'ASIA' +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_nation, s_nation, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q3.2.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q3.2.sql new file mode 100644 index 00000000000000..a416fbea8b1768 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q3.2.sql @@ -0,0 +1,28 @@ +-- 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. + +SELECT c_city, s_city, d_year, sum(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND c_nation = 'UNITED STATES' +AND s_nation = 'UNITED STATES' +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q3.3.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q3.3.sql new file mode 100644 index 00000000000000..98e29b72e70bf0 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q3.3.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT c_city, s_city, d_year, SUM(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND (c_city='UNITED KI1' +OR c_city='UNITED KI5') +AND (s_city='UNITED KI1' +OR s_city='UNITED KI5') +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q3.4.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q3.4.sql new file mode 100644 index 00000000000000..65fe992ca4f12b --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q3.4.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT c_city, s_city, d_year, SUM(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND (c_city='UNITED KI1' +OR c_city='UNITED KI5') +AND (s_city='UNITED KI1' +OR s_city='UNITED KI5') +AND d_yearmonth = 'Dec1997' +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q4.1.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q4.1.sql new file mode 100644 index 00000000000000..bdcd730bf922fe --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q4.1.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT d_year, c_nation, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND c_region = 'AMERICA' +AND s_region = 'AMERICA' +AND (p_mfgr = 'MFGR#1' +OR p_mfgr = 'MFGR#2') +GROUP BY d_year, c_nation +ORDER BY d_year, c_nation; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q4.2.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q4.2.sql new file mode 100644 index 00000000000000..24c82cf682d155 --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q4.2.sql @@ -0,0 +1,31 @@ +-- 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. + +SELECT d_year, s_nation, p_category, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND c_region = 'AMERICA' +AND s_region = 'AMERICA' +AND (d_year = 1997 OR d_year = 1998) +AND (p_mfgr = 'MFGR#1' +OR p_mfgr = 'MFGR#2') +GROUP BY d_year, s_nation, p_category +ORDER BY d_year, s_nation, p_category; diff --git a/regression-test/suites/s3_vault/default_vault_p2/sql/q4.3.sql b/regression-test/suites/s3_vault/default_vault_p2/sql/q4.3.sql new file mode 100644 index 00000000000000..0dcc08bd26c8ad --- /dev/null +++ b/regression-test/suites/s3_vault/default_vault_p2/sql/q4.3.sql @@ -0,0 +1,29 @@ +-- 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. + +SELECT d_year, s_city, p_brand, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND s_nation = 'UNITED STATES' +AND (d_year = 1997 OR d_year = 1998) +AND p_category = 'MFGR#14' +GROUP BY d_year, s_city, p_brand +ORDER BY d_year, s_city, p_brand; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/customer_create.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/customer_create.sql new file mode 100644 index 00000000000000..746691100a6a58 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/customer_create.sql @@ -0,0 +1,16 @@ +CREATE TABLE IF NOT EXISTS `customer` ( + `c_custkey` int(11) NOT NULL COMMENT "", + `c_name` varchar(26) NOT NULL COMMENT "", + `c_address` varchar(41) NOT NULL COMMENT "", + `c_city` varchar(11) NOT NULL COMMENT "", + `c_nation` varchar(16) NOT NULL COMMENT "", + `c_region` varchar(13) NOT NULL COMMENT "", + `c_phone` varchar(16) NOT NULL COMMENT "", + `c_mktsegment` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`c_custkey`) +DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "multi_vault_ssb_hdfs_vault_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/customer_delete.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/customer_delete.sql new file mode 100644 index 00000000000000..fe22a226fedf85 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/customer_delete.sql @@ -0,0 +1 @@ +truncate table customer; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/date_create.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/date_create.sql new file mode 100644 index 00000000000000..33d445150eb388 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/date_create.sql @@ -0,0 +1,25 @@ +CREATE TABLE IF NOT EXISTS `date` ( + `d_datekey` int(11) NOT NULL COMMENT "", + `d_date` varchar(20) NOT NULL COMMENT "", + `d_dayofweek` varchar(10) NOT NULL COMMENT "", + `d_month` varchar(11) NOT NULL COMMENT "", + `d_year` int(11) NOT NULL COMMENT "", + `d_yearmonthnum` int(11) NOT NULL COMMENT "", + `d_yearmonth` varchar(9) NOT NULL COMMENT "", + `d_daynuminweek` int(11) NOT NULL COMMENT "", + `d_daynuminmonth` int(11) NOT NULL COMMENT "", + `d_daynuminyear` int(11) NOT NULL COMMENT "", + `d_monthnuminyear` int(11) NOT NULL COMMENT "", + `d_weeknuminyear` int(11) NOT NULL COMMENT "", + `d_sellingseason` varchar(14) NOT NULL COMMENT "", + `d_lastdayinweekfl` int(11) NOT NULL COMMENT "", + `d_lastdayinmonthfl` int(11) NOT NULL COMMENT "", + `d_holidayfl` int(11) NOT NULL COMMENT "", + `d_weekdayfl` int(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`d_datekey`) +DISTRIBUTED BY HASH(`d_datekey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "multi_vault_ssb_hdfs_vault_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/date_delete.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/date_delete.sql new file mode 100644 index 00000000000000..12933cbbad92da --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/date_delete.sql @@ -0,0 +1 @@ +truncate table `date`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_create.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_create.sql new file mode 100644 index 00000000000000..b6ee44d6ee056d --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_create.sql @@ -0,0 +1,25 @@ +CREATE TABLE IF NOT EXISTS `lineorder` ( + `lo_orderkey` bigint(20) NOT NULL COMMENT "", + `lo_linenumber` bigint(20) NOT NULL COMMENT "", + `lo_custkey` int(11) NOT NULL COMMENT "", + `lo_partkey` int(11) NOT NULL COMMENT "", + `lo_suppkey` int(11) NOT NULL COMMENT "", + `lo_orderdate` int(11) NOT NULL COMMENT "", + `lo_orderpriority` varchar(16) NOT NULL COMMENT "", + `lo_shippriority` int(11) NOT NULL COMMENT "", + `lo_quantity` bigint(20) NOT NULL COMMENT "", + `lo_extendedprice` bigint(20) NOT NULL COMMENT "", + `lo_ordtotalprice` bigint(20) NOT NULL COMMENT "", + `lo_discount` bigint(20) NOT NULL COMMENT "", + `lo_revenue` bigint(20) NOT NULL COMMENT "", + `lo_supplycost` bigint(20) NOT NULL COMMENT "", + `lo_tax` bigint(20) NOT NULL COMMENT "", + `lo_commitdate` bigint(20) NOT NULL COMMENT "", + `lo_shipmode` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`lo_orderkey`, `lo_linenumber`) +DISTRIBUTED BY HASH(`lo_orderkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "multi_vault_ssb_hdfs_vault_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_delete.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_delete.sql new file mode 100644 index 00000000000000..329e040060edc6 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_delete.sql @@ -0,0 +1 @@ +truncate table lineorder; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_flat_create.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_flat_create.sql new file mode 100644 index 00000000000000..5a1f716da625d2 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_flat_create.sql @@ -0,0 +1,46 @@ +CREATE TABLE IF NOT EXISTS `lineorder_flat` ( + `LO_ORDERDATE` date NOT NULL COMMENT "", + `LO_ORDERKEY` int(11) NOT NULL COMMENT "", + `LO_LINENUMBER` tinyint(4) NOT NULL COMMENT "", + `LO_CUSTKEY` int(11) NOT NULL COMMENT "", + `LO_PARTKEY` int(11) NOT NULL COMMENT "", + `LO_SUPPKEY` int(11) NOT NULL COMMENT "", + `LO_ORDERPRIORITY` varchar(100) NOT NULL COMMENT "", + `LO_SHIPPRIORITY` tinyint(4) NOT NULL COMMENT "", + `LO_QUANTITY` tinyint(4) NOT NULL COMMENT "", + `LO_EXTENDEDPRICE` int(11) NOT NULL COMMENT "", + `LO_ORDTOTALPRICE` int(11) NOT NULL COMMENT "", + `LO_DISCOUNT` tinyint(4) NOT NULL COMMENT "", + `LO_REVENUE` int(11) NOT NULL COMMENT "", + `LO_SUPPLYCOST` int(11) NOT NULL COMMENT "", + `LO_TAX` tinyint(4) NOT NULL COMMENT "", + `LO_COMMITDATE` date NOT NULL COMMENT "", + `LO_SHIPMODE` varchar(100) NOT NULL COMMENT "", + `C_NAME` varchar(100) NOT NULL COMMENT "", + `C_ADDRESS` varchar(100) NOT NULL COMMENT "", + `C_CITY` varchar(100) NOT NULL COMMENT "", + `C_NATION` varchar(100) NOT NULL COMMENT "", + `C_REGION` varchar(100) NOT NULL COMMENT "", + `C_PHONE` varchar(100) NOT NULL COMMENT "", + `C_MKTSEGMENT` varchar(100) NOT NULL COMMENT "", + `S_NAME` varchar(100) NOT NULL COMMENT "", + `S_ADDRESS` varchar(100) NOT NULL COMMENT "", + `S_CITY` varchar(100) NOT NULL COMMENT "", + `S_NATION` varchar(100) NOT NULL COMMENT "", + `S_REGION` varchar(100) NOT NULL COMMENT "", + `S_PHONE` varchar(100) NOT NULL COMMENT "", + `P_NAME` varchar(100) NOT NULL COMMENT "", + `P_MFGR` varchar(100) NOT NULL COMMENT "", + `P_CATEGORY` varchar(100) NOT NULL COMMENT "", + `P_BRAND` varchar(100) NOT NULL COMMENT "", + `P_COLOR` varchar(100) NOT NULL COMMENT "", + `P_TYPE` varchar(100) NOT NULL COMMENT "", + `P_SIZE` tinyint(4) NOT NULL COMMENT "", + `P_CONTAINER` varchar(100) NOT NULL COMMENT "" +) ENGINE=OLAP +UNIQUE KEY(`LO_ORDERDATE`, `LO_ORDERKEY`, `LO_LINENUMBER`) +DISTRIBUTED BY HASH(`LO_ORDERKEY`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "multi_vault_ssb_flat_hdfs_vault_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_flat_delete.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_flat_delete.sql new file mode 100644 index 00000000000000..13f6c32bb65a60 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/lineorder_flat_delete.sql @@ -0,0 +1 @@ +truncate table lineorder_flat; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/part_create.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/part_create.sql new file mode 100644 index 00000000000000..48c08e2d9b450a --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/part_create.sql @@ -0,0 +1,17 @@ +CREATE TABLE IF NOT EXISTS `part` ( + `p_partkey` int(11) NOT NULL COMMENT "", + `p_name` varchar(23) NOT NULL COMMENT "", + `p_mfgr` varchar(7) NOT NULL COMMENT "", + `p_category` varchar(8) NOT NULL COMMENT "", + `p_brand` varchar(10) NOT NULL COMMENT "", + `p_color` varchar(12) NOT NULL COMMENT "", + `p_type` varchar(26) NOT NULL COMMENT "", + `p_size` int(11) NOT NULL COMMENT "", + `p_container` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`p_partkey`) +DISTRIBUTED BY HASH(`p_partkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "multi_vault_ssb_hdfs_vault_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/part_delete.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/part_delete.sql new file mode 100644 index 00000000000000..02c6abd2539add --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/part_delete.sql @@ -0,0 +1 @@ +truncate table `part`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/supplier_create.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/supplier_create.sql new file mode 100644 index 00000000000000..00360492a45935 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/supplier_create.sql @@ -0,0 +1,15 @@ +CREATE TABLE IF NOT EXISTS `supplier` ( + `s_suppkey` int(11) NOT NULL COMMENT "", + `s_name` varchar(26) NOT NULL COMMENT "", + `s_address` varchar(26) NOT NULL COMMENT "", + `s_city` varchar(11) NOT NULL COMMENT "", + `s_nation` varchar(16) NOT NULL COMMENT "", + `s_region` varchar(13) NOT NULL COMMENT "", + `s_phone` varchar(16) NOT NULL COMMENT "" +) +UNIQUE KEY (`s_suppkey`) +DISTRIBUTED BY HASH(`s_suppkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "multi_vault_ssb_hdfs_vault_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/ddl/supplier_delete.sql b/regression-test/suites/s3_vault/multi_vault_p2/ddl/supplier_delete.sql new file mode 100644 index 00000000000000..39e663134cabd0 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/ddl/supplier_delete.sql @@ -0,0 +1 @@ +truncate table `supplier`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/load.groovy b/regression-test/suites/s3_vault/multi_vault_p2/load.groovy new file mode 100644 index 00000000000000..cc05cba69b6a36 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/load.groovy @@ -0,0 +1,158 @@ +// 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. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq +suite("load") { + + sql """ + CREATE STORAGE VAULT IF NOT EXISTS multi_vault_ssb_hdfs_vault_s3 + PROPERTIES ( + "type"="S3", + "s3.endpoint"="${getS3Endpoint()}", + "s3.region" = "${getS3Region()}", + "s3.access_key" = "${getS3AK()}", + "s3.secret_key" = "${getS3SK()}", + "s3.root.path" = "ssb_sf1_p2_s3", + "s3.bucket" = "${getS3BucketName()}", + "s3.external_endpoint" = "", + "provider" = "${getS3Provider()}" + ); + """ + + sql """ + CREATE STORAGE VAULT IF NOT EXISTS multi_vault_ssb_flat_hdfs_vault_s3 + PROPERTIES ( + "type"="S3", + "s3.endpoint"="${getS3Endpoint()}", + "s3.region" = "${getS3Region()}", + "s3.access_key" = "${getS3AK()}", + "s3.secret_key" = "${getS3SK()}", + "s3.root.path" = "ssb_sf1_p2_s3", + "s3.bucket" = "${getS3BucketName()}", + "s3.external_endpoint" = "", + "provider" = "${getS3Provider()}" + ); + """ + + // sleep 3 minutes to wait for BE sync the vault info from MS + sleep(180000) + + // ssb_sf1_p1 is writted to test unique key table merge correctly. + // It creates unique key table and sets bucket num to 1 in order to make sure that + // many rowsets will be created during loading and then the merge process will be triggered. + + def tables = ["customer", "lineorder", "part", "date", "supplier"] + def columns = ["""c_custkey,c_name,c_address,c_city,c_nation,c_region,c_phone,c_mktsegment,no_use""", + """lo_orderkey,lo_linenumber,lo_custkey,lo_partkey,lo_suppkey,lo_orderdate,lo_orderpriority, + lo_shippriority,lo_quantity,lo_extendedprice,lo_ordtotalprice,lo_discount, + lo_revenue,lo_supplycost,lo_tax,lo_commitdate,lo_shipmode,lo_dummy""", + """p_partkey,p_name,p_mfgr,p_category,p_brand,p_color,p_type,p_size,p_container,p_dummy""", + """d_datekey,d_date,d_dayofweek,d_month,d_year,d_yearmonthnum,d_yearmonth, + d_daynuminweek,d_daynuminmonth,d_daynuminyear,d_monthnuminyear,d_weeknuminyear, + d_sellingseason,d_lastdayinweekfl,d_lastdayinmonthfl,d_holidayfl,d_weekdayfl,d_dummy""", + """s_suppkey,s_name,s_address,s_city,s_nation,s_region,s_phone,s_dummy"""] + + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_create.sql""").text + sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text + } + def i = 0 + for (String tableName in tables) { + streamLoad { + // a default db 'regression_test' is specified in + // ${DORIS_HOME}/conf/regression-conf.groovy + table tableName + + // default label is UUID: + // set 'label' UUID.randomUUID().toString() + + // default column_separator is specify in doris fe config, usually is '\t'. + // this line change to ',' + set 'column_separator', '|' + set 'compress_type', 'GZ' + set 'columns', columns[i] + + + // relate to ${DORIS_HOME}/regression-test/data/demo/streamload_input.csv. + // also, you can stream load a http stream, e.g. http://xxx/some.csv + file """${getS3Url()}/regression/ssb/sf1/${tableName}.tbl.gz""" + + time 10000 // limit inflight 10s + + // stream load action will check result, include Success status, and NumberTotalRows == NumberLoadedRows + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(json.NumberTotalRows, json.NumberLoadedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + i++ + } + + def table = "lineorder_flat" + def table_rows = 6001215 + sql new File("""${context.file.parent}/ddl/${table}_create.sql""").text + def rowCount = sql "select count(*) from ${table}" + if (rowCount[0][0] != table_rows) { + sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text + sql "set insert_timeout=3600" + def r = sql "select @@insert_timeout" + assertEquals(3600, r[0][0]) + year_cons = [ + 'lo_orderdate<19930101', + 'lo_orderdate>=19930101 and lo_orderdate<19940101', + 'lo_orderdate>=19940101 and lo_orderdate<19950101', + 'lo_orderdate>=19950101 and lo_orderdate<19960101', + 'lo_orderdate>=19960101 and lo_orderdate<19970101', + 'lo_orderdate>=19970101 and lo_orderdate<19980101', + 'lo_orderdate>=19980101' + ] + for (String con in year_cons){ + sql """ + INSERT INTO lineorder_flat + SELECT 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 + FROM ( SELECT lo_orderkey, lo_linenumber, lo_custkey, lo_partkey, lo_suppkey, + lo_orderdate, lo_orderpriority, lo_shippriority, lo_quantity, + lo_extendedprice, lo_ordtotalprice, lo_discount, lo_revenue, + lo_supplycost, lo_tax, lo_commitdate, lo_shipmode FROM lineorder WHERE ${con} ) l + INNER JOIN customer c ON (c.c_custkey = l.lo_custkey) + INNER JOIN supplier s ON (s.s_suppkey = l.lo_suppkey) + INNER JOIN part p ON (p.p_partkey = l.lo_partkey);""" + } + rowCount = sql "select count(*) from ${table}" + assertEquals(table_rows, rowCount[0][0]) + } +} diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.1.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.1.sql new file mode 100644 index 00000000000000..eae02823a08dac --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.1.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + LO_ORDERDATE >= 19930101 + AND LO_ORDERDATE <= 19931231 + AND LO_DISCOUNT BETWEEN 1 AND 3 + AND LO_QUANTITY < 25; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.2.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.2.sql new file mode 100644 index 00000000000000..3a899c93442a20 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.2.sql @@ -0,0 +1,24 @@ +-- 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. +--Q1.2 +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + LO_ORDERDATE >= 19940101 + AND LO_ORDERDATE <= 19940131 + AND LO_DISCOUNT BETWEEN 4 AND 6 + AND LO_QUANTITY BETWEEN 26 AND 35; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.3.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.3.sql new file mode 100644 index 00000000000000..5aaeff83a7e8bb --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q1.3.sql @@ -0,0 +1,25 @@ +-- 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. +--Q1.3 +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + weekofyear(LO_ORDERDATE) = 6 + AND LO_ORDERDATE >= 19940101 + AND LO_ORDERDATE <= 19941231 + AND LO_DISCOUNT BETWEEN 5 AND 7 + AND LO_QUANTITY BETWEEN 26 AND 35; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.1.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.1.sql new file mode 100644 index 00000000000000..254ea6481ac128 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.1.sql @@ -0,0 +1,24 @@ +-- 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. +--Q2.1 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE P_CATEGORY = 'MFGR#12' AND S_REGION = 'AMERICA' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.2.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.2.sql new file mode 100644 index 00000000000000..6a636f3a9e6f88 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.2.sql @@ -0,0 +1,27 @@ +-- 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. +--Q2.2 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE + P_BRAND >= 'MFGR#2221' + AND P_BRAND <= 'MFGR#2228' + AND S_REGION = 'ASIA' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.3.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.3.sql new file mode 100644 index 00000000000000..a2ef0c6df3789f --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q2.3.sql @@ -0,0 +1,26 @@ +-- 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. +--Q2.3 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE + P_BRAND = 'MFGR#2239' + AND S_REGION = 'EUROPE' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.1.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.1.sql new file mode 100644 index 00000000000000..8df98222c42b85 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.1.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.1 +SELECT + C_NATION, + S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_REGION = 'ASIA' + AND S_REGION = 'ASIA' + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_NATION, S_NATION, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.2.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.2.sql new file mode 100644 index 00000000000000..c588b5bbce66e4 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.2.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.2 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_NATION = 'UNITED STATES' + AND S_NATION = 'UNITED STATES' + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.3.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.3.sql new file mode 100644 index 00000000000000..9a099d1732170b --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.3.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.3 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_CITY IN ('UNITED KI1', 'UNITED KI5') + AND S_CITY IN ('UNITED KI1', 'UNITED KI5') + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.4.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.4.sql new file mode 100644 index 00000000000000..6bd71b5891a1a6 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q3.4.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.4 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_CITY IN ('UNITED KI1', 'UNITED KI5') + AND S_CITY IN ('UNITED KI1', 'UNITED KI5') + AND LO_ORDERDATE >= 19971201 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.1.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.1.sql new file mode 100644 index 00000000000000..aedd0e047e67fb --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.1.sql @@ -0,0 +1,27 @@ +-- 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. +--Q4.1 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + C_NATION, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + C_REGION = 'AMERICA' + AND S_REGION = 'AMERICA' + AND P_MFGR IN ('MFGR#1', 'MFGR#2') +GROUP BY YEAR, C_NATION +ORDER BY YEAR ASC, C_NATION ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.2.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.2.sql new file mode 100644 index 00000000000000..b9891ee4087b2f --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.2.sql @@ -0,0 +1,33 @@ +-- 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. +--Q4.2 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + S_NATION, + P_CATEGORY, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + C_REGION = 'AMERICA' + AND S_REGION = 'AMERICA' + AND LO_ORDERDATE >= 19970101 + AND LO_ORDERDATE <= 19981231 + AND P_MFGR IN ('MFGR#1', 'MFGR#2') +GROUP BY YEAR, S_NATION, P_CATEGORY +ORDER BY + YEAR ASC, + S_NATION ASC, + P_CATEGORY ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.3.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.3.sql new file mode 100644 index 00000000000000..6871023137ac95 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/flat_q4.3.sql @@ -0,0 +1,29 @@ +-- 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. +--Q4.3 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + S_CITY, + P_BRAND, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + S_NATION = 'UNITED STATES' + AND LO_ORDERDATE >= 19970101 + AND LO_ORDERDATE <= 19981231 + AND P_CATEGORY = 'MFGR#14' +GROUP BY YEAR, S_CITY, P_BRAND +ORDER BY YEAR ASC, S_CITY ASC, P_BRAND ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.1.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.1.sql new file mode 100644 index 00000000000000..4ef15e93ea2f5f --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.1.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_year = 1993 +AND lo_discount BETWEEN 1 AND 3 +AND lo_quantity < 25; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.2.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.2.sql new file mode 100644 index 00000000000000..1b8442bd939454 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.2.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_yearmonth = 'Jan1994' +AND lo_discount BETWEEN 4 AND 6 +AND lo_quantity BETWEEN 26 AND 35; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.3.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.3.sql new file mode 100644 index 00000000000000..ed6e51b1cfd264 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q1.3.sql @@ -0,0 +1,25 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_weeknuminyear= 6 +AND d_year = 1994 +AND lo_discount BETWEEN 5 AND 7 +AND lo_quantity BETWEEN 26 AND 35; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.1.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.1.sql new file mode 100644 index 00000000000000..e1a1f52d189e4e --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.1.sql @@ -0,0 +1,26 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_category = 'MFGR#12' +AND s_region = 'AMERICA' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.2.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.2.sql new file mode 100644 index 00000000000000..3db617011947ef --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.2.sql @@ -0,0 +1,27 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_brand BETWEEN 'MFGR#2221' +AND 'MFGR#2228' +AND s_region = 'ASIA' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.3.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.3.sql new file mode 100644 index 00000000000000..b70ca90666b8fe --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q2.3.sql @@ -0,0 +1,26 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_brand = 'MFGR#2239' +AND s_region = 'EUROPE' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.1.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.1.sql new file mode 100644 index 00000000000000..70f17d789b45a2 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.1.sql @@ -0,0 +1,28 @@ +-- 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. + +SELECT c_nation, s_nation, d_year, +SUM(lo_revenue) AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND c_region = 'ASIA' +AND s_region = 'ASIA' +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_nation, s_nation, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.2.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.2.sql new file mode 100644 index 00000000000000..a416fbea8b1768 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.2.sql @@ -0,0 +1,28 @@ +-- 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. + +SELECT c_city, s_city, d_year, sum(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND c_nation = 'UNITED STATES' +AND s_nation = 'UNITED STATES' +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.3.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.3.sql new file mode 100644 index 00000000000000..98e29b72e70bf0 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.3.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT c_city, s_city, d_year, SUM(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND (c_city='UNITED KI1' +OR c_city='UNITED KI5') +AND (s_city='UNITED KI1' +OR s_city='UNITED KI5') +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.4.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.4.sql new file mode 100644 index 00000000000000..65fe992ca4f12b --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q3.4.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT c_city, s_city, d_year, SUM(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND (c_city='UNITED KI1' +OR c_city='UNITED KI5') +AND (s_city='UNITED KI1' +OR s_city='UNITED KI5') +AND d_yearmonth = 'Dec1997' +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.1.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.1.sql new file mode 100644 index 00000000000000..bdcd730bf922fe --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.1.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT d_year, c_nation, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND c_region = 'AMERICA' +AND s_region = 'AMERICA' +AND (p_mfgr = 'MFGR#1' +OR p_mfgr = 'MFGR#2') +GROUP BY d_year, c_nation +ORDER BY d_year, c_nation; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.2.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.2.sql new file mode 100644 index 00000000000000..24c82cf682d155 --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.2.sql @@ -0,0 +1,31 @@ +-- 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. + +SELECT d_year, s_nation, p_category, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND c_region = 'AMERICA' +AND s_region = 'AMERICA' +AND (d_year = 1997 OR d_year = 1998) +AND (p_mfgr = 'MFGR#1' +OR p_mfgr = 'MFGR#2') +GROUP BY d_year, s_nation, p_category +ORDER BY d_year, s_nation, p_category; diff --git a/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.3.sql b/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.3.sql new file mode 100644 index 00000000000000..0dcc08bd26c8ad --- /dev/null +++ b/regression-test/suites/s3_vault/multi_vault_p2/sql/q4.3.sql @@ -0,0 +1,29 @@ +-- 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. + +SELECT d_year, s_city, p_brand, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND s_nation = 'UNITED STATES' +AND (d_year = 1997 OR d_year = 1998) +AND p_category = 'MFGR#14' +GROUP BY d_year, s_city, p_brand +ORDER BY d_year, s_city, p_brand; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/customer_create.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/customer_create.sql new file mode 100644 index 00000000000000..de62670708bcb5 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/customer_create.sql @@ -0,0 +1,16 @@ +CREATE TABLE IF NOT EXISTS `customer` ( + `c_custkey` int(11) NOT NULL COMMENT "", + `c_name` varchar(26) NOT NULL COMMENT "", + `c_address` varchar(41) NOT NULL COMMENT "", + `c_city` varchar(11) NOT NULL COMMENT "", + `c_nation` varchar(16) NOT NULL COMMENT "", + `c_region` varchar(13) NOT NULL COMMENT "", + `c_phone` varchar(16) NOT NULL COMMENT "", + `c_mktsegment` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`c_custkey`) +DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "ssb_sf1_p2_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/customer_delete.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/customer_delete.sql new file mode 100644 index 00000000000000..fe22a226fedf85 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/customer_delete.sql @@ -0,0 +1 @@ +truncate table customer; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/date_create.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/date_create.sql new file mode 100644 index 00000000000000..fadfff0118fb29 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/date_create.sql @@ -0,0 +1,25 @@ +CREATE TABLE IF NOT EXISTS `date` ( + `d_datekey` int(11) NOT NULL COMMENT "", + `d_date` varchar(20) NOT NULL COMMENT "", + `d_dayofweek` varchar(10) NOT NULL COMMENT "", + `d_month` varchar(11) NOT NULL COMMENT "", + `d_year` int(11) NOT NULL COMMENT "", + `d_yearmonthnum` int(11) NOT NULL COMMENT "", + `d_yearmonth` varchar(9) NOT NULL COMMENT "", + `d_daynuminweek` int(11) NOT NULL COMMENT "", + `d_daynuminmonth` int(11) NOT NULL COMMENT "", + `d_daynuminyear` int(11) NOT NULL COMMENT "", + `d_monthnuminyear` int(11) NOT NULL COMMENT "", + `d_weeknuminyear` int(11) NOT NULL COMMENT "", + `d_sellingseason` varchar(14) NOT NULL COMMENT "", + `d_lastdayinweekfl` int(11) NOT NULL COMMENT "", + `d_lastdayinmonthfl` int(11) NOT NULL COMMENT "", + `d_holidayfl` int(11) NOT NULL COMMENT "", + `d_weekdayfl` int(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`d_datekey`) +DISTRIBUTED BY HASH(`d_datekey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "ssb_sf1_p2_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/date_delete.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/date_delete.sql new file mode 100644 index 00000000000000..12933cbbad92da --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/date_delete.sql @@ -0,0 +1 @@ +truncate table `date`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_create.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_create.sql new file mode 100644 index 00000000000000..eee10a809d563d --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_create.sql @@ -0,0 +1,25 @@ +CREATE TABLE IF NOT EXISTS `lineorder` ( + `lo_orderkey` bigint(20) NOT NULL COMMENT "", + `lo_linenumber` bigint(20) NOT NULL COMMENT "", + `lo_custkey` int(11) NOT NULL COMMENT "", + `lo_partkey` int(11) NOT NULL COMMENT "", + `lo_suppkey` int(11) NOT NULL COMMENT "", + `lo_orderdate` int(11) NOT NULL COMMENT "", + `lo_orderpriority` varchar(16) NOT NULL COMMENT "", + `lo_shippriority` int(11) NOT NULL COMMENT "", + `lo_quantity` bigint(20) NOT NULL COMMENT "", + `lo_extendedprice` bigint(20) NOT NULL COMMENT "", + `lo_ordtotalprice` bigint(20) NOT NULL COMMENT "", + `lo_discount` bigint(20) NOT NULL COMMENT "", + `lo_revenue` bigint(20) NOT NULL COMMENT "", + `lo_supplycost` bigint(20) NOT NULL COMMENT "", + `lo_tax` bigint(20) NOT NULL COMMENT "", + `lo_commitdate` bigint(20) NOT NULL COMMENT "", + `lo_shipmode` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`lo_orderkey`, `lo_linenumber`) +DISTRIBUTED BY HASH(`lo_orderkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "ssb_sf1_p2_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_delete.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_delete.sql new file mode 100644 index 00000000000000..329e040060edc6 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_delete.sql @@ -0,0 +1 @@ +truncate table lineorder; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_flat_create.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_flat_create.sql new file mode 100644 index 00000000000000..709f3ac1c2b151 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_flat_create.sql @@ -0,0 +1,46 @@ +CREATE TABLE IF NOT EXISTS `lineorder_flat` ( + `LO_ORDERDATE` date NOT NULL COMMENT "", + `LO_ORDERKEY` int(11) NOT NULL COMMENT "", + `LO_LINENUMBER` tinyint(4) NOT NULL COMMENT "", + `LO_CUSTKEY` int(11) NOT NULL COMMENT "", + `LO_PARTKEY` int(11) NOT NULL COMMENT "", + `LO_SUPPKEY` int(11) NOT NULL COMMENT "", + `LO_ORDERPRIORITY` varchar(100) NOT NULL COMMENT "", + `LO_SHIPPRIORITY` tinyint(4) NOT NULL COMMENT "", + `LO_QUANTITY` tinyint(4) NOT NULL COMMENT "", + `LO_EXTENDEDPRICE` int(11) NOT NULL COMMENT "", + `LO_ORDTOTALPRICE` int(11) NOT NULL COMMENT "", + `LO_DISCOUNT` tinyint(4) NOT NULL COMMENT "", + `LO_REVENUE` int(11) NOT NULL COMMENT "", + `LO_SUPPLYCOST` int(11) NOT NULL COMMENT "", + `LO_TAX` tinyint(4) NOT NULL COMMENT "", + `LO_COMMITDATE` date NOT NULL COMMENT "", + `LO_SHIPMODE` varchar(100) NOT NULL COMMENT "", + `C_NAME` varchar(100) NOT NULL COMMENT "", + `C_ADDRESS` varchar(100) NOT NULL COMMENT "", + `C_CITY` varchar(100) NOT NULL COMMENT "", + `C_NATION` varchar(100) NOT NULL COMMENT "", + `C_REGION` varchar(100) NOT NULL COMMENT "", + `C_PHONE` varchar(100) NOT NULL COMMENT "", + `C_MKTSEGMENT` varchar(100) NOT NULL COMMENT "", + `S_NAME` varchar(100) NOT NULL COMMENT "", + `S_ADDRESS` varchar(100) NOT NULL COMMENT "", + `S_CITY` varchar(100) NOT NULL COMMENT "", + `S_NATION` varchar(100) NOT NULL COMMENT "", + `S_REGION` varchar(100) NOT NULL COMMENT "", + `S_PHONE` varchar(100) NOT NULL COMMENT "", + `P_NAME` varchar(100) NOT NULL COMMENT "", + `P_MFGR` varchar(100) NOT NULL COMMENT "", + `P_CATEGORY` varchar(100) NOT NULL COMMENT "", + `P_BRAND` varchar(100) NOT NULL COMMENT "", + `P_COLOR` varchar(100) NOT NULL COMMENT "", + `P_TYPE` varchar(100) NOT NULL COMMENT "", + `P_SIZE` tinyint(4) NOT NULL COMMENT "", + `P_CONTAINER` varchar(100) NOT NULL COMMENT "" +) ENGINE=OLAP +UNIQUE KEY(`LO_ORDERDATE`, `LO_ORDERKEY`, `LO_LINENUMBER`) +DISTRIBUTED BY HASH(`LO_ORDERKEY`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "ssb_sf1_p2_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_flat_delete.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_flat_delete.sql new file mode 100644 index 00000000000000..13f6c32bb65a60 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/lineorder_flat_delete.sql @@ -0,0 +1 @@ +truncate table lineorder_flat; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/part_create.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/part_create.sql new file mode 100644 index 00000000000000..7f1b4620a9c6c4 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/part_create.sql @@ -0,0 +1,17 @@ +CREATE TABLE IF NOT EXISTS `part` ( + `p_partkey` int(11) NOT NULL COMMENT "", + `p_name` varchar(23) NOT NULL COMMENT "", + `p_mfgr` varchar(7) NOT NULL COMMENT "", + `p_category` varchar(8) NOT NULL COMMENT "", + `p_brand` varchar(10) NOT NULL COMMENT "", + `p_color` varchar(12) NOT NULL COMMENT "", + `p_type` varchar(26) NOT NULL COMMENT "", + `p_size` int(11) NOT NULL COMMENT "", + `p_container` varchar(11) NOT NULL COMMENT "" +) +UNIQUE KEY (`p_partkey`) +DISTRIBUTED BY HASH(`p_partkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "ssb_sf1_p2_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/part_delete.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/part_delete.sql new file mode 100644 index 00000000000000..02c6abd2539add --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/part_delete.sql @@ -0,0 +1 @@ +truncate table `part`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/supplier_create.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/supplier_create.sql new file mode 100644 index 00000000000000..bcaf9ea9b73941 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/supplier_create.sql @@ -0,0 +1,15 @@ +CREATE TABLE IF NOT EXISTS `supplier` ( + `s_suppkey` int(11) NOT NULL COMMENT "", + `s_name` varchar(26) NOT NULL COMMENT "", + `s_address` varchar(26) NOT NULL COMMENT "", + `s_city` varchar(11) NOT NULL COMMENT "", + `s_nation` varchar(16) NOT NULL COMMENT "", + `s_region` varchar(13) NOT NULL COMMENT "", + `s_phone` varchar(16) NOT NULL COMMENT "" +) +UNIQUE KEY (`s_suppkey`) +DISTRIBUTED BY HASH(`s_suppkey`) BUCKETS 1 +PROPERTIES ( +"replication_num" = "1", +"storage_vault" = "ssb_sf1_p2_s3" +); \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/supplier_delete.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/supplier_delete.sql new file mode 100644 index 00000000000000..39e663134cabd0 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/ddl/supplier_delete.sql @@ -0,0 +1 @@ +truncate table `supplier`; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/load.groovy b/regression-test/suites/s3_vault/ssb_sf1_p2/load.groovy new file mode 100644 index 00000000000000..98dcb09463373d --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/load.groovy @@ -0,0 +1,143 @@ +// 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. + +// Most of the cases are copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. + +// Note: To filter out tables from sql files, use the following one-liner comamnd +// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq +suite("load") { + + sql """ + CREATE STORAGE VAULT IF NOT EXISTS ssb_sf1_p2_s3 + PROPERTIES ( + "type"="S3", + "s3.endpoint"="${getS3Endpoint()}", + "s3.region" = "${getS3Region()}", + "s3.access_key" = "${getS3AK()}", + "s3.secret_key" = "${getS3SK()}", + "s3.root.path" = "ssb_sf1_p2_s3", + "s3.bucket" = "${getS3BucketName()}", + "s3.external_endpoint" = "", + "provider" = "${getS3Provider()}" + ); + """ + + // sleep 3 minutes to wait for BE sync the vault info from MS + sleep(180000) + + // ssb_sf1_p1 is writted to test unique key table merge correctly. + // It creates unique key table and sets bucket num to 1 in order to make sure that + // many rowsets will be created during loading and then the merge process will be triggered. + + def tables = ["customer", "lineorder", "part", "date", "supplier"] + def columns = ["""c_custkey,c_name,c_address,c_city,c_nation,c_region,c_phone,c_mktsegment,no_use""", + """lo_orderkey,lo_linenumber,lo_custkey,lo_partkey,lo_suppkey,lo_orderdate,lo_orderpriority, + lo_shippriority,lo_quantity,lo_extendedprice,lo_ordtotalprice,lo_discount, + lo_revenue,lo_supplycost,lo_tax,lo_commitdate,lo_shipmode,lo_dummy""", + """p_partkey,p_name,p_mfgr,p_category,p_brand,p_color,p_type,p_size,p_container,p_dummy""", + """d_datekey,d_date,d_dayofweek,d_month,d_year,d_yearmonthnum,d_yearmonth, + d_daynuminweek,d_daynuminmonth,d_daynuminyear,d_monthnuminyear,d_weeknuminyear, + d_sellingseason,d_lastdayinweekfl,d_lastdayinmonthfl,d_holidayfl,d_weekdayfl,d_dummy""", + """s_suppkey,s_name,s_address,s_city,s_nation,s_region,s_phone,s_dummy"""] + + for (String table in tables) { + sql new File("""${context.file.parent}/ddl/${table}_create.sql""").text + sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text + } + def i = 0 + for (String tableName in tables) { + streamLoad { + // a default db 'regression_test' is specified in + // ${DORIS_HOME}/conf/regression-conf.groovy + table tableName + + // default label is UUID: + // set 'label' UUID.randomUUID().toString() + + // default column_separator is specify in doris fe config, usually is '\t'. + // this line change to ',' + set 'column_separator', '|' + set 'compress_type', 'GZ' + set 'columns', columns[i] + + + // relate to ${DORIS_HOME}/regression-test/data/demo/streamload_input.csv. + // also, you can stream load a http stream, e.g. http://xxx/some.csv + file """${getS3Url()}/regression/ssb/sf1/${tableName}.tbl.gz""" + + time 10000 // limit inflight 10s + + // stream load action will check result, include Success status, and NumberTotalRows == NumberLoadedRows + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(json.NumberTotalRows, json.NumberLoadedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + i++ + } + + def table = "lineorder_flat" + def table_rows = 6001215 + sql new File("""${context.file.parent}/ddl/${table}_create.sql""").text + def rowCount = sql "select count(*) from ${table}" + if (rowCount[0][0] != table_rows) { + sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text + sql "set insert_timeout=3600" + def r = sql "select @@insert_timeout" + assertEquals(3600, r[0][0]) + year_cons = [ + 'lo_orderdate<19930101', + 'lo_orderdate>=19930101 and lo_orderdate<19940101', + 'lo_orderdate>=19940101 and lo_orderdate<19950101', + 'lo_orderdate>=19950101 and lo_orderdate<19960101', + 'lo_orderdate>=19960101 and lo_orderdate<19970101', + 'lo_orderdate>=19970101 and lo_orderdate<19980101', + 'lo_orderdate>=19980101' + ] + for (String con in year_cons){ + sql """ + INSERT INTO lineorder_flat + SELECT 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 + FROM ( SELECT lo_orderkey, lo_linenumber, lo_custkey, lo_partkey, lo_suppkey, + lo_orderdate, lo_orderpriority, lo_shippriority, lo_quantity, + lo_extendedprice, lo_ordtotalprice, lo_discount, lo_revenue, + lo_supplycost, lo_tax, lo_commitdate, lo_shipmode FROM lineorder WHERE ${con} ) l + INNER JOIN customer c ON (c.c_custkey = l.lo_custkey) + INNER JOIN supplier s ON (s.s_suppkey = l.lo_suppkey) + INNER JOIN part p ON (p.p_partkey = l.lo_partkey);""" + } + rowCount = sql "select count(*) from ${table}" + assertEquals(table_rows, rowCount[0][0]) + } +} diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.1.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.1.sql new file mode 100644 index 00000000000000..eae02823a08dac --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.1.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + LO_ORDERDATE >= 19930101 + AND LO_ORDERDATE <= 19931231 + AND LO_DISCOUNT BETWEEN 1 AND 3 + AND LO_QUANTITY < 25; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.2.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.2.sql new file mode 100644 index 00000000000000..3a899c93442a20 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.2.sql @@ -0,0 +1,24 @@ +-- 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. +--Q1.2 +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + LO_ORDERDATE >= 19940101 + AND LO_ORDERDATE <= 19940131 + AND LO_DISCOUNT BETWEEN 4 AND 6 + AND LO_QUANTITY BETWEEN 26 AND 35; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.3.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.3.sql new file mode 100644 index 00000000000000..5aaeff83a7e8bb --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q1.3.sql @@ -0,0 +1,25 @@ +-- 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. +--Q1.3 +SELECT SUM(LO_EXTENDEDPRICE * LO_DISCOUNT) AS revenue +FROM lineorder_flat +WHERE + weekofyear(LO_ORDERDATE) = 6 + AND LO_ORDERDATE >= 19940101 + AND LO_ORDERDATE <= 19941231 + AND LO_DISCOUNT BETWEEN 5 AND 7 + AND LO_QUANTITY BETWEEN 26 AND 35; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.1.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.1.sql new file mode 100644 index 00000000000000..254ea6481ac128 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.1.sql @@ -0,0 +1,24 @@ +-- 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. +--Q2.1 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE P_CATEGORY = 'MFGR#12' AND S_REGION = 'AMERICA' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.2.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.2.sql new file mode 100644 index 00000000000000..6a636f3a9e6f88 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.2.sql @@ -0,0 +1,27 @@ +-- 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. +--Q2.2 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE + P_BRAND >= 'MFGR#2221' + AND P_BRAND <= 'MFGR#2228' + AND S_REGION = 'ASIA' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.3.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.3.sql new file mode 100644 index 00000000000000..a2ef0c6df3789f --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q2.3.sql @@ -0,0 +1,26 @@ +-- 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. +--Q2.3 +SELECT + SUM(LO_REVENUE), (LO_ORDERDATE DIV 10000) AS YEAR, + P_BRAND +FROM lineorder_flat +WHERE + P_BRAND = 'MFGR#2239' + AND S_REGION = 'EUROPE' +GROUP BY YEAR, P_BRAND +ORDER BY YEAR, P_BRAND; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.1.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.1.sql new file mode 100644 index 00000000000000..8df98222c42b85 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.1.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.1 +SELECT + C_NATION, + S_NATION, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_REGION = 'ASIA' + AND S_REGION = 'ASIA' + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_NATION, S_NATION, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.2.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.2.sql new file mode 100644 index 00000000000000..c588b5bbce66e4 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.2.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.2 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_NATION = 'UNITED STATES' + AND S_NATION = 'UNITED STATES' + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.3.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.3.sql new file mode 100644 index 00000000000000..9a099d1732170b --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.3.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.3 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_CITY IN ('UNITED KI1', 'UNITED KI5') + AND S_CITY IN ('UNITED KI1', 'UNITED KI5') + AND LO_ORDERDATE >= 19920101 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.4.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.4.sql new file mode 100644 index 00000000000000..6bd71b5891a1a6 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q3.4.sql @@ -0,0 +1,29 @@ +-- 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. +--Q3.4 +SELECT + C_CITY, + S_CITY, (LO_ORDERDATE DIV 10000) AS YEAR, + SUM(LO_REVENUE) AS revenue +FROM lineorder_flat +WHERE + C_CITY IN ('UNITED KI1', 'UNITED KI5') + AND S_CITY IN ('UNITED KI1', 'UNITED KI5') + AND LO_ORDERDATE >= 19971201 + AND LO_ORDERDATE <= 19971231 +GROUP BY C_CITY, S_CITY, YEAR +ORDER BY YEAR ASC, revenue DESC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.1.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.1.sql new file mode 100644 index 00000000000000..aedd0e047e67fb --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.1.sql @@ -0,0 +1,27 @@ +-- 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. +--Q4.1 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + C_NATION, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + C_REGION = 'AMERICA' + AND S_REGION = 'AMERICA' + AND P_MFGR IN ('MFGR#1', 'MFGR#2') +GROUP BY YEAR, C_NATION +ORDER BY YEAR ASC, C_NATION ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.2.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.2.sql new file mode 100644 index 00000000000000..b9891ee4087b2f --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.2.sql @@ -0,0 +1,33 @@ +-- 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. +--Q4.2 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + S_NATION, + P_CATEGORY, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + C_REGION = 'AMERICA' + AND S_REGION = 'AMERICA' + AND LO_ORDERDATE >= 19970101 + AND LO_ORDERDATE <= 19981231 + AND P_MFGR IN ('MFGR#1', 'MFGR#2') +GROUP BY YEAR, S_NATION, P_CATEGORY +ORDER BY + YEAR ASC, + S_NATION ASC, + P_CATEGORY ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.3.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.3.sql new file mode 100644 index 00000000000000..6871023137ac95 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/flat_q4.3.sql @@ -0,0 +1,29 @@ +-- 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. +--Q4.3 +SELECT (LO_ORDERDATE DIV 10000) AS YEAR, + S_CITY, + P_BRAND, + SUM(LO_REVENUE - LO_SUPPLYCOST) AS profit +FROM lineorder_flat +WHERE + S_NATION = 'UNITED STATES' + AND LO_ORDERDATE >= 19970101 + AND LO_ORDERDATE <= 19981231 + AND P_CATEGORY = 'MFGR#14' +GROUP BY YEAR, S_CITY, P_BRAND +ORDER BY YEAR ASC, S_CITY ASC, P_BRAND ASC; \ No newline at end of file diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.1.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.1.sql new file mode 100644 index 00000000000000..4ef15e93ea2f5f --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.1.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_year = 1993 +AND lo_discount BETWEEN 1 AND 3 +AND lo_quantity < 25; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.2.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.2.sql new file mode 100644 index 00000000000000..1b8442bd939454 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.2.sql @@ -0,0 +1,24 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_yearmonth = 'Jan1994' +AND lo_discount BETWEEN 4 AND 6 +AND lo_quantity BETWEEN 26 AND 35; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.3.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.3.sql new file mode 100644 index 00000000000000..ed6e51b1cfd264 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q1.3.sql @@ -0,0 +1,25 @@ +-- 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. + +SELECT SUM(lo_extendedprice*lo_discount) AS +REVENUE +FROM lineorder, date +WHERE lo_orderdate = d_datekey +AND d_weeknuminyear= 6 +AND d_year = 1994 +AND lo_discount BETWEEN 5 AND 7 +AND lo_quantity BETWEEN 26 AND 35; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.1.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.1.sql new file mode 100644 index 00000000000000..e1a1f52d189e4e --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.1.sql @@ -0,0 +1,26 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_category = 'MFGR#12' +AND s_region = 'AMERICA' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.2.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.2.sql new file mode 100644 index 00000000000000..3db617011947ef --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.2.sql @@ -0,0 +1,27 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_brand BETWEEN 'MFGR#2221' +AND 'MFGR#2228' +AND s_region = 'ASIA' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.3.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.3.sql new file mode 100644 index 00000000000000..b70ca90666b8fe --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q2.3.sql @@ -0,0 +1,26 @@ +-- 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. + +SELECT SUM(lo_revenue), d_year, p_brand +FROM lineorder, date, part, supplier +WHERE lo_orderdate = d_datekey +AND lo_partkey = p_partkey +AND lo_suppkey = s_suppkey +AND p_brand = 'MFGR#2239' +AND s_region = 'EUROPE' +GROUP BY d_year, p_brand +ORDER BY d_year, p_brand; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.1.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.1.sql new file mode 100644 index 00000000000000..70f17d789b45a2 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.1.sql @@ -0,0 +1,28 @@ +-- 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. + +SELECT c_nation, s_nation, d_year, +SUM(lo_revenue) AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND c_region = 'ASIA' +AND s_region = 'ASIA' +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_nation, s_nation, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.2.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.2.sql new file mode 100644 index 00000000000000..a416fbea8b1768 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.2.sql @@ -0,0 +1,28 @@ +-- 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. + +SELECT c_city, s_city, d_year, sum(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND c_nation = 'UNITED STATES' +AND s_nation = 'UNITED STATES' +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.3.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.3.sql new file mode 100644 index 00000000000000..98e29b72e70bf0 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.3.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT c_city, s_city, d_year, SUM(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND (c_city='UNITED KI1' +OR c_city='UNITED KI5') +AND (s_city='UNITED KI1' +OR s_city='UNITED KI5') +AND d_year >= 1992 AND d_year <= 1997 +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.4.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.4.sql new file mode 100644 index 00000000000000..65fe992ca4f12b --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q3.4.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT c_city, s_city, d_year, SUM(lo_revenue) +AS REVENUE +FROM customer, lineorder, supplier, date +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_orderdate = d_datekey +AND (c_city='UNITED KI1' +OR c_city='UNITED KI5') +AND (s_city='UNITED KI1' +OR s_city='UNITED KI5') +AND d_yearmonth = 'Dec1997' +GROUP BY c_city, s_city, d_year +ORDER BY d_year ASC, REVENUE DESC; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.1.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.1.sql new file mode 100644 index 00000000000000..bdcd730bf922fe --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.1.sql @@ -0,0 +1,30 @@ +-- 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. + +SELECT d_year, c_nation, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND c_region = 'AMERICA' +AND s_region = 'AMERICA' +AND (p_mfgr = 'MFGR#1' +OR p_mfgr = 'MFGR#2') +GROUP BY d_year, c_nation +ORDER BY d_year, c_nation; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.2.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.2.sql new file mode 100644 index 00000000000000..24c82cf682d155 --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.2.sql @@ -0,0 +1,31 @@ +-- 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. + +SELECT d_year, s_nation, p_category, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND c_region = 'AMERICA' +AND s_region = 'AMERICA' +AND (d_year = 1997 OR d_year = 1998) +AND (p_mfgr = 'MFGR#1' +OR p_mfgr = 'MFGR#2') +GROUP BY d_year, s_nation, p_category +ORDER BY d_year, s_nation, p_category; diff --git a/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.3.sql b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.3.sql new file mode 100644 index 00000000000000..0dcc08bd26c8ad --- /dev/null +++ b/regression-test/suites/s3_vault/ssb_sf1_p2/sql/q4.3.sql @@ -0,0 +1,29 @@ +-- 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. + +SELECT d_year, s_city, p_brand, +SUM(lo_revenue - lo_supplycost) AS PROFIT +FROM date, customer, supplier, part, lineorder +WHERE lo_custkey = c_custkey +AND lo_suppkey = s_suppkey +AND lo_partkey = p_partkey +AND lo_orderdate = d_datekey +AND s_nation = 'UNITED STATES' +AND (d_year = 1997 OR d_year = 1998) +AND p_category = 'MFGR#14' +GROUP BY d_year, s_city, p_brand +ORDER BY d_year, s_city, p_brand; From 347d3b3c5a06986eaf3f4f90ee6c0af95e485e58 Mon Sep 17 00:00:00 2001 From: deardeng <565620795@qq.com> Date: Sun, 14 Apr 2024 09:44:25 +0800 Subject: [PATCH 53/71] =?UTF-8?q?[fix](create=20table)=20Fix=20create=20ta?= =?UTF-8?q?ble=20exception=20without=20cleaning=20the=20e=E2=80=A6=20(#335?= =?UTF-8?q?74)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../doris/datasource/InternalCatalog.java | 24 +++- .../test_create_table_exception.groovy | 127 ++++++++++++++++++ 2 files changed, 147 insertions(+), 4 deletions(-) create mode 100644 regression-test/suites/partition_p0/test_create_table_exception.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index f2c96ac3473d30..f06ce7d63a3bff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -127,6 +127,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.io.CountingDataOutputStream; import org.apache.doris.common.util.DbUtil; +import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.common.util.DynamicPartitionUtil; import org.apache.doris.common.util.IdGeneratorUtil; import org.apache.doris.common.util.MetaLockUtils; @@ -961,15 +962,14 @@ public boolean unprotectDropTable(Database db, Table table, boolean isForceDrop, return true; } - public void replayDropTable(Database db, long tableId, boolean isForceDrop, - Long recycleTime) throws MetaNotFoundException { + public void dropTable(Database db, long tableId, boolean isForceDrop, + Long recycleTime) throws MetaNotFoundException { Table table = db.getTableOrMetaException(tableId); db.writeLock(); table.writeLock(); try { unprotectDropTable(db, table, isForceDrop, true, recycleTime); - Env.getCurrentEnv().getQueryStats().clear(Env.getCurrentInternalCatalog().getId(), db.getId(), - tableId); + Env.getCurrentEnv().getQueryStats().clear(Env.getCurrentInternalCatalog().getId(), db.getId(), tableId); Env.getCurrentEnv().getAnalysisManager().removeTableStats(table.getId()); } finally { table.writeUnlock(); @@ -977,6 +977,11 @@ public void replayDropTable(Database db, long tableId, boolean isForceDrop, } } + public void replayDropTable(Database db, long tableId, boolean isForceDrop, + Long recycleTime) throws MetaNotFoundException { + dropTable(db, tableId, isForceDrop, recycleTime); + } + public void replayEraseTable(long tableId) { Env.getCurrentRecycleBin().replayEraseTable(tableId); } @@ -2803,6 +2808,11 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep if (!result.first) { ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); } + if (DebugPointUtil.isEnable("FE.createOlapTable.exception")) { + LOG.info("debug point FE.createOlapTable.exception, throw e"); + // not commit, not log edit + throw new DdlException("debug point FE.createOlapTable.exception"); + } if (result.second) { if (Env.getCurrentColocateIndex().isColocateTable(tableId)) { @@ -2834,6 +2844,7 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep TimeUtils.getCurrentFormatTime()); } } catch (DdlException e) { + LOG.warn("create table failed {} - {}", tabletIdSet, e.getMessage()); for (Long tabletId : tabletIdSet) { Env.getCurrentInvertedIndex().deleteTablet(tabletId); } @@ -2841,6 +2852,11 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws UserExcep if (Env.getCurrentColocateIndex().isColocateTable(tableId)) { Env.getCurrentColocateIndex().removeTable(tableId); } + try { + dropTable(db, tableId, true, 0L); + } catch (Exception ex) { + LOG.warn("drop table", ex); + } throw e; } diff --git a/regression-test/suites/partition_p0/test_create_table_exception.groovy b/regression-test/suites/partition_p0/test_create_table_exception.groovy new file mode 100644 index 00000000000000..49cadcd3af4bf9 --- /dev/null +++ b/regression-test/suites/partition_p0/test_create_table_exception.groovy @@ -0,0 +1,127 @@ +// 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.apache.doris.regression.suite.ClusterOptions +import org.apache.doris.regression.util.NodeType +import org.apache.doris.regression.suite.SuiteCluster + +suite("test_create_table_exception") { + def options = new ClusterOptions() + options.enableDebugPoints() + options.setFeNum(3) + options.feConfigs.add('max_dynamic_partition_num=2000') + + docker(options) { + sleep 2000 + def table1 = "normal_table" + def table2 = "range_table" + def table3 = "dynamic_partition_table" + try { + GetDebugPoint().enableDebugPointForAllFEs('FE.createOlapTable.exception', null) + def createTable = { -> + try_sql """ + CREATE TABLE $table1 ( + `k1` int(11) NULL, + `k2` int(11) NULL + ) + DUPLICATE KEY(`k1`, `k2`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES ( + "replication_num"="3" + ); + """ + + try_sql """ + CREATE TABLE IF NOT EXISTS $table2 ( + lo_orderdate int(11) NOT NULL COMMENT "", + lo_orderkey bigint(20) NOT NULL COMMENT "", + lo_linenumber bigint(20) NOT NULL COMMENT "", + lo_custkey int(11) NOT NULL COMMENT "", + lo_partkey int(11) NOT NULL COMMENT "", + lo_suppkey int(11) NOT NULL COMMENT "", + lo_orderpriority varchar(64) NOT NULL COMMENT "", + lo_shippriority int(11) NOT NULL COMMENT "", + lo_quantity bigint(20) NOT NULL COMMENT "", + lo_extendedprice bigint(20) NOT NULL COMMENT "", + lo_ordtotalprice bigint(20) NOT NULL COMMENT "", + lo_discount bigint(20) NOT NULL COMMENT "", + lo_revenue bigint(20) NOT NULL COMMENT "", + lo_supplycost bigint(20) NOT NULL COMMENT "", + lo_tax bigint(20) NOT NULL COMMENT "", + lo_commitdate bigint(20) NOT NULL COMMENT "", + lo_shipmode varchar(64) NOT NULL COMMENT "" ) + ENGINE=OLAP + UNIQUE KEY(lo_orderdate, lo_orderkey, lo_linenumber) + COMMENT "OLAP" + PARTITION BY RANGE(lo_orderdate) ( + PARTITION p1992 VALUES [("-2147483648"), ("19930101")), + PARTITION p1993 VALUES [("19930101"), ("19940101")), + PARTITION p1994 VALUES [("19940101"), ("19950101")), + PARTITION p1995 VALUES [("19950101"), ("19960101")), + PARTITION p1996 VALUES [("19960101"), ("19970101")), + PARTITION p1997 VALUES [("19970101"), ("19980101")), + PARTITION p1998 VALUES [("19980101"), ("19990101"))) + DISTRIBUTED BY HASH(lo_orderkey) BUCKETS 48; + """ + + try_sql """ + CREATE TABLE $table3 ( + time date, + key1 int, + key2 int, + value1 int, + value2 int + ) ENGINE = OLAP UNIQUE KEY( + `time`, + `key1`, + `key2` + ) COMMENT 'OLAP' PARTITION BY RANGE(`time`)() + DISTRIBUTED BY HASH(`key1`) BUCKETS 6 PROPERTIES ( + "file_cache_ttl_seconds" = "0", + "bloom_filter_columns" = "time", + "dynamic_partition.enable" = "true", + "dynamic_partition.time_unit" = "DAY", + "dynamic_partition.time_zone" = "Asia/Shanghai", + "dynamic_partition.start" = "-730", + "dynamic_partition.end" = "3", + "dynamic_partition.prefix" = "p", + "dynamic_partition.buckets" = "2", + "dynamic_partition.create_history_partition" = "true", + "dynamic_partition.history_partition_num" = "-1", + "dynamic_partition.hot_partition_num" = "0", + "dynamic_partition.reserved_history_periods" = "NULL", + "enable_unique_key_merge_on_write" = "true", + "light_schema_change" = "true" + ); + """ + } + createTable() + def result = sql """show tables;""" + assertEquals(result.size(), 0) + GetDebugPoint().disableDebugPointForAllFEs('FE.createOlapTable.exception') + createTable() + result = sql """show tables;""" + log.info(result.toString()) + assertEquals(result.size(), 3) + } finally { + GetDebugPoint().disableDebugPointForAllFEs('FE.createOlapTable.exception') + sql """drop table if exists ${table1}""" + sql """drop table if exists ${table2}""" + sql """drop table if exists ${table3}""" + } + } +} From 0b69fe4cd964b415263c5d2e5e19b9011a504f46 Mon Sep 17 00:00:00 2001 From: yujun Date: Sun, 14 Apr 2024 09:44:43 +0800 Subject: [PATCH 54/71] [fix](schema change) follow fe set sc fail replicas as bad (#33569) --- be/src/olap/schema_change.cpp | 6 ++ .../org/apache/doris/alter/AlterHandler.java | 1 + .../org/apache/doris/alter/AlterJobV2.java | 5 + .../org/apache/doris/alter/RollupJobV2.java | 27 +++--- .../apache/doris/alter/SchemaChangeJobV2.java | 27 +++--- .../test_schema_change_fail.groovy | 94 +++++++++++++++++++ 6 files changed, 138 insertions(+), 22 deletions(-) create mode 100644 regression-test/suites/schema_change_p2/test_schema_change_fail.groovy diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 13182467591a5e..2fab18d7858f49 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -820,6 +820,12 @@ Status SchemaChangeJob::_do_process_alter_tablet(const TAlterTabletReqV2& reques break; } + DBUG_EXECUTE_IF("SchemaChangeJob.process_alter_tablet.alter_fail", { + LOG(WARNING) << "inject alter tablet failed. base_tablet=" << request.base_tablet_id + << ", new_tablet=" << request.new_tablet_id; + break; + }); + // should check the max_version >= request.alter_version, if not the convert is useless if (max_rowset == nullptr || max_rowset->end_version() < request.alter_version) { res = Status::InternalError( diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java index 362d84caed85ad..8d18c687a40773 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java @@ -270,6 +270,7 @@ public void replayAlterJobV2(AlterJobV2 alterJob) throws AnalysisException { alterJob.replay(alterJob); alterJobsV2.put(alterJob.getJobId(), alterJob); } else { + existingJob.failedTabletBackends = alterJob.failedTabletBackends; existingJob.replay(alterJob); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java index de96d297cb2d4b..292e347a1298e3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java @@ -31,6 +31,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.task.AgentTask; +import com.google.common.collect.Maps; import com.google.gson.annotations.SerializedName; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -38,6 +39,7 @@ import java.io.DataInput; import java.io.IOException; import java.util.List; +import java.util.Map; /* * Version 2 of AlterJob, for replacing the old version of AlterJob. @@ -98,6 +100,9 @@ public enum JobType { @SerializedName(value = "watershedTxnId") protected long watershedTxnId = -1; + // save failed task after retry three times, tablet -> backends + @SerializedName(value = "failedTabletBackends") + protected Map> failedTabletBackends = Maps.newHashMap(); public AlterJobV2(String rawSql, long jobId, JobType jobType, long dbId, long tableId, String tableName, long timeoutMs) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index 9fd269b7986493..60c02b6b5252b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -135,8 +135,6 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable { // save all create rollup tasks private AgentBatchTask rollupBatchTask = new AgentBatchTask(); - // save failed task after retry three times, tabletId -> agentTask - private Map> failedAgentTasks = Maps.newHashMap(); private Analyzer analyzer; @@ -522,17 +520,18 @@ protected void runRunningJob() throws AlterCancelException { task.setFinished(true); AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.ALTER, task.getSignature()); LOG.warn("rollup task failed: " + task.getErrorMsg()); - if (!failedAgentTasks.containsKey(task.getTabletId())) { - failedAgentTasks.put(task.getTabletId(), Lists.newArrayList(task)); - } else { - failedAgentTasks.get(task.getTabletId()).add(task); + List failedBackends = failedTabletBackends.get(task.getTabletId()); + if (failedBackends == null) { + failedBackends = Lists.newArrayList(); + failedTabletBackends.put(task.getTabletId(), failedBackends); } + failedBackends.add(task.getBackendId()); int expectSucceedTaskNum = tbl.getPartitionInfo() .getReplicaAllocation(task.getPartitionId()).getTotalReplicaNum(); - int failedTaskCount = failedAgentTasks.get(task.getTabletId()).size(); + int failedTaskCount = failedBackends.size(); if (expectSucceedTaskNum - failedTaskCount < expectSucceedTaskNum / 2 + 1) { throw new AlterCancelException("rollup tasks failed on same tablet reach threshold " - + failedAgentTasks.get(task.getTabletId()) + ", reason=" + task.getErrorMsg()); + + failedTaskCount + ", reason=" + task.getErrorMsg()); } } } @@ -547,9 +546,11 @@ protected void runRunningJob() throws AlterCancelException { try { Preconditions.checkState(tbl.getState() == OlapTableState.ROLLUP); TabletInvertedIndex invertedIndex = Env.getCurrentInvertedIndex(); - for (List tasks : failedAgentTasks.values()) { - for (AgentTask task : tasks) { - invertedIndex.getReplica(task.getTabletId(), task.getBackendId()).setBad(true); + for (Map.Entry> entry : failedTabletBackends.entrySet()) { + long tabletId = entry.getKey(); + List failedBackends = entry.getValue(); + for (long backendId : failedBackends) { + invertedIndex.getReplica(tabletId, backendId).setBad(true); } } for (Map.Entry entry : this.partitionIdToRollupIndex.entrySet()) { @@ -599,8 +600,12 @@ private void onFinished(OlapTable tbl) { MaterializedIndex rollupIndex = partition.getIndex(rollupIndexId); Preconditions.checkNotNull(rollupIndex, rollupIndexId); for (Tablet tablet : rollupIndex.getTablets()) { + List failedBackends = failedTabletBackends.get(tablet.getId()); for (Replica replica : tablet.getReplicas()) { replica.setState(ReplicaState.NORMAL); + if (failedBackends != null && failedBackends.contains(replica.getBackendId())) { + replica.setBad(true); + } } } partition.visualiseShadowIndex(rollupIndexId, false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java index d65d873c33c274..f00177ec53746f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java @@ -131,8 +131,6 @@ public class SchemaChangeJobV2 extends AlterJobV2 { // save all schema change tasks private AgentBatchTask schemaChangeBatchTask = new AgentBatchTask(); - // save failed task after retry three times, tabletId -> agentTask - private Map> failedAgentTasks = Maps.newHashMap(); protected SchemaChangeJobV2() { super(JobType.SCHEMA_CHANGE); @@ -532,17 +530,18 @@ protected void runRunningJob() throws AlterCancelException { task.setFinished(true); AgentTaskQueue.removeTask(task.getBackendId(), TTaskType.ALTER, task.getSignature()); LOG.warn("schema change task failed: " + task.getErrorMsg()); - if (!failedAgentTasks.containsKey(task.getTabletId())) { - failedAgentTasks.put(task.getTabletId(), Lists.newArrayList(task)); - } else { - failedAgentTasks.get(task.getTabletId()).add(task); + List failedBackends = failedTabletBackends.get(task.getTabletId()); + if (failedBackends == null) { + failedBackends = Lists.newArrayList(); + failedTabletBackends.put(task.getTabletId(), failedBackends); } + failedBackends.add(task.getBackendId()); int expectSucceedTaskNum = tbl.getPartitionInfo() .getReplicaAllocation(task.getPartitionId()).getTotalReplicaNum(); - int failedTaskCount = failedAgentTasks.get(task.getTabletId()).size(); + int failedTaskCount = failedBackends.size(); if (expectSucceedTaskNum - failedTaskCount < expectSucceedTaskNum / 2 + 1) { throw new AlterCancelException("schema change tasks failed on same tablet reach threshold " - + failedAgentTasks.get(task.getTabletId())); + + failedTaskCount); } } } @@ -560,9 +559,11 @@ protected void runRunningJob() throws AlterCancelException { try { Preconditions.checkState(tbl.getState() == OlapTableState.SCHEMA_CHANGE); TabletInvertedIndex invertedIndex = Env.getCurrentInvertedIndex(); - for (List tasks : failedAgentTasks.values()) { - for (AgentTask task : tasks) { - invertedIndex.getReplica(task.getTabletId(), task.getBackendId()).setBad(true); + for (Map.Entry> entry : failedTabletBackends.entrySet()) { + long tabletId = entry.getKey(); + List failedBackends = entry.getValue(); + for (long backendId : failedBackends) { + invertedIndex.getReplica(tabletId, backendId).setBad(true); } } for (long partitionId : partitionIndexMap.rowKeySet()) { @@ -639,8 +640,12 @@ private void onFinished(OlapTable tbl) { // set replica state for (Tablet tablet : shadowIdx.getTablets()) { + List failedBackends = failedTabletBackends.get(tablet.getId()); for (Replica replica : tablet.getReplicas()) { replica.setState(ReplicaState.NORMAL); + if (failedBackends != null && failedBackends.contains(replica.getBackendId())) { + replica.setBad(true); + } } } diff --git a/regression-test/suites/schema_change_p2/test_schema_change_fail.groovy b/regression-test/suites/schema_change_p2/test_schema_change_fail.groovy new file mode 100644 index 00000000000000..f7ec1b42f0c1bd --- /dev/null +++ b/regression-test/suites/schema_change_p2/test_schema_change_fail.groovy @@ -0,0 +1,94 @@ +// 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.apache.doris.regression.util.DebugPoint +import org.apache.doris.regression.util.NodeType + +suite('test_schema_change_fail', 'nonConcurrent') { + if (isCloudMode()) { + return + } + + def frontends = sql_return_maparray('show frontends') + def backends = sql_return_maparray('show backends') + def forceReplicaNum = getFeConfig('force_olap_table_replication_num').toInteger() + if (frontends.size() < 2 || backends.size() < 3 || forceReplicaNum == 1) { + return + } + + def tbl = 'test_schema_change_fail' + + def beId = backends[0].BackendId.toLong() + def beHost = backends[0].Host + def beHttpPort = backends[0].HttpPort.toInteger() + def injectName = 'SchemaChangeJob.process_alter_tablet.alter_fail' + + def checkReplicaBad = { -> + def tabletId = sql_return_maparray("SHOW TABLETS FROM ${tbl}")[0].TabletId.toLong() + def replicas = sql_return_maparray(sql_return_maparray("SHOW TABLET ${tabletId}").DetailCmd) + assertEquals(backends.size(), replicas.size()) + for (def replica : replicas) { + if (replica.BackendId.toLong() == beId) { + assertEquals(true, replica.IsBad.toBoolean()) + } + } + } + + def followFe = frontends.stream().filter(fe -> !fe.IsMaster.toBoolean()).findFirst().orElse(null) + def followFeUrl = "jdbc:mysql://${followFe.Host}:${followFe.QueryPort}/?useLocalSessionState=false&allowLoadLocalInfile=false" + followFeUrl = context.config.buildUrlWithDb(followFeUrl, context.dbName) + + sql "DROP TABLE IF EXISTS ${tbl} FORCE" + sql """ + CREATE TABLE ${tbl} + ( + `a` TINYINT NOT NULL, + `b` TINYINT NULL + ) + UNIQUE KEY (`a`) + DISTRIBUTED BY HASH(`a`) BUCKETS 1 + PROPERTIES + ( + 'replication_num' = '${backends.size()}', + 'light_schema_change' = 'false' + ) + """ + + sql "INSERT INTO ${tbl} VALUES (1, 2), (3, 4)" + + try { + DebugPoint.enableDebugPoint(beHost, beHttpPort, NodeType.BE, injectName) + setFeConfig('disable_tablet_scheduler', true) + + sleep(1000) + sql "ALTER TABLE ${tbl} MODIFY COLUMN b DOUBLE" + sleep(5 * 1000) + + def jobs = sql_return_maparray "SHOW ALTER TABLE COLUMN WHERE TableName = '${tbl}' ORDER BY CreateTime DESC LIMIT 1" + assertEquals(1, jobs.size()) + assertEquals('FINISHED', jobs[0].State) + + checkReplicaBad() + connect('root', '', followFeUrl) { + checkReplicaBad() + } + } finally { + DebugPoint.disableDebugPoint(beHost, beHttpPort, NodeType.BE, injectName) + setFeConfig('disable_tablet_scheduler', false) + sql "DROP TABLE IF EXISTS ${tbl} FORCE" + } +} From 54ae5d4f33eb9bfd42621e2bc34f833c06e9ffba Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Sun, 14 Apr 2024 15:45:07 +0800 Subject: [PATCH 55/71] [fix](Nereids) create view should forward to master (#33626) fix: create view should forward to master --- .../doris/nereids/trees/plans/commands/CreateViewCommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateViewCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateViewCommand.java index d78308664b5e38..25cd171959df67 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateViewCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateViewCommand.java @@ -26,7 +26,7 @@ import org.apache.doris.qe.StmtExecutor; /** CreateViewCommand */ -public class CreateViewCommand extends Command { +public class CreateViewCommand extends Command implements ForwardWithSync { private final CreateViewInfo createViewInfo; public CreateViewCommand(CreateViewInfo createViewInfo) { From 69d6666f5ee8fb7fd97c8da005e0806e65bc7e8e Mon Sep 17 00:00:00 2001 From: Xin Liao Date: Sun, 14 Apr 2024 16:37:23 +0800 Subject: [PATCH 56/71] [fix](merge-iterator) Fix mem leak when get next batch failed (#33627) --- be/src/vec/olap/vgeneric_iterators.cpp | 4 ++-- be/src/vec/olap/vgeneric_iterators.h | 24 ++++++++---------------- 2 files changed, 10 insertions(+), 18 deletions(-) diff --git a/be/src/vec/olap/vgeneric_iterators.cpp b/be/src/vec/olap/vgeneric_iterators.cpp index 26fe590dd6038f..4e3df66cd0f694 100644 --- a/be/src/vec/olap/vgeneric_iterators.cpp +++ b/be/src/vec/olap/vgeneric_iterators.cpp @@ -334,14 +334,14 @@ Status VMergeIterator::init(const StorageReadOptions& opts) { _record_rowids = opts.record_rowids; for (auto& iter : _origin_iters) { - auto ctx = std::make_unique(std::move(iter), _sequence_id_idx, + auto ctx = std::make_shared(std::move(iter), _sequence_id_idx, _is_unique, _is_reverse, opts.read_orderby_key_columns); RETURN_IF_ERROR(ctx->init(opts)); if (!ctx->valid()) { continue; } - _merge_heap.push(ctx.release()); + _merge_heap.push(ctx); } _origin_iters.clear(); diff --git a/be/src/vec/olap/vgeneric_iterators.h b/be/src/vec/olap/vgeneric_iterators.h index d67bb68fefa542..89eb130348fac9 100644 --- a/be/src/vec/olap/vgeneric_iterators.h +++ b/be/src/vec/olap/vgeneric_iterators.h @@ -194,13 +194,7 @@ class VMergeIterator : public RowwiseIterator { _is_reverse(is_reverse), _merged_rows(merged_rows) {} - ~VMergeIterator() override { - while (!_merge_heap.empty()) { - auto ctx = _merge_heap.top(); - _merge_heap.pop(); - delete ctx; - } - } + ~VMergeIterator() override = default; Status init(const StorageReadOptions& opts) override; @@ -232,7 +226,7 @@ class VMergeIterator : public RowwiseIterator { _block_row_locations.resize(_block_row_max); } size_t row_idx = 0; - VMergeIteratorContext* pre_ctx = nullptr; + std::shared_ptr pre_ctx; while (_get_size(block) < _block_row_max) { if (_merge_heap.empty()) { break; @@ -249,7 +243,7 @@ class VMergeIterator : public RowwiseIterator { } pre_ctx = ctx; } - pre_ctx->set_pre_ctx_same(ctx); + pre_ctx->set_pre_ctx_same(ctx.get()); if (UNLIKELY(_record_rowids)) { _block_row_locations[row_idx] = ctx->current_row_location(); } @@ -272,9 +266,6 @@ class VMergeIterator : public RowwiseIterator { RETURN_IF_ERROR(ctx->advance()); if (ctx->valid()) { _merge_heap.push(ctx); - } else { - // Release ctx earlier to reduce resource consumed - delete ctx; } } if (!_merge_heap.empty()) { @@ -295,14 +286,15 @@ class VMergeIterator : public RowwiseIterator { const Schema* _schema = nullptr; struct VMergeContextComparator { - bool operator()(const VMergeIteratorContext* lhs, const VMergeIteratorContext* rhs) const { + bool operator()(const std::shared_ptr& lhs, + const std::shared_ptr& rhs) const { return lhs->compare(*rhs); } }; - using VMergeHeap = - std::priority_queue, - VMergeContextComparator>; + using VMergeHeap = std::priority_queue, + std::vector>, + VMergeContextComparator>; VMergeHeap _merge_heap; From 7fd008946f31a07a715b5333b36243fca0213bc2 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Sun, 14 Apr 2024 16:54:19 +0800 Subject: [PATCH 57/71] [fix](regression-test) fix unstable test (#33628) --- .../cache/parse_sql_from_sql_cache.groovy | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy index 7523df43f0cbb8..0a318218301dfc 100644 --- a/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy +++ b/regression-test/suites/nereids_p0/cache/parse_sql_from_sql_cache.groovy @@ -226,7 +226,7 @@ suite("parse_sql_from_sql_cache") { sql "alter view test_use_plan_cache9_view as select id from test_use_plan_cache9" assertNoCache "select * from test_use_plan_cache9_view" }), - extraThread( "testDropView", { + extraThread("testDropView", { createTestTable "test_use_plan_cache10" sql "drop view if exists test_use_plan_cache10_view" @@ -291,6 +291,8 @@ suite("parse_sql_from_sql_cache") { sql "select * from test_use_plan_cache12" assertHasCache "select * from test_use_plan_cache12" + sql "sync" + extraThread("test_cache_user1_thread", { connect(user = "test_cache_user1", password="DORIS@2024") { @@ -321,6 +323,8 @@ suite("parse_sql_from_sql_cache") { // after partition changed 10s, the sql cache can be used sleep(10000) + sql "sync" + extraThread("test_cache_user2_thread", { connect(user = "test_cache_user2", password="DORIS@2024") { sql "use ${dbName}" @@ -342,6 +346,8 @@ suite("parse_sql_from_sql_cache") { USING (id = 'concat(id, "**")')""" sql "set enable_nereids_planner=true" + sql "sync" + // after row policy changed, the cache is invalidate extraThread("test_cache_user2_thread2", { connect(user = "test_cache_user2", password="DORIS@2024") { @@ -377,6 +383,8 @@ suite("parse_sql_from_sql_cache") { USING (id = 'concat(id, "**")')""" sql "set enable_nereids_planner=true" + sql "sync" + // after partition changed 10s, the sql cache can be used sleep(10000) @@ -400,6 +408,8 @@ suite("parse_sql_from_sql_cache") { FOR test_cache_user3""" sql "set enable_nereids_planner=true" + sql "sync" + // after row policy changed, the cache is invalidate extraThread("test_cache_user3_thread2", { connect(user = "test_cache_user3", password="DORIS@2024") { @@ -425,6 +435,8 @@ suite("parse_sql_from_sql_cache") { sql "GRANT SELECT_PRIV ON regression_test.* TO test_cache_user4" sql "GRANT SELECT_PRIV ON ${dbName}.test_use_plan_cache15 TO test_cache_user4" + sql "sync" + extraThread("test_cache_user4_thread", { connect(user = "test_cache_user4", password="DORIS@2024") { sql "use ${dbName}" @@ -440,6 +452,8 @@ suite("parse_sql_from_sql_cache") { sql "REVOKE SELECT_PRIV ON ${dbName}.test_use_plan_cache15 FROM test_cache_user4" + sql "sync" + // after privileges changed, the cache is invalidate extraThread("test_cache_user4_thread2", { connect(user = "test_cache_user4", password="DORIS@2024") { @@ -484,7 +498,7 @@ suite("parse_sql_from_sql_cache") { extraThread("testUserVariable", { // make sure if the table has been dropped, the cache should invalidate, // so we should retry twice to check - for (i in 0..2) { + for (def i in 0..2) { createTestTable "test_use_plan_cache17" // after partition changed 10s, the sql cache can be used From 1b753be0116cd499afc9fe4c80a1524fc5c814aa Mon Sep 17 00:00:00 2001 From: Pxl Date: Sun, 14 Apr 2024 22:59:03 +0800 Subject: [PATCH 58/71] [Chore](status) change unknow filter error to internal error (#33632) change unknow filter error to internal error --- be/src/runtime/runtime_filter_mgr.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/be/src/runtime/runtime_filter_mgr.cpp b/be/src/runtime/runtime_filter_mgr.cpp index f1adb4ed198b95..65c1852625b131 100644 --- a/be/src/runtime/runtime_filter_mgr.cpp +++ b/be/src/runtime/runtime_filter_mgr.cpp @@ -72,7 +72,8 @@ Status RuntimeFilterMgr::get_consume_filters(const int filter_id, std::lock_guard l(_lock); auto iter = _consumer_map.find(filter_id); if (iter == _consumer_map.end()) { - return Status::InvalidArgument("unknown filter: {}, role: CONSUMER.", filter_id); + return Status::InternalError("get_consume_filters meet unknown filter: {}, role: CONSUMER.", + filter_id); } for (auto& holder : iter->second) { consumer_filters.emplace_back(holder.filter); @@ -153,8 +154,10 @@ Status RuntimeFilterMgr::get_local_merge_producer_filters( std::lock_guard l(_lock); auto iter = _local_merge_producer_map.find(filter_id); if (iter == _local_merge_producer_map.end()) { - return Status::InvalidArgument("unknown filter: {}, role: LOCAL_MERGE_PRODUCER.", - filter_id); + return Status::InternalError( + "get_local_merge_producer_filters meet unknown filter: {}, role: " + "LOCAL_MERGE_PRODUCER.", + filter_id); } *local_merge_filters = &iter->second; DCHECK(!iter->second.filters.empty()); @@ -194,7 +197,8 @@ Status RuntimeFilterMgr::update_filter(const PPublishFilterRequest* request, std::lock_guard l(_lock); auto iter = _consumer_map.find(filter_id); if (iter == _consumer_map.end()) { - return Status::InvalidArgument("unknown filter: {}, role: CONSUMER.", filter_id); + return Status::InternalError("update_filter meet unknown filter: {}, role: CONSUMER.", + filter_id); } for (auto& holder : iter->second) { filters.emplace_back(holder.filter); From 1cd7d69d7d7c5847a1e2897afd370c0d90efb7a5 Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Mon, 15 Apr 2024 08:49:31 +0800 Subject: [PATCH 59/71] [Fix](compaction) Fix single compaction to get all local versions (#33480) --- be/src/olap/olap_server.cpp | 2 +- be/src/olap/single_replica_compaction.cpp | 2 +- be/src/olap/tablet.cpp | 11 +++---- be/src/olap/tablet.h | 4 ++- be/test/olap/tablet_test.cpp | 36 +++++++++++++++++++++++ 5 files changed, 47 insertions(+), 8 deletions(-) diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index 239816be485f90..aaa1e1dd7ac5ad 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -803,7 +803,7 @@ void StorageEngine::get_tablet_rowset_versions(const PGetTabletVersionsRequest* response->mutable_status()->set_status_code(TStatusCode::CANCELLED); return; } - std::vector local_versions = tablet->get_all_versions(); + std::vector local_versions = tablet->get_all_local_versions(); for (const auto& local_version : local_versions) { auto version = response->add_versions(); version->set_first(local_version.first); diff --git a/be/src/olap/single_replica_compaction.cpp b/be/src/olap/single_replica_compaction.cpp index 9485f127e5eb9a..cd83f28be49f2f 100644 --- a/be/src/olap/single_replica_compaction.cpp +++ b/be/src/olap/single_replica_compaction.cpp @@ -187,7 +187,7 @@ Status SingleReplicaCompaction::_get_rowset_verisons_from_peer( bool SingleReplicaCompaction::_find_rowset_to_fetch(const std::vector& peer_versions, Version* proper_version) { // already sorted - std::vector local_versions = tablet()->get_all_versions(); + std::vector local_versions = tablet()->get_all_local_versions(); for (const auto& v : local_versions) { VLOG_CRITICAL << _tablet->tablet_id() << " tablet local version: " << v.first << " - " << v.second; diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index a91dab3c11dc03..6734bc7006309c 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -1638,13 +1638,14 @@ void Tablet::execute_single_replica_compaction(SingleReplicaCompaction& compacti set_last_failure_time(this, compaction, 0); } -std::vector Tablet::get_all_versions() { +std::vector Tablet::get_all_local_versions() { std::vector local_versions; { - std::lock_guard wrlock(_meta_lock); - SCOPED_SIMPLE_TRACE_IF_TIMEOUT(TRACE_TABLET_LOCK_THRESHOLD); - for (const auto& it : _rs_version_map) { - local_versions.emplace_back(it.first); + std::shared_lock rlock(_meta_lock); + for (const auto& [version, rs] : _rs_version_map) { + if (rs->is_local()) { + local_versions.emplace_back(version); + } } } std::sort(local_versions.begin(), local_versions.end(), diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h index f9e78c6bdfbea5..bf3c34a1a2ef5e 100644 --- a/be/src/olap/tablet.h +++ b/be/src/olap/tablet.h @@ -263,7 +263,9 @@ class Tablet final : public BaseTablet { std::vector pick_candidate_rowsets_to_build_inverted_index( const std::set& alter_index_uids, bool is_drop_op); - std::vector get_all_versions(); + // used for single compaction to get the local versions + // Single compaction does not require remote rowsets and cannot violate the cooldown semantics + std::vector get_all_local_versions(); std::vector pick_first_consecutive_empty_rowsets(int limit); diff --git a/be/test/olap/tablet_test.cpp b/be/test/olap/tablet_test.cpp index c8bba09b11c50a..1ab2766948db32 100644 --- a/be/test/olap/tablet_test.cpp +++ b/be/test/olap/tablet_test.cpp @@ -146,6 +146,20 @@ class TestTablet : public testing::Test { pb1->set_tablet_schema(_tablet_meta->tablet_schema()); } + void init_rs_meta_resource(RowsetMetaSharedPtr& pb1, int64_t start, int64_t end, + bool is_local) { + RowsetMetaPB rowset_meta_pb; + json2pb::JsonToProtoMessage(_json_rowset_meta, &rowset_meta_pb); + rowset_meta_pb.set_start_version(start); + rowset_meta_pb.set_end_version(end); + rowset_meta_pb.set_creation_time(10000); + if (!is_local) { + rowset_meta_pb.set_resource_id("100"); + } + pb1->init_from_pb(rowset_meta_pb); + pb1->set_tablet_schema(_tablet_meta->tablet_schema()); + } + void init_all_rs_meta(std::vector* rs_metas) { RowsetMetaSharedPtr ptr1(new RowsetMeta()); init_rs_meta(ptr1, 0, 0); @@ -395,4 +409,26 @@ TEST_F(TestTablet, cooldown_policy) { } } +TEST_F(TestTablet, get_local_versions) { + // 10 remote rowsets + for (int i = 1; i <= 10; i++) { + auto ptr = std::make_shared(); + init_rs_meta_resource(ptr, i, i, false); + static_cast(_tablet_meta->add_rs_meta(ptr)); + } + + // 20 local rowsets + for (int i = 11; i <= 30; i++) { + auto ptr = std::make_shared(); + init_rs_meta_resource(ptr, i, i, true); + static_cast(_tablet_meta->add_rs_meta(ptr)); + } + + static_cast(_data_dir->init()); + TabletSharedPtr _tablet(new Tablet(*k_engine, _tablet_meta, _data_dir.get())); + static_cast(_tablet->init()); + const auto& local_versions = _tablet->get_all_local_versions(); + ASSERT_EQ(local_versions.size(), 20); +} + } // namespace doris From 8e7454fc2a7ef1bf862eddf1a46ff935648af12d Mon Sep 17 00:00:00 2001 From: qiye Date: Mon, 15 Apr 2024 09:13:13 +0800 Subject: [PATCH 60/71] [improvement](binlog)Support inverted index format v2 in CCR (#33415) --- be/src/olap/rowset/beta_rowset.cpp | 49 ++-- be/src/olap/snapshot_manager.cpp | 57 +++-- be/src/olap/tablet.cpp | 19 +- be/src/service/backend_service.cpp | 104 +++++--- .../doris/regression/suite/Syncer.groovy | 10 +- .../test_binlog_config_change.groovy | 217 ++++++++++++++++ .../inverted_index/test_get_binlog.groovy | 239 ++++++++++++++++++ .../inverted_index/test_ingest_binlog.groovy | 223 ++++++++++++++++ .../inverted_index/test_multi_buckets.groovy | 180 +++++++++++++ .../inverted_index/test_backup_restore.groovy | 196 ++++++++++++++ 10 files changed, 1220 insertions(+), 74 deletions(-) create mode 100644 regression-test/suites/ccr_syncer_p0/inverted_index/test_binlog_config_change.groovy create mode 100644 regression-test/suites/ccr_syncer_p0/inverted_index/test_get_binlog.groovy create mode 100644 regression-test/suites/ccr_syncer_p0/inverted_index/test_ingest_binlog.groovy create mode 100644 regression-test/suites/ccr_syncer_p0/inverted_index/test_multi_buckets.groovy create mode 100644 regression-test/suites/ccr_syncer_p1/inverted_index/test_backup_restore.groovy diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp index 405cfb15af7198..ac97fe0f28dc07 100644 --- a/be/src/olap/rowset/beta_rowset.cpp +++ b/be/src/olap/rowset/beta_rowset.cpp @@ -576,24 +576,41 @@ Status BetaRowset::add_to_binlog() { } linked_success_files.push_back(binlog_file); - for (const auto& index : _schema->indexes()) { - if (index.index_type() != IndexType::INVERTED) { - continue; + if (_schema->get_inverted_index_storage_format() == InvertedIndexStorageFormatPB::V1) { + for (const auto& index : _schema->indexes()) { + if (index.index_type() != IndexType::INVERTED) { + continue; + } + auto index_id = index.index_id(); + auto index_file = InvertedIndexDescriptor::get_index_file_name( + seg_file, index_id, index.get_index_suffix()); + auto binlog_index_file = (std::filesystem::path(binlog_dir) / + std::filesystem::path(index_file).filename()) + .string(); + VLOG_DEBUG << "link " << index_file << " to " << binlog_index_file; + if (!local_fs->link_file(index_file, binlog_index_file).ok()) { + status = Status::Error( + "fail to create hard link. from={}, to={}, errno={}", index_file, + binlog_index_file, Errno::no()); + return status; + } + linked_success_files.push_back(binlog_index_file); } - auto index_id = index.index_id(); - auto index_file = InvertedIndexDescriptor::get_index_file_name( - seg_file, index_id, index.get_index_suffix()); - auto binlog_index_file = (std::filesystem::path(binlog_dir) / - std::filesystem::path(index_file).filename()) - .string(); - VLOG_DEBUG << "link " << index_file << " to " << binlog_index_file; - if (!local_fs->link_file(index_file, binlog_index_file).ok()) { - status = Status::Error( - "fail to create hard link. from={}, to={}, errno={}", index_file, - binlog_index_file, Errno::no()); - return status; + } else { + if (_schema->has_inverted_index()) { + auto index_file = InvertedIndexDescriptor::get_index_file_name(seg_file); + auto binlog_index_file = (std::filesystem::path(binlog_dir) / + std::filesystem::path(index_file).filename()) + .string(); + VLOG_DEBUG << "link " << index_file << " to " << binlog_index_file; + if (!local_fs->link_file(index_file, binlog_index_file).ok()) { + status = Status::Error( + "fail to create hard link. from={}, to={}, errno={}", index_file, + binlog_index_file, Errno::no()); + return status; + } + linked_success_files.push_back(binlog_index_file); } - linked_success_files.push_back(binlog_index_file); } } diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 1c06414a311074..f19abfb0d28126 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -663,26 +663,47 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet } linked_success_files.push_back(snapshot_segment_file_path); - for (const auto& index : tablet_schema.indexes()) { - if (index.index_type() != IndexType::INVERTED) { - continue; + if (tablet_schema.get_inverted_index_storage_format() == + InvertedIndexStorageFormatPB::V1) { + for (const auto& index : tablet_schema.indexes()) { + if (index.index_type() != IndexType::INVERTED) { + continue; + } + auto index_id = index.index_id(); + auto index_file = ref_tablet->get_segment_index_filepath( + rowset_id, segment_index, index_id); + auto snapshot_segment_index_file_path = + fmt::format("{}/{}_{}_{}.binlog-index", schema_full_path, rowset_id, + segment_index, index_id); + VLOG_DEBUG << "link " << index_file << " to " + << snapshot_segment_index_file_path; + res = io::global_local_filesystem()->link_file( + index_file, snapshot_segment_index_file_path); + if (!res.ok()) { + LOG(WARNING) << "fail to link binlog index file. [src=" << index_file + << ", dest=" << snapshot_segment_index_file_path << "]"; + break; + } + linked_success_files.push_back(snapshot_segment_index_file_path); } - auto index_id = index.index_id(); - auto index_file = ref_tablet->get_segment_index_filepath( - rowset_id, segment_index, index_id); - auto snapshot_segment_index_file_path = - fmt::format("{}/{}_{}_{}.binlog-index", schema_full_path, rowset_id, - segment_index, index_id); - VLOG_DEBUG << "link " << index_file << " to " - << snapshot_segment_index_file_path; - res = io::global_local_filesystem()->link_file( - index_file, snapshot_segment_index_file_path); - if (!res.ok()) { - LOG(WARNING) << "fail to link binlog index file. [src=" << index_file - << ", dest=" << snapshot_segment_index_file_path << "]"; - break; + } else { + if (tablet_schema.has_inverted_index()) { + auto index_file = + InvertedIndexDescriptor::get_index_file_name(segment_file_path); + auto snapshot_segment_index_file_path = + fmt::format("{}/{}_{}.binlog-index", schema_full_path, rowset_id, + segment_index); + VLOG_DEBUG << "link " << index_file << " to " + << snapshot_segment_index_file_path; + res = io::global_local_filesystem()->link_file( + index_file, snapshot_segment_index_file_path); + if (!res.ok()) { + LOG(WARNING) << "fail to link binlog index file. [src=" << index_file + << ", dest=" << snapshot_segment_index_file_path << "]"; + break; + } + linked_success_files.push_back(snapshot_segment_index_file_path); } - linked_success_files.push_back(snapshot_segment_index_file_path); } } diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 6734bc7006309c..b97fdfe8e72d95 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -2390,14 +2390,25 @@ std::string Tablet::get_segment_filepath(std::string_view rowset_id, int64_t seg std::string Tablet::get_segment_index_filepath(std::string_view rowset_id, std::string_view segment_index, std::string_view index_id) const { - // TODO(qiye): support inverted index file format v2, when https://github.com/apache/doris/pull/30145 is merged - return fmt::format("{}/_binlog/{}_{}_{}.idx", _tablet_path, rowset_id, segment_index, index_id); + auto format = _tablet_meta->tablet_schema()->get_inverted_index_storage_format(); + if (format == doris::InvertedIndexStorageFormatPB::V1) { + return fmt::format("{}/_binlog/{}_{}_{}.idx", _tablet_path, rowset_id, segment_index, + index_id); + } else { + return fmt::format("{}/_binlog/{}_{}.idx", _tablet_path, rowset_id, segment_index); + } } std::string Tablet::get_segment_index_filepath(std::string_view rowset_id, int64_t segment_index, int64_t index_id) const { - // TODO(qiye): support inverted index file format v2, when https://github.com/apache/doris/pull/30145 is merged - return fmt::format("{}/_binlog/{}_{}_{}.idx", _tablet_path, rowset_id, segment_index, index_id); + auto format = _tablet_meta->tablet_schema()->get_inverted_index_storage_format(); + if (format == doris::InvertedIndexStorageFormatPB::V1) { + return fmt::format("{}/_binlog/{}_{}_{}.idx", _tablet_path, rowset_id, segment_index, + index_id); + } else { + DCHECK(index_id == -1); + return fmt::format("{}/_binlog/{}_{}.idx", _tablet_path, rowset_id, segment_index); + } } std::vector Tablet::get_binlog_filepath(std::string_view binlog_version) const { diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index c61e0b86556f93..b1a110144ef8c7 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -307,41 +307,81 @@ void _ingest_binlog(StorageEngine& engine, IngestBinlogArg* arg) { std::vector segment_index_file_sizes; std::vector segment_index_file_names; auto tablet_schema = rowset_meta->tablet_schema(); - for (const auto& index : tablet_schema->indexes()) { - if (index.index_type() != IndexType::INVERTED) { - continue; + if (tablet_schema->get_inverted_index_storage_format() == InvertedIndexStorageFormatPB::V1) { + for (const auto& index : tablet_schema->indexes()) { + if (index.index_type() != IndexType::INVERTED) { + continue; + } + auto index_id = index.index_id(); + for (int64_t segment_index = 0; segment_index < num_segments; ++segment_index) { + auto get_segment_index_file_size_url = fmt::format( + "{}?method={}&tablet_id={}&rowset_id={}&segment_index={}&segment_index_id={" + "}", + binlog_api_url, "get_segment_index_file", request.remote_tablet_id, + remote_rowset_id, segment_index, index_id); + uint64_t segment_index_file_size; + auto get_segment_index_file_size_cb = + [&get_segment_index_file_size_url, + &segment_index_file_size](HttpClient* client) { + RETURN_IF_ERROR(client->init(get_segment_index_file_size_url)); + client->set_timeout_ms(kMaxTimeoutMs); + RETURN_IF_ERROR(client->head()); + return client->get_content_length(&segment_index_file_size); + }; + auto index_file = InvertedIndexDescriptor::inverted_index_file_path( + local_tablet->tablet_path(), rowset_meta->rowset_id(), segment_index, + index_id, index.get_index_suffix()); + segment_index_file_names.push_back(index_file); + + status = HttpClient::execute_with_retry(max_retry, 1, + get_segment_index_file_size_cb); + if (!status.ok()) { + LOG(WARNING) << "failed to get segment file size from " + << get_segment_index_file_size_url + << ", status=" << status.to_string(); + status.to_thrift(&tstatus); + return; + } + + segment_index_file_sizes.push_back(segment_index_file_size); + segment_index_file_urls.push_back(std::move(get_segment_index_file_size_url)); + } } - auto index_id = index.index_id(); + } else { for (int64_t segment_index = 0; segment_index < num_segments; ++segment_index) { - auto get_segment_index_file_size_url = fmt::format( - "{}?method={}&tablet_id={}&rowset_id={}&segment_index={}&segment_index_id={" - "}", - binlog_api_url, "get_segment_index_file", request.remote_tablet_id, - remote_rowset_id, segment_index, index_id); - uint64_t segment_index_file_size; - auto get_segment_index_file_size_cb = [&get_segment_index_file_size_url, - &segment_index_file_size](HttpClient* client) { - RETURN_IF_ERROR(client->init(get_segment_index_file_size_url)); - client->set_timeout_ms(kMaxTimeoutMs); - RETURN_IF_ERROR(client->head()); - return client->get_content_length(&segment_index_file_size); - }; - auto index_file = InvertedIndexDescriptor::inverted_index_file_path( - local_tablet->tablet_path(), rowset_meta->rowset_id(), segment_index, index_id, - index.get_index_suffix()); - segment_index_file_names.push_back(index_file); - - status = HttpClient::execute_with_retry(max_retry, 1, get_segment_index_file_size_cb); - if (!status.ok()) { - LOG(WARNING) << "failed to get segment file size from " - << get_segment_index_file_size_url - << ", status=" << status.to_string(); - status.to_thrift(&tstatus); - return; - } + if (tablet_schema->has_inverted_index()) { + auto get_segment_index_file_size_url = fmt::format( + "{}?method={}&tablet_id={}&rowset_id={}&segment_index={}&segment_index_id={" + "}", + binlog_api_url, "get_segment_index_file", request.remote_tablet_id, + remote_rowset_id, segment_index, -1); + uint64_t segment_index_file_size; + auto get_segment_index_file_size_cb = + [&get_segment_index_file_size_url, + &segment_index_file_size](HttpClient* client) { + RETURN_IF_ERROR(client->init(get_segment_index_file_size_url)); + client->set_timeout_ms(kMaxTimeoutMs); + RETURN_IF_ERROR(client->head()); + return client->get_content_length(&segment_index_file_size); + }; + auto local_segment_path = BetaRowset::segment_file_path( + local_tablet->tablet_path(), rowset_meta->rowset_id(), segment_index); + auto index_file = InvertedIndexDescriptor::get_index_file_name(local_segment_path); + segment_index_file_names.push_back(index_file); + + status = HttpClient::execute_with_retry(max_retry, 1, + get_segment_index_file_size_cb); + if (!status.ok()) { + LOG(WARNING) << "failed to get segment file size from " + << get_segment_index_file_size_url + << ", status=" << status.to_string(); + status.to_thrift(&tstatus); + return; + } - segment_index_file_sizes.push_back(segment_index_file_size); - segment_index_file_urls.push_back(std::move(get_segment_index_file_size_url)); + segment_index_file_sizes.push_back(segment_index_file_size); + segment_index_file_urls.push_back(std::move(get_segment_index_file_size_url)); + } } } diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Syncer.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Syncer.groovy index 47000ab74fd92d..874a0e5c0be52e 100644 --- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Syncer.groovy +++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Syncer.groovy @@ -427,7 +427,9 @@ class Syncer { Boolean checkRestoreFinish() { String checkSQL = "SHOW RESTORE FROM TEST_" + context.db - List row = suite.sql(checkSQL)[0] + int size = suite.sql(checkSQL).size() + logger.info("Now size is ${size}") + List row = suite.sql(checkSQL)[size-1] logger.info("Now row is ${row}") return (row[4] as String) == "FINISHED" @@ -645,9 +647,9 @@ class Syncer { // step 2: get partitionIds metaMap.values().forEach { - baseSql += "/" + it.id.toString() + "/partitions" + def partitionSql = baseSql + "/" + it.id.toString() + "/partitions" Map partitionInfo = Maps.newHashMap() - sqlInfo = sendSql.call(baseSql, toSrc) + sqlInfo = sendSql.call(partitionSql, toSrc) for (List row : sqlInfo) { partitionInfo.put(row[0] as Long, row[2] as Long) } @@ -660,7 +662,7 @@ class Syncer { for (Entry info : partitionInfo) { // step 3.1: get partition/indexId - String partitionSQl = baseSql + "/" + info.key.toString() + String partitionSQl = partitionSql + "/" + info.key.toString() sqlInfo = sendSql.call(partitionSQl, toSrc) if (sqlInfo.isEmpty()) { logger.error("Target cluster partition-${info.key} indexId fault.") diff --git a/regression-test/suites/ccr_syncer_p0/inverted_index/test_binlog_config_change.groovy b/regression-test/suites/ccr_syncer_p0/inverted_index/test_binlog_config_change.groovy new file mode 100644 index 00000000000000..cccf25780d7caa --- /dev/null +++ b/regression-test/suites/ccr_syncer_p0/inverted_index/test_binlog_config_change.groovy @@ -0,0 +1,217 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_binlog_config_change_index") { + + def syncer = getSyncer() + if (!syncer.checkEnableFeatureBinlog()) { + logger.info("fe enable_feature_binlog is false, skip case test_binlog_config_change_index") + return + } + def insert_data = { tableName -> + [ + """INSERT INTO ${tableName} VALUES (1, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (1, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (2, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (2, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (3, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (3, "bason", "bason hate pear", 99);""" + ] + } + + def sqls = { tableName -> + [ + """ select * from ${tableName} order by id, name, hobbies, score """, + """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """, + """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """, + """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + ] + } + + def run_sql = { tableName -> + sqls(tableName).each { sqlStatement -> + def target_res = target_sql sqlStatement + def res = sql sqlStatement + assertEquals(res, target_res) + } + } + + def create_table_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1"); + """ + } + + def create_table_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1", "inverted_index_storage_format" = "V2"); + """ + } + + def create_table_mow_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true"); + """ + } + + def create_table_mow_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true", + "inverted_index_storage_format" = "V2"); + """ + } + + def run_test = {create_table, tableName -> + sql "DROP TABLE IF EXISTS ${tableName}" + sql create_table + sql """ALTER TABLE ${tableName} set ("binlog.enable" = "true")""" + + target_sql "DROP TABLE IF EXISTS ${tableName}" + target_sql create_table + + assertTrue(syncer.getTargetMeta("${tableName}")) + + // test 1: target cluster follow source cluster + logger.info("=== Test 1: Target cluster follow source cluster case ===") + + insert_data(tableName).each { sqlStatement -> + sql sqlStatement + assertTrue(syncer.getBinlog("${tableName}")) + assertTrue(syncer.beginTxn("${tableName}")) + assertTrue(syncer.getBackendClients()) + assertTrue(syncer.ingestBinlog()) + assertTrue(syncer.commitTxn()) + assertTrue(syncer.checkTargetVersion()) + syncer.closeBackendClients() + } + + target_sql " sync " + def res = target_sql """SELECT * FROM ${tableName}""" + if (tableName.contains("mow")) { + assertEquals(res.size(), insert_data(tableName).size() / 2 as Integer) + } else { + assertEquals(res.size(), insert_data(tableName).size()) + } + run_sql(tableName) + } + + + // inverted index format v1 + logger.info("=== Test 1: Inverted index format v1 case ===") + def tableName = "tbl_binlog_config_change_index_v1" + run_test.call(create_table_v1(tableName), tableName) + + // inverted index format v2 + logger.info("=== Test 2: Inverted index format v2 case ===") + tableName = "tbl_binlog_config_change_index_v2" + run_test.call(create_table_v2(tableName), tableName) + + // inverted index format v1 with mow + logger.info("=== Test 3: Inverted index format v1 with mow case ===") + tableName = "tbl_binlog_config_change_index_mow_v1" + run_test.call(create_table_mow_v1(tableName), tableName) + + // inverted index format v2 with mow + logger.info("=== Test 4: Inverted index format v2 with mow case ===") + tableName = "tbl_binlog_config_change_index_mow_v2" + run_test.call(create_table_mow_v2(tableName), tableName) + + // TODO: bugfix + // test 2: source cluster disable and re-enable binlog + // target_sql "DROP TABLE IF EXISTS ${tableName}" + // target_sql """ + // CREATE TABLE if NOT EXISTS ${tableName} + // ( + // `test` INT, + // `id` INT + // ) + // ENGINE=OLAP + // UNIQUE KEY(`test`, `id`) + // DISTRIBUTED BY HASH(id) BUCKETS 1 + // PROPERTIES ( + // "replication_allocation" = "tag.location.default: 1" + // ) + // """ + // sql """ALTER TABLE ${tableName} set ("binlog.enable" = "false")""" + // sql """ALTER TABLE ${tableName} set ("binlog.enable" = "true")""" + + // syncer.context.seq = -1 + + // assertTrue(syncer.getBinlog("${tableName}")) + // assertTrue(syncer.beginTxn("${tableName}")) + // assertTrue(syncer.ingestBinlog()) + // assertTrue(syncer.commitTxn()) + // assertTrue(syncer.checkTargetVersion()) + + // res = target_sql """SELECT * FROM ${tableName} WHERE test=${test_num}""" + // assertTrue(res.size() == insert_num) + +} \ No newline at end of file diff --git a/regression-test/suites/ccr_syncer_p0/inverted_index/test_get_binlog.groovy b/regression-test/suites/ccr_syncer_p0/inverted_index/test_get_binlog.groovy new file mode 100644 index 00000000000000..b837f799e5884f --- /dev/null +++ b/regression-test/suites/ccr_syncer_p0/inverted_index/test_get_binlog.groovy @@ -0,0 +1,239 @@ +// 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_get_binlog_case_index") { + def syncer = getSyncer() + if (!syncer.checkEnableFeatureBinlog()) { + logger.info("fe enable_feature_binlog is false, skip case test_get_binlog_case_index") + return + } + + def insert_data = { tableName -> + [ + """INSERT INTO ${tableName} VALUES (1, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (2, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (2, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (3, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (3, "bason", "bason hate pear", 99);""" + ] + } + + def sqls = { tableName -> + [ + """ select * from ${tableName} order by id, name, hobbies, score """, + """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """, + """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """, + """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + ] + } + + def run_sql = { tableName -> + sqls(tableName).each { sqlStatement -> + def target_res = target_sql sqlStatement + def res = sql sqlStatement + assertEquals(res, target_res) + } + } + + def create_table_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1"); + """ + } + + def create_table_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1", "inverted_index_storage_format" = "V2"); + """ + } + + def create_table_mow_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true"); + """ + } + + def create_table_mow_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true", + "inverted_index_storage_format" = "V2"); + """ + } + + def run_test = { create_table, tableName -> + long seq = -1 + sql "DROP TABLE IF EXISTS ${tableName}" + sql create_table + sql """ALTER TABLE ${tableName} set ("binlog.enable" = "true")""" + sql """ INSERT INTO ${tableName} VALUES (1, "andy", "andy love apple", 100); """ + assertTrue(syncer.getBinlog("${tableName}")) + long firstSeq = syncer.context.seq + + logger.info("=== Test 1: normal case ===") + insert_data(tableName).each { sqlStatement -> + sql sqlStatement + assertTrue(syncer.getBinlog("${tableName}")) + } + + long endSeq = syncer.context.seq + + logger.info("=== Test 2: Abnormal seq case ===") + logger.info("=== Test 2.1: too old seq case ===") + syncer.context.seq = -1 + assertTrue(syncer.context.seq == -1) + assertTrue(syncer.getBinlog("${tableName}")) + assertTrue(syncer.context.seq == firstSeq) + + + logger.info("=== Test 2.2: too new seq case ===") + syncer.context.seq = endSeq + 100 + assertTrue((syncer.getBinlog("${tableName}")) == false) + + + logger.info("=== Test 2.3: not find table case ===") + assertTrue(syncer.getBinlog("this_is_an_invalid_tbl") == false) + + + logger.info("=== Test 2.4: seq between first and end case ===") + long midSeq = (firstSeq + endSeq) / 2 + syncer.context.seq = midSeq + assertTrue(syncer.getBinlog("${tableName}")) + long test5Seq = syncer.context.seq + assertTrue(firstSeq <= test5Seq && test5Seq <= endSeq) + + logger.info("=== Test 3: Get binlog with different priv user case ===") + logger.info("=== Test 3.1: read only user get binlog case ===") + // TODO: bugfix + // syncer.context.seq = -1 + // readOnlyUser = "read_only_user" + // sql """DROP USER IF EXISTS ${readOnlyUser}""" + // sql """CREATE USER ${readOnlyUser} IDENTIFIED BY '123456'""" + // sql """GRANT ALL ON ${context.config.defaultDb}.* TO ${readOnlyUser}""" + // sql """GRANT SELECT_PRIV ON TEST_${context.dbName}.${tableName} TO ${readOnlyUser}""" + // syncer.context.user = "${readOnlyUser}" + // syncer.context.passwd = "123456" + // assertTrue(syncer.getBinlog("${tableName}")) + + } + + // inverted index format v1 + logger.info("=== Test 1: Inverted index format v1 case ===") + def tableName = "tbl_get_binlog_case_index_v1" + run_test.call(create_table_v1(tableName), tableName) + + // inverted index format v2 + logger.info("=== Test 2: Inverted index format v2 case ===") + tableName = "tbl_get_binlog_case_index_v2" + run_test.call(create_table_v2(tableName), tableName) + + // inverted index format v1 with mow + logger.info("=== Test 3: Inverted index format v1 with mow case ===") + tableName = "tbl_get_binlog_case_index_mow_v1" + run_test.call(create_table_mow_v1(tableName), tableName) + + // inverted index format v2 with mow + logger.info("=== Test 4: Inverted index format v2 with mow case ===") + tableName = "tbl_get_binlog_case_index_mow_v2" + run_test.call(create_table_mow_v2(tableName), tableName) + + logger.info("=== Test 3: no priv user get binlog case ===") + syncer.context.seq = -1 + def noPrivUser = "no_priv_user2" + def emptyTable = "tbl_empty_test" + sql "DROP TABLE IF EXISTS ${emptyTable}" + sql """ + CREATE TABLE ${emptyTable} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1"); + """ + sql """CREATE USER IF NOT EXISTS ${noPrivUser} IDENTIFIED BY '123456'""" + sql """GRANT ALL ON ${context.config.defaultDb}.* TO ${noPrivUser}""" + sql """GRANT ALL ON TEST_${context.dbName}.${emptyTable} TO ${noPrivUser}""" + syncer.context.user = "${noPrivUser}" + syncer.context.passwd = "123456" + assertTrue((syncer.getBinlog("${tableName}")) == false) + + + logger.info("=== Test 3.3: Non-existent user set in syncer get binlog case ===") + syncer.context.user = "this_is_an_invalid_user" + syncer.context.passwd = "this_is_an_invalid_user" + assertTrue(syncer.getBinlog("${tableName}", false) == false) +} \ No newline at end of file diff --git a/regression-test/suites/ccr_syncer_p0/inverted_index/test_ingest_binlog.groovy b/regression-test/suites/ccr_syncer_p0/inverted_index/test_ingest_binlog.groovy new file mode 100644 index 00000000000000..12ba49e084dae9 --- /dev/null +++ b/regression-test/suites/ccr_syncer_p0/inverted_index/test_ingest_binlog.groovy @@ -0,0 +1,223 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_ingest_binlog_index") { + + def syncer = getSyncer() + if (!syncer.checkEnableFeatureBinlog()) { + logger.info("fe enable_feature_binlog is false, skip case test_ingest_binlog_index") + return + } + + def insert_data = { tableName -> + [ + """INSERT INTO ${tableName} VALUES (1, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (1, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (2, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (2, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (3, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (3, "bason", "bason hate pear", 99);""" + ] + } + + def sqls = { tableName -> + [ + """ select * from ${tableName} order by id, name, hobbies, score """, + """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """, + """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """, + """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + ] + } + + def run_sql = { tableName -> + sqls(tableName).each { sqlStatement -> + def target_res = target_sql sqlStatement + def res = sql sqlStatement + assertEquals(res, target_res) + } + } + + def create_table_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1"); + """ + } + + def create_table_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( "replication_num" = "1", "inverted_index_storage_format" = "V2"); + """ + } + + def create_table_mow_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true"); + """ + } + + def create_table_mow_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true", + "inverted_index_storage_format" = "V2"); + """ + } + + def run_test = { create_table, tableName -> + sql "DROP TABLE IF EXISTS ${tableName}" + sql create_table + sql """ALTER TABLE ${tableName} set ("binlog.enable" = "true")""" + + target_sql "DROP TABLE IF EXISTS ${tableName}" + target_sql create_table + assertTrue(syncer.getTargetMeta("${tableName}")) + + logger.info("=== Test 1: Common ingest binlog case ===") + insert_data.call(tableName).each { sqlStatement -> + sql sqlStatement + assertTrue(syncer.getBinlog("${tableName}")) + assertTrue(syncer.beginTxn("${tableName}")) + assertTrue(syncer.getBackendClients()) + assertTrue(syncer.ingestBinlog()) + assertTrue(syncer.commitTxn()) + assertTrue(syncer.checkTargetVersion()) + syncer.closeBackendClients() + } + + target_sql " sync " + res = target_sql """SELECT * FROM ${tableName}""" + if (tableName.contains("mow")) { + assertEquals(res.size(), insert_data(tableName).size() / 2 as Integer) + } else { + assertEquals(res.size(), insert_data(tableName).size()) + } + run_sql.call(tableName) + + logger.info("=== Test 2: Wrong IngestBinlogRequest case ===") + sql """INSERT INTO ${tableName} VALUES (4, "bason", "bason hate pear", 99);""" + assertTrue(syncer.getBinlog("${tableName}")) + assertTrue(syncer.beginTxn("${tableName}")) + assertTrue(syncer.getBackendClients()) + + + logger.info("=== Test 2.1: Wrong txnId case ===") + // TODO: bugfix + // def originTxnId = syncer.context.txnId + // syncer.context.txnId = -1 + // assertTrue(syncer.ingestBinlog() == false) + // syncer.context.txnId = originTxnId + + + logger.info("=== Test 2.2: Wrong binlog version case ===") + // -1 means use the number of syncer.context + // Boolean ingestBinlog(long fakePartitionId = -1, long fakeVersion = -1) + // use fakeVersion = 1, 1 is doris be talet first version, so no binlog, only http error + assertTrue(syncer.ingestBinlog(-1, 1) == false) + + + logger.info("=== Test 2.3: Wrong partitionId case ===") + // TODO: bugfix + // assertTrue(syncer.ingestBinlog(1, -1) == false) + + + logger.info("=== Test 2.4: Right case ===") + assertTrue(syncer.ingestBinlog()) + assertTrue(syncer.commitTxn()) + assertTrue(syncer.checkTargetVersion()) + target_sql " sync " + res = target_sql """SELECT * FROM ${tableName} WHERE id=4""" + assertEquals(res.size(), 1) + + + // End Test 2 + syncer.closeBackendClients() + } + + // inverted index format v1 + logger.info("=== Test 1: Inverted index format v1 case ===") + def tableName = "tbl_ingest_binlog_index_v1" + run_test.call(create_table_v1(tableName), tableName) + + // inverted index format v2 + logger.info("=== Test 2: Inverted index format v2 case ===") + tableName = "tbl_ingest_binlog_index_v2" + run_test.call(create_table_v2(tableName), tableName) + + // inverted index format v1 with mow + logger.info("=== Test 3: Inverted index format v1 with mow case ===") + tableName = "tbl_ingest_binlog_index_mow_v1" + run_test.call(create_table_mow_v1(tableName), tableName) + + // inverted index format v2 with mow + logger.info("=== Test 4: Inverted index format v2 with mow case ===") + tableName = "tbl_ingest_binlog_index_mow_v2" + run_test.call(create_table_mow_v2(tableName), tableName) + +} diff --git a/regression-test/suites/ccr_syncer_p0/inverted_index/test_multi_buckets.groovy b/regression-test/suites/ccr_syncer_p0/inverted_index/test_multi_buckets.groovy new file mode 100644 index 00000000000000..4a7b7263af281f --- /dev/null +++ b/regression-test/suites/ccr_syncer_p0/inverted_index/test_multi_buckets.groovy @@ -0,0 +1,180 @@ +// 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_multi_buckets_index") { + + def syncer = getSyncer() + if (!syncer.checkEnableFeatureBinlog()) { + logger.info("fe enable_feature_binlog is false, skip case test_multi_buckets_index") + return + } + def insert_data = { tableName -> + [ + """INSERT INTO ${tableName} VALUES (1, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (1, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (2, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (2, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (3, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (3, "bason", "bason hate pear", 99);""" + ] + } + + def sqls = { tableName -> + [ + """ select * from ${tableName} order by id, name, hobbies, score """, + """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """, + """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """, + """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + ] + } + + def run_sql = { tableName -> + sqls(tableName).each { sqlStatement -> + def target_res = target_sql sqlStatement + def res = sql sqlStatement + assertEquals(res, target_res) + } + } + + def create_table_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( "replication_num" = "1"); + """ + } + + def create_table_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( "replication_num" = "1", "inverted_index_storage_format" = "V2"); + """ + } + + def create_table_mow_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true"); + """ + } + + def create_table_mow_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true", + "inverted_index_storage_format" = "V2"); + """ + } + + def run_test = { create_table, tableName -> + sql "DROP TABLE IF EXISTS ${tableName}" + sql create_table + sql """ALTER TABLE ${tableName} set ("binlog.enable" = "true")""" + + target_sql "DROP TABLE IF EXISTS ${tableName}" + target_sql create_table + assertTrue(syncer.getTargetMeta("${tableName}")) + + insert_data(tableName).each { sqlStatement -> + sql sqlStatement + assertTrue(syncer.getBinlog("${tableName}")) + assertTrue(syncer.beginTxn("${tableName}")) + assertTrue(syncer.getBackendClients()) + assertTrue(syncer.ingestBinlog()) + assertTrue(syncer.commitTxn()) + assertTrue(syncer.checkTargetVersion()) + syncer.closeBackendClients() + } + + target_sql " sync " + def res = target_sql """SELECT * FROM ${tableName}""" + if (tableName.contains("mow")) { + assertEquals(res.size(), insert_data(tableName).size() / 2 as Integer) + } else { + assertEquals(res.size(), insert_data(tableName).size()) + } + run_sql(tableName) + } + + // inverted index format v1 + logger.info("=== Test 1: Inverted index format v1 case ===") + def tableName = "tbl_multi_buckets_index_v1" + run_test.call(create_table_v1(tableName), tableName) + // inverted index format v2 + logger.info("=== Test 2: Inverted index format v2 case ===") + tableName = "tbl_multi_buckets_index_v2" + run_test.call(create_table_v2(tableName), tableName) + + // inverted index format v1 with mow + logger.info("=== Test 3: Inverted index format v1 with mow case ===") + tableName = "tbl_multi_buckets_index_mow_v1" + run_test.call(create_table_mow_v1(tableName), tableName) + + // inverted index format v2 with mow + logger.info("=== Test 4: Inverted index format v2 with mow case ===") + tableName = "tbl_multi_buckets_index_mow_v2" + run_test.call(create_table_mow_v2(tableName), tableName) +} diff --git a/regression-test/suites/ccr_syncer_p1/inverted_index/test_backup_restore.groovy b/regression-test/suites/ccr_syncer_p1/inverted_index/test_backup_restore.groovy new file mode 100644 index 00000000000000..86f4cc89913b3f --- /dev/null +++ b/regression-test/suites/ccr_syncer_p1/inverted_index/test_backup_restore.groovy @@ -0,0 +1,196 @@ +// 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_backup_restore_index") { + + def syncer = getSyncer() + if (!syncer.checkEnableFeatureBinlog()) { + logger.info("fe enable_feature_binlog is false, skip case test_backup_restore") + return + } + + def insert_data = { tableName -> + [ + """INSERT INTO ${tableName} VALUES (1, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (1, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (2, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (2, "bason", "bason hate pear", 99);""", + """INSERT INTO ${tableName} VALUES (3, "andy", "andy love apple", 100);""", + """INSERT INTO ${tableName} VALUES (3, "bason", "bason hate pear", 99);""" + ] + } + + def sqls = { tableName -> + [ + """ select * from ${tableName} order by id, name, hobbies, score """, + """ select * from ${tableName} where name match "andy" order by id, name, hobbies, score """, + """ select * from ${tableName} where hobbies match "pear" order by id, name, hobbies, score """, + """ select * from ${tableName} where score < 100 order by id, name, hobbies, score """ + ] + } + + def run_sql = { tableName -> + sqls(tableName).each { sqlStatement -> + def target_res = target_sql sqlStatement + def res = sql sqlStatement + assertEquals(res, target_res) + } + } + + def create_table_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true" + ); + """ + } + + def create_table_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "inverted_index_storage_format" = "V2" + ); + """ + } + + def create_table_mow_v1 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true"); + """ + } + + def create_table_mow_v2 = { tableName -> + """ + CREATE TABLE ${tableName} ( + `id` int(11) NULL, + `name` varchar(255) NULL, + `hobbies` text NULL, + `score` int(11) NULL, + index index_name (name) using inverted, + index index_hobbies (hobbies) using inverted properties("parser"="english"), + index index_score (score) using inverted + ) ENGINE=OLAP + UNIQUE KEY(`id`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`id`) BUCKETS 1 + PROPERTIES ( + "replication_num" = "1", + "binlog.enable" = "true", + "enable_unique_key_merge_on_write" = "true", + "inverted_index_storage_format" = "V2"); + """ + } + + def run_test = { create_table, tableName -> + sql "DROP TABLE IF EXISTS ${tableName}" + sql create_table + + logger.info("=== Test 1: Common backup and restore ===") + def snapshotName = "snapshot_test_" + tableName + insert_data(tableName).each { sqlStatement -> + sql sqlStatement + } + sql " sync " + def res = sql "SELECT * FROM ${tableName}" + if (tableName.contains("mow")) { + assertEquals(res.size(), insert_data(tableName).size() / 2 as Integer) + } else { + assertEquals(res.size(), insert_data(tableName).size()) + } + + sql """ + BACKUP SNAPSHOT ${context.dbName}.${snapshotName} + TO `__keep_on_local__` + ON (${tableName}) + PROPERTIES ("type" = "full") + """ + syncer.waitSnapshotFinish() + assertTrue(syncer.getSnapshot("${snapshotName}", "${tableName}")) + assertTrue(syncer.restoreSnapshot(true)) + syncer.waitTargetRestoreFinish() + target_sql " sync " + res = target_sql "SELECT * FROM ${tableName}" + if (tableName.contains("mow")) { + assertEquals(res.size(), insert_data(tableName).size() / 2 as Integer) + } else { + assertEquals(res.size(), insert_data(tableName).size()) + } + run_sql(tableName) + } + + // inverted index format v1 + logger.info("=== Test 1: Inverted index format v1 case ===") + def tableName = "tbl_backup_restore_index_v1" + run_test.call(create_table_v1(tableName), tableName) + + // inverted index format v2 + logger.info("=== Test 2: Inverted index format v2 case ===") + tableName = "tbl_backup_restore_index_v2" + run_test.call(create_table_v2(tableName), tableName) + + // inverted index format v1 with mow + logger.info("=== Test 3: Inverted index format v1 with mow case ===") + tableName = "tbl_backup_restore_index_mow_v1" + run_test.call(create_table_mow_v1(tableName), tableName) + + // inverted index format v2 with mow + logger.info("=== Test 4: Inverted index format v2 with mow case ===") + tableName = "tbl_backup_restore_index_mow_v2" + run_test.call(create_table_mow_v2(tableName), tableName) +} \ No newline at end of file From b778648a28c5857a2dc565c4d1061e7fc118608c Mon Sep 17 00:00:00 2001 From: AlexYue Date: Mon, 15 Apr 2024 09:45:33 +0800 Subject: [PATCH 61/71] [feature](Cloud) Forbid alter table's storage vault related properties (#33620) --- .../apache/doris/analysis/ModifyTablePropertiesClause.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java index 8a3543d3d36f8f..c5a4036f81f123 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java @@ -302,6 +302,10 @@ public void analyze(Analyzer analyzer) throws AnalysisException { } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_FILE_CACHE_TTL_SECONDS)) { this.needTableStable = false; this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC; + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_VAULT_NAME)) { + throw new AnalysisException("You can not modify storage vault name"); + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_VAULT_ID)) { + throw new AnalysisException("You can not modify storage vault id"); } else { throw new AnalysisException("Unknown table property: " + properties.keySet()); } From 9fbe6f5294ed32ea5b6f11129209ddadce5a2aeb Mon Sep 17 00:00:00 2001 From: Kaijie Chen Date: Mon, 15 Apr 2024 10:14:46 +0800 Subject: [PATCH 62/71] [improve](move-memtable) add more info in LoadStreamStub errors (#33618) --- be/src/vec/sink/load_stream_stub.cpp | 24 +++++++++++++----------- be/src/vec/sink/load_stream_stub.h | 2 ++ 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/be/src/vec/sink/load_stream_stub.cpp b/be/src/vec/sink/load_stream_stub.cpp index 6d661a4c88e0b0..78e1bc691ccb9f 100644 --- a/be/src/vec/sink/load_stream_stub.cpp +++ b/be/src/vec/sink/load_stream_stub.cpp @@ -293,8 +293,7 @@ Status LoadStreamStub::wait_for_schema(int64_t partition_id, int64_t index_id, i Status LoadStreamStub::close_wait(RuntimeState* state, int64_t timeout_ms) { DBUG_EXECUTE_IF("LoadStreamStub::close_wait.long_wait", DBUG_BLOCK); if (!_is_init.load()) { - return Status::InternalError("stream {} is not opened, load_id={}", _stream_id, - print_id(_load_id)); + return Status::InternalError("stream {} is not opened, {}", _stream_id, to_string()); } if (_is_closed.load()) { return _check_cancel(); @@ -310,17 +309,13 @@ Status LoadStreamStub::close_wait(RuntimeState* state, int64_t timeout_ms) { << ", is_cancelled=" << state->get_query_ctx()->is_cancelled(); int ret = _close_cv.wait_for(lock, 1000000); if (ret != 0 && timeout_sec <= 0) { - return Status::InternalError( - "stream close_wait timeout, error={}, timeout_ms={}, load_id={}, dst_id={}, " - "stream_id={}", - ret, timeout_ms, print_id(_load_id), _dst_id, _stream_id); + return Status::InternalError("stream close_wait timeout, error={}, timeout_ms={}, {}", + ret, timeout_ms, to_string()); } } RETURN_IF_ERROR(_check_cancel()); if (!_is_eos.load()) { - return Status::InternalError( - "stream closed without eos, load_id={}, dst_id={}, stream_id={}", - print_id(_load_id), _dst_id, _stream_id); + return Status::InternalError("stream closed without eos, {}", to_string()); } return Status::OK(); } @@ -391,16 +386,23 @@ Status LoadStreamStub::_send_with_retry(butil::IOBuf& buf) { const timespec time = butil::seconds_from_now(config::load_stream_eagain_wait_seconds); int wait_ret = brpc::StreamWait(_stream_id, &time); if (wait_ret != 0) { - return Status::InternalError("StreamWait failed, err={}", wait_ret); + return Status::InternalError("StreamWait failed, err={}, {}", wait_ret, + to_string()); } break; } default: - return Status::InternalError("StreamWrite failed, err={}", ret); + return Status::InternalError("StreamWrite failed, err={}, {}", ret, to_string()); } } } +std::string LoadStreamStub::to_string() { + std::ostringstream ss; + ss << *this; + return ss.str(); +} + inline std::ostream& operator<<(std::ostream& ostr, const LoadStreamStub& stub) { ostr << "LoadStreamStub load_id=" << print_id(stub._load_id) << ", src_id=" << stub._src_id << ", dst_id=" << stub._dst_id << ", stream_id=" << stub._stream_id; diff --git a/be/src/vec/sink/load_stream_stub.h b/be/src/vec/sink/load_stream_stub.h index aa8b850760e793..5ebec9f9d78f88 100644 --- a/be/src/vec/sink/load_stream_stub.h +++ b/be/src/vec/sink/load_stream_stub.h @@ -198,6 +198,8 @@ class LoadStreamStub { friend std::ostream& operator<<(std::ostream& ostr, const LoadStreamStub& stub); + std::string to_string(); + private: Status _encode_and_send(PStreamHeader& header, std::span data = {}); Status _send_with_buffer(butil::IOBuf& buf, bool sync = false); From 96eb7ec9a21a792a2d091c73dacf13d2c6f611ea Mon Sep 17 00:00:00 2001 From: jakevin Date: Mon, 15 Apr 2024 10:18:05 +0800 Subject: [PATCH 63/71] [feature](Nereids): date literal suppose Zone (#33534) support ``` '2022-05-01 01:02:55+02:30 '2022-05-01 01:02:55Asia/Shanghai ``` --- .../expressions/literal/DateLiteral.java | 13 ------ .../nereids/util/DateTimeFormatterUtils.java | 1 - .../doris/nereids/util/ExpressionUtils.java | 33 --------------- .../literal/DateTimeLiteralTest.java | 41 ++++++++++--------- 4 files changed, 22 insertions(+), 66 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java index 89910b74374f38..93127933ea7160 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/literal/DateLiteral.java @@ -246,19 +246,6 @@ static String normalize(String s) { sb.append(s.substring(i)); - // Zone Part - // while(i < s.length()) { - // - // } - - // add missing :00 in Zone part - // int len = sb.length(); - // int signIdx = sb.indexOf("+", 10); // from index:10, skip date part (it contains '-') - // signIdx = signIdx == -1 ? sb.indexOf("-", 10) : signIdx; - // if (signIdx != -1 && len - signIdx == 3) { - // sb.append(":00"); - // } - return sb.toString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateTimeFormatterUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateTimeFormatterUtils.java index 50d0f7169a5aab..6b52fb24a9346a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateTimeFormatterUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/DateTimeFormatterUtils.java @@ -41,7 +41,6 @@ public class DateTimeFormatterUtils { public static final DateTimeFormatter ZONE_FORMATTER = new DateTimeFormatterBuilder() .optionalStart() - // .appendZoneText(TextStyle.FULL) .appendZoneOrOffsetId() .optionalEnd() .toFormatter() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 401b8e0736bb14..e5e79f7b725d70 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -427,32 +427,6 @@ public Expression visit(Expression expr, Map replaceMap; - // if the key of replaceMap is named expr and withAlias is true, we should - // add alias after replaced - private final boolean withAliasIfKeyNamed; - - private ExpressionReplacerContext(Map replaceMap, - boolean withAliasIfKeyNamed) { - this.replaceMap = replaceMap; - this.withAliasIfKeyNamed = withAliasIfKeyNamed; - } - - public static ExpressionReplacerContext of(Map replaceMap, - boolean withAliasIfKeyNamed) { - return new ExpressionReplacerContext(replaceMap, withAliasIfKeyNamed); - } - - public Map getReplaceMap() { - return replaceMap; - } - - public boolean isWithAliasIfKeyNamed() { - return withAliasIfKeyNamed; - } - } - /** * merge arguments into an expression array * @@ -833,13 +807,6 @@ public static Set getInputSlotSet(Collection exprs) return set; } - public static boolean checkTypeSkipCast(Expression expression, Class cls) { - while (expression instanceof Cast) { - expression = ((Cast) expression).child(); - } - return cls.isInstance(expression); - } - public static Expression getExpressionCoveredByCast(Expression expression) { while (expression instanceof Cast) { expression = ((Cast) expression).child(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteralTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteralTest.java index 8555d44d634343..ea930e2b73ddb6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteralTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/literal/DateTimeLiteralTest.java @@ -137,25 +137,7 @@ void testZone() { new DateTimeV2Literal("2022-08-01 01:01:01Z"); new DateTimeV2Literal("2022-08-01 01:01:01Europe/Berlin"); new DateTimeV2Literal("2022-08-01 01:01:01Europe/London"); - } - - @Test - @Disabled("Test results can change over time") - void testZoneOrOffsetRight() { - java.util.function.BiConsumer assertHour = (dateTimeV2Literal, expectHour) -> { - Assertions.assertEquals(dateTimeV2Literal.hour, expectHour); - }; - DateTimeV2Literal dateTimeV2Literal; - dateTimeV2Literal = new DateTimeV2Literal("2022-08-01 00:00:00Europe/London"); // +01:00 - assertHour.accept(dateTimeV2Literal, 7L); - dateTimeV2Literal = new DateTimeV2Literal("2022-08-01 00:00:00America/New_York"); // -04:00 - assertHour.accept(dateTimeV2Literal, 12L); - dateTimeV2Literal = new DateTimeV2Literal("2022-08-01 00:00:00Asia/Shanghai"); - assertHour.accept(dateTimeV2Literal, 0L); - dateTimeV2Literal = new DateTimeV2Literal("2022-08-01 00:00:00+01:00"); - assertHour.accept(dateTimeV2Literal, 7L); - dateTimeV2Literal = new DateTimeV2Literal("2022-08-01 00:00:00-01:00"); - assertHour.accept(dateTimeV2Literal, 9L); + new DateTimeV2Literal("2022-08-01 00:00:00Asia/Shanghai"); } @Test @@ -238,6 +220,27 @@ void testDateTimeZone() { new DateTimeV2Literal("0001-01-01"); } + @Test + void testDateTimeZone1() { + Consumer assertFunc = (datetime) -> { + Assertions.assertEquals(2022, datetime.year); + Assertions.assertEquals(1, datetime.month); + Assertions.assertEquals(2, datetime.day); + Assertions.assertEquals(12, datetime.hour); + Assertions.assertEquals(0, datetime.minute); + Assertions.assertEquals(0, datetime.second); + }; + DateTimeV2Literal literal; + literal = new DateTimeV2Literal("2022-01-02 12:00:00UTC+08:00"); + assertFunc.accept(literal); + literal = new DateTimeV2Literal("2022-01-02 04:00:00UTC"); + assertFunc.accept(literal); + literal = new DateTimeV2Literal("2022-01-01 20:00:00UTC-08:00"); + assertFunc.accept(literal); + literal = new DateTimeV2Literal("2022-01-02 04:00:00Z"); + assertFunc.accept(literal); + } + @Test void testIrregularDateTime() { From 229b9b9d654cb632913a07e2da622448393d2032 Mon Sep 17 00:00:00 2001 From: zhangdong <493738387@qq.com> Date: Mon, 15 Apr 2024 10:47:23 +0800 Subject: [PATCH 64/71] [fix](auth)fix missing authentication (#33347) - Modified permission prompt information: added display of which permission items are required to perform the current operation, removed prompt information for "current user" and "IP" - `AdminCopyTablet` use `PrivPredicate.ADMIN` check auth, before is `PrivPredicate.OPERATOR` - `row policy` use `PrivPredicate.GRANT` - `storage policy` keep `PrivPredicate.ADMIN` unchanged - `create/alter view` need has `select_priv` on base table - `cancel alter system` need `PrivPredicate.OPERATOR` - `drop sync materialized view` need `alter_priv` before is `drop_priv` - `SetLdapPassVar` need `admin_priv` before only `root/admin` can do - `ShowCatalogRecycleBin` use `PrivPredicate.ADMIN` - `ShowCreateRepository` use `PrivPredicate.ADMIN` - `ShowDataRepository` use `PrivPredicate.ADMIN` - `ShowEncryptKey` use `PrivPredicate.ADMIN` - `ShowPlugins` use `PrivPredicate.ADMIN` - `ShowRepositories` use `PrivPredicate.ADMIN` - `ShowSnapshot` use `PrivPredicate.ADMIN` - `ShowTabletsBelong` use `PrivPredicate.ADMIN` - `ShowTranscation` use `PrivPredicate.ADMIN` - `show create db` use `show` instead of `PrivPredicate.ALTER_CREATE_DROP` - `cancel export`,if table size >1,use `db.PrivPredicate.SELECT`,else use `table.PrivPredicate.SELECT` - `show stream load`,`table.PrivPredicate.LOAD` - `show load`,if table size >1, use `db.PrivPredicate.LOAD`,else use `table.PrivPredicate.LOAD` --- .../doris/analysis/AdminCopyTabletStmt.java | 5 +- .../doris/analysis/AlterPolicyStmt.java | 6 +- .../apache/doris/analysis/AlterViewStmt.java | 7 +- .../apache/doris/analysis/BaseViewStmt.java | 27 +++++++ .../doris/analysis/CancelAlterSystemStmt.java | 9 +++ .../doris/analysis/CreatePolicyStmt.java | 16 +++- .../apache/doris/analysis/CreateViewStmt.java | 5 +- .../analysis/DropMaterializedViewStmt.java | 5 +- .../apache/doris/analysis/DropPolicyStmt.java | 16 +++- .../apache/doris/analysis/SetLdapPassVar.java | 12 +-- .../analysis/ShowCatalogRecycleBinStmt.java | 11 +++ .../doris/analysis/ShowCreateDbStmt.java | 10 +-- .../analysis/ShowCreateRepositoryStmt.java | 11 ++- .../apache/doris/analysis/ShowDataStmt.java | 7 +- .../doris/analysis/ShowEncryptKeysStmt.java | 12 +-- .../doris/analysis/ShowPluginsStmt.java | 13 +++- .../doris/analysis/ShowRepositoriesStmt.java | 15 ++++ .../doris/analysis/ShowSnapshotStmt.java | 11 +++ .../doris/analysis/ShowTabletsBelongStmt.java | 10 +++ .../doris/analysis/ShowTransactionStmt.java | 11 ++- .../org/apache/doris/common/ErrorCode.java | 12 ++- .../apache/doris/job/manager/JobManager.java | 56 +++++++++++++- .../java/org/apache/doris/load/ExportMgr.java | 30 +++++++- .../apache/doris/load/StreamLoadRecord.java | 8 ++ .../doris/load/StreamLoadRecordMgr.java | 10 +++ .../apache/doris/load/loadv2/LoadManager.java | 32 +++++++- .../doris/mysql/privilege/PrivBitSet.java | 9 ++- .../org/apache/doris/qe/ShowExecutor.java | 2 +- .../doris/analysis/CancelExportStmtTest.java | 28 ++++++- .../DropMaterializedViewStmtTest.java | 2 +- .../doris/catalog/RefreshTableTest.java | 2 +- .../doris/job/manager/JobManagerTest.java | 65 ++++++++++++++++ .../doris/load/loadv2/LoadManagerTest.java | 38 ++++++++++ .../org/apache/doris/planner/PlannerTest.java | 2 +- .../doris/utframe/TestWithFeService.java | 2 +- .../data/auth_p0/test_strict_mode.csv | 2 + .../test_admin_copy_tablet_auth.groovy | 35 +++++++++ .../auth_p0/test_alter_policy_auth.groovy | 37 +++++++++ .../auth_p0/test_alter_view_auth.groovy | 66 ++++++++++++++++ .../test_cancel_alter_system_auth.groovy | 35 +++++++++ .../auth_p0/test_create_policy_auth.groovy | 47 ++++++++++++ .../auth_p0/test_create_view_auth.groovy | 62 +++++++++++++++ .../test_drop_materialized_view_auth.groovy | 35 +++++++++ .../auth_p0/test_drop_policy_auth.groovy | 43 +++++++++++ .../test_set_ldap_admin_password_auth.groovy | 35 +++++++++ .../test_show_catalog_recycle_bin_auth.groovy | 35 +++++++++ .../test_show_create_database_auth.groovy | 35 +++++++++ .../test_show_create_repository_auth.groovy | 35 +++++++++ .../suites/auth_p0/test_show_data_auth.groovy | 35 +++++++++ .../auth_p0/test_show_encryptkeys_auth.groovy | 36 +++++++++ .../auth_p0/test_show_plugins_auth.groovy | 35 +++++++++ .../test_show_repositories_auth.groovy | 35 +++++++++ .../auth_p0/test_show_snapshot_auth.groovy | 35 +++++++++ .../auth_p0/test_show_stream_load_auth.groovy | 75 +++++++++++++++++++ .../test_show_tablets_belong_auth.groovy | 35 +++++++++ .../auth_p0/test_show_transaction_auth.groovy | 35 +++++++++ 56 files changed, 1284 insertions(+), 56 deletions(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/job/manager/JobManagerTest.java create mode 100644 regression-test/data/auth_p0/test_strict_mode.csv create mode 100644 regression-test/suites/auth_p0/test_admin_copy_tablet_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_alter_policy_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_alter_view_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_cancel_alter_system_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_create_policy_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_create_view_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_drop_materialized_view_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_drop_policy_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_set_ldap_admin_password_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_catalog_recycle_bin_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_create_database_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_create_repository_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_data_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_encryptkeys_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_plugins_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_repositories_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_snapshot_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_stream_load_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_tablets_belong_auth.groovy create mode 100644 regression-test/suites/auth_p0/test_show_transaction_auth.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCopyTabletStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCopyTabletStmt.java index 63832d9cd65d22..475219ca343c53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCopyTabletStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AdminCopyTabletStmt.java @@ -71,8 +71,9 @@ public long getExpirationMinutes() { @Override public void analyze(Analyzer analyzer) throws AnalysisException { - if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.OPERATOR)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "NODE"); + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); } if (properties == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterPolicyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterPolicyStmt.java index 91a5f143752c3b..c8128e2bcbdb68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterPolicyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterPolicyStmt.java @@ -53,8 +53,10 @@ public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); // check auth - if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + if (!Env.getCurrentEnv().getAccessManager() + .checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); } if (properties == null || properties.isEmpty()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java index 6e0da716b078bf..355c9723c8b220 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java @@ -62,9 +62,8 @@ public void analyze(Analyzer analyzer) throws UserException { if (!Env.getCurrentEnv().getAccessManager() .checkTblPriv(ConnectContext.get(), tableName.getCtl(), tableName.getDb(), tableName.getTbl(), PrivPredicate.ALTER)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "ALTER VIEW", - ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), - tableName.getDb() + ": " + tableName.getTbl()); + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_ACCESS_DENIED_ERROR, + PrivPredicate.ALTER.getPrivs().toString(), tableName.getTbl()); } if (cols != null) { @@ -74,7 +73,7 @@ public void analyze(Analyzer analyzer) throws UserException { viewDefStmt.setNeedToSql(true); Analyzer viewAnalyzer = new Analyzer(analyzer); viewDefStmt.analyze(viewAnalyzer); - + checkQueryAuth(); createColumnAndViewDefs(analyzer); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java index d8740f03f523b9..545d7c1c57a4dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BaseViewStmt.java @@ -18,15 +18,20 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; import org.apache.doris.common.util.ToSqlContext; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -72,6 +77,28 @@ public String getInlineViewDef() { return inlineViewDef; } + protected void checkQueryAuth() throws UserException { + for (int i = 0; i < viewDefStmt.getBaseTblResultExprs().size(); ++i) { + Expr expr = viewDefStmt.getBaseTblResultExprs().get(i); + if (!(expr instanceof SlotRef)) { + continue; + } + SlotRef slotRef = (SlotRef) expr; + TableName queryTableName = slotRef.getTableName(); + if (queryTableName == null) { + continue; + } + String queryColumnName = slotRef.getColumnName(); + String ctlName = StringUtils.isEmpty(queryTableName.getCtl()) ? InternalCatalog.INTERNAL_CATALOG_NAME + : queryTableName.getCtl(); + // check privilege + Env.getCurrentEnv().getAccessManager() + .checkColumnsPriv(ConnectContext.get().getCurrentUserIdentity(), ctlName, + queryTableName.getDb(), queryTableName.getTbl(), Sets.newHashSet(queryColumnName), + PrivPredicate.SELECT); + } + } + /** * Sets the originalViewDef and the expanded inlineViewDef based on viewDefStmt. * If columnNames were given, checks that they do not contain duplicate column names diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelAlterSystemStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelAlterSystemStmt.java index 9b547a4de53b77..e3a465d2d904ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelAlterSystemStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CancelAlterSystemStmt.java @@ -17,7 +17,12 @@ package org.apache.doris.analysis; +import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.system.SystemInfoService; import org.apache.doris.system.SystemInfoService.HostInfo; @@ -44,6 +49,10 @@ public CancelAlterSystemStmt(List params) { @Override public void analyze(Analyzer analyzer) throws AnalysisException { + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.OPERATOR)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.OPERATOR.getPrivs().toString()); + } for (String param : params) { if (!param.contains(":")) { ids.add(param); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreatePolicyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreatePolicyStmt.java index 4d8527c0f79245..8aedccb6e75ca2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreatePolicyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreatePolicyStmt.java @@ -101,6 +101,12 @@ public void analyze(Analyzer analyzer) throws UserException { throw new UserException("storage policy feature is disabled by default. " + "Enable it by setting 'enable_storage_policy=true' in fe.conf"); } + // check auth + if (!Env.getCurrentEnv().getAccessManager() + .checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } break; case ROW: default: @@ -112,10 +118,12 @@ public void analyze(Analyzer analyzer) throws UserException { user.getQualifiedUser(), user.getHost(), tableName.getTbl()); } } - } - // check auth - if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + // check auth + if (!Env.getCurrentEnv().getAccessManager() + .checkGlobalPriv(ConnectContext.get(), PrivPredicate.GRANT)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.GRANT.getPrivs().toString()); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateViewStmt.java index 8b53d18fd9b602..2029b464100daa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateViewStmt.java @@ -67,7 +67,8 @@ public void analyze(Analyzer analyzer) throws UserException { if (!Env.getCurrentEnv().getAccessManager() .checkTblPriv(ConnectContext.get(), tableName.getCtl(), tableName.getDb(), tableName.getTbl(), PrivPredicate.CREATE)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "CREATE"); + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_ACCESS_DENIED_ERROR, + PrivPredicate.CREATE.getPrivs().toString(), tableName.getTbl()); } // Do not rewrite nondeterministic functions to constant in create view's def stmt @@ -84,7 +85,7 @@ public void analyze(Analyzer analyzer) throws UserException { Analyzer viewAnalyzer = new Analyzer(analyzer); viewDefStmt.forbiddenMVRewrite(); viewDefStmt.analyze(viewAnalyzer); - + checkQueryAuth(); createColumnAndViewDefs(viewAnalyzer); } finally { // must reset this flag, otherwise, all following query statement in this connection diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropMaterializedViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropMaterializedViewStmt.java index 9fe01f20a066fc..377cdcf415240e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropMaterializedViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropMaterializedViewStmt.java @@ -73,8 +73,9 @@ public void analyze(Analyzer analyzer) throws UserException { // check access if (!Env.getCurrentEnv().getAccessManager() .checkTblPriv(ConnectContext.get(), tableName.getCtl(), tableName.getDb(), - tableName.getTbl(), PrivPredicate.DROP)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "DROP"); + tableName.getTbl(), PrivPredicate.ALTER)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_ACCESS_DENIED_ERROR, + PrivPredicate.ALTER.getPrivs().toString(), tableName.getTbl()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropPolicyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropPolicyStmt.java index 2a3e3a2bf5c5fc..4bd20fe8c9b084 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DropPolicyStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DropPolicyStmt.java @@ -60,6 +60,12 @@ public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); switch (type) { case STORAGE: + // check auth + if (!Env.getCurrentEnv().getAccessManager() + .checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } break; case ROW: default: @@ -67,10 +73,12 @@ public void analyze(Analyzer analyzer) throws UserException { if (user != null) { user.analyze(); } - } - // check auth - if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + // check auth + if (!Env.getCurrentEnv().getAccessManager() + .checkGlobalPriv(ConnectContext.get(), PrivPredicate.GRANT)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.GRANT.getPrivs().toString()); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetLdapPassVar.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetLdapPassVar.java index c4eed0e5b262e6..5f33c171a81567 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SetLdapPassVar.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SetLdapPassVar.java @@ -17,8 +17,11 @@ package org.apache.doris.analysis; +import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; -import org.apache.doris.mysql.privilege.Auth; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; public class SetLdapPassVar extends SetVar { @@ -35,11 +38,10 @@ public String getLdapPassword() { @Override public void analyze(Analyzer analyzer) throws AnalysisException { - if (!ConnectContext.get().getCurrentUserIdentity().getQualifiedUser().equals(Auth.ROOT_USER) - && !ConnectContext.get().getCurrentUserIdentity().getQualifiedUser().equals(Auth.ADMIN_USER)) { - throw new AnalysisException("Only root and admin user can set ldap admin password."); + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); } - if (!passVar.isPlain()) { throw new AnalysisException("Only support set ldap password with plain text"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogRecycleBinStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogRecycleBinStmt.java index fe241acf04764f..f15c3657240d87 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogRecycleBinStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogRecycleBinStmt.java @@ -18,12 +18,17 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.CaseSensibility; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.PatternMatcher; import org.apache.doris.common.PatternMatcherWrapper; import org.apache.doris.common.UserException; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; import com.google.common.base.Strings; @@ -52,6 +57,12 @@ public String getNameValue() { public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } + if (where == null) { return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java index d6c00c959ee4cc..b709be8d7e95c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateDbStmt.java @@ -24,7 +24,6 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; -import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; @@ -67,11 +66,10 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { ErrorReport.reportAnalysisException(ErrorCode.ERR_WRONG_DB_NAME, db); } - if (!Env.getCurrentEnv().getAccessManager() - .checkDbPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, db, - PrivPredicate.ALTER_CREATE_DROP)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, - ConnectContext.get().getQualifiedUser(), db); + if (!Env.getCurrentEnv().getAccessManager().checkDbPriv(ConnectContext.get(), ctl, db, + PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, + PrivPredicate.SHOW.getPrivs().toString(), db); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateRepositoryStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateRepositoryStmt.java index f11a02253699c3..9de7dd0e9eed72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateRepositoryStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCreateRepositoryStmt.java @@ -18,8 +18,13 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; // SHOW CREATE REPOSITORY statement @@ -43,7 +48,11 @@ public String getRepoName() { @Override public void analyze(Analyzer analyzer) throws AnalysisException { - + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java index 04534b17a467dd..97828a005f9114 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java @@ -424,7 +424,12 @@ public String toString() { return toSql(); } - private void getAllDbStats() { + private void getAllDbStats() throws AnalysisException { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } List dbNames = Env.getCurrentInternalCatalog().getDbNames(); if (dbNames == null || dbNames.isEmpty()) { return; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEncryptKeysStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEncryptKeysStmt.java index f72c972c6e1c91..83358209c23539 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEncryptKeysStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowEncryptKeysStmt.java @@ -24,7 +24,6 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; -import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; @@ -64,14 +63,11 @@ public void analyze(Analyzer analyzer) throws AnalysisException, UserException { } } - // must check after analyze dbName, for case dbName is null. - if (!Env.getCurrentEnv().getAccessManager() - .checkDbPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, - PrivPredicate.ADMIN)) { - ErrorReport.reportAnalysisException( - ErrorCode.ERR_DBACCESS_DENIED_ERROR, ConnectContext.get().getQualifiedUser(), dbName); + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); } - } public boolean like(String str) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPluginsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPluginsStmt.java index 65ffa65622af33..249e49da28dbaf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPluginsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPluginsStmt.java @@ -18,7 +18,13 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; // Show plugins statement. @@ -39,7 +45,12 @@ public class ShowPluginsStmt extends ShowStmt { .build(); @Override - public void analyze(Analyzer analyzer) { + public void analyze(Analyzer analyzer) throws AnalysisException { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRepositoriesStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRepositoriesStmt.java index dbbd5d7b36f98d..6e5166a5c870b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRepositoriesStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRepositoriesStmt.java @@ -18,7 +18,13 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; import com.google.common.collect.ImmutableList; @@ -33,6 +39,15 @@ public ShowRepositoriesStmt() { } + @Override + public void analyze(Analyzer analyzer) throws AnalysisException { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } + } + @Override public ShowResultSetMetaData getMetaData() { ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSnapshotStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSnapshotStmt.java index d10d216b120ad1..83465a34600587 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSnapshotStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowSnapshotStmt.java @@ -19,9 +19,14 @@ import org.apache.doris.analysis.CompoundPredicate.Operator; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; import com.google.common.base.Strings; @@ -55,6 +60,12 @@ public ShowSnapshotStmt(String repoName, Expr where) { public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } + // analyze where clause if not null if (where != null) { // eg: WHERE snapshot="snapshot_label" [and timestamp="2018-04-19-11-11:11"]; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletsBelongStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletsBelongStmt.java index b321dace7f555e..3819541fea9a72 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletsBelongStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTabletsBelongStmt.java @@ -18,8 +18,13 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; import com.google.common.collect.ImmutableList; @@ -54,6 +59,11 @@ public List getTabletIds() { @Override public void analyze(Analyzer analyzer) throws UserException { + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } if (tabletIds == null || tabletIds.isEmpty()) { throw new UserException("Please supply at least one tablet id"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTransactionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTransactionStmt.java index f34c0e5e3363bb..3d5d03bdf8f768 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTransactionStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowTransactionStmt.java @@ -19,12 +19,15 @@ import org.apache.doris.analysis.BinaryPredicate.Operator; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; import org.apache.doris.common.proc.TransProcDir; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.transaction.TransactionStatus; @@ -70,9 +73,15 @@ public boolean labelMatch() { } @Override - public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); + // check auth + if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, + PrivPredicate.ADMIN.getPrivs().toString()); + } + if (Strings.isNullOrEmpty(dbName)) { dbName = analyzer.getDefaultDb(); if (Strings.isNullOrEmpty(dbName)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/ErrorCode.java b/fe/fe-core/src/main/java/org/apache/doris/common/ErrorCode.java index c5841f571b4752..6474f208c6d958 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/ErrorCode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/ErrorCode.java @@ -73,8 +73,14 @@ public enum ErrorCode { ERR_USER_LIMIT_REACHED(1226, new byte[]{'4', '2', '0', '0', '0'}, "User '%s' has exceeded the '%s' resource " + "(current value: %d)"), ERR_SPECIFIC_ACCESS_DENIED_ERROR(1227, new byte[]{'4', '2', '0', '0', '0'}, "Access denied; you need (at least " - + "one of) the %s privilege(s) for this operation"), - ERR_SPECIFIC_ALL_ACCESS_DENIED_ERROR(1227, new byte[] {'4', '2', '0', '0', '0'}, "Access denied; you need all " + + "one of) the (%s) privilege(s) for this operation"), + ERR_DB_ACCESS_DENIED_ERROR(1225, new byte[]{'4', '2', '0', '0', '0'}, "Access denied; you need (at least " + + "one of) the (%s) privilege(s) on database %s for this operation"), + + ERR_TABLE_ACCESS_DENIED_ERROR(1224, new byte[]{'4', '2', '0', '0', '0'}, "Access denied; you need (at least " + + "one of) the (%s) privilege(s) on table %s for this operation"), + + ERR_SPECIFIC_ALL_ACCESS_DENIED_ERROR(1223, new byte[] {'4', '2', '0', '0', '0'}, "Access denied; you need all " + " %s privilege(s) for this operation"), ERR_LOCAL_VARIABLE(1228, new byte[]{'H', 'Y', '0', '0', '0'}, "Variable '%s' is a SESSION variable and can't be " + "used with SET GLOBAL"), @@ -1019,6 +1025,8 @@ public enum ErrorCode { + "DISCARD the tablespace before IMPORT."), ERR_TABLESPACE_DISCARDED(1814, new byte[]{'H', 'Y', '0', '0', '0'}, "Tablespace has been discarded for table '%s'"), ERR_INTERNAL_ERROR(1815, new byte[]{'H', 'Y', '0', '0', '0'}, "Internal error: %s"), + + ERR_MUST_CHANGE_PASSWORD_LOGIN(1862, new byte[]{'H', 'Y', '0', '0', '0'}, "Your password has expired. To log in " + "you must change it using a client that supports expired passwords."), ERR_CREDENTIALS_CONTRADICT_TO_HISTORY(3638, new byte[] {'H', 'Y', '0', '0', '0'}, diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 7e8b01ce28775a..d8a30a968a6716 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -24,11 +24,14 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.CaseSensibility; import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.PatternMatcher; import org.apache.doris.common.PatternMatcherWrapper; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; +import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.common.JobType; @@ -37,6 +40,8 @@ import org.apache.doris.job.extensions.insert.InsertJob; import org.apache.doris.job.scheduler.JobScheduler; import org.apache.doris.load.loadv2.JobState; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; import lombok.extern.log4j.Log4j2; @@ -48,6 +53,7 @@ import java.util.ArrayList; import java.util.LinkedList; import java.util.List; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; @@ -341,7 +347,7 @@ public T getJob(Long jobId) { public List> getLoadJobInfosByDb(long dbId, String dbName, String labelValue, boolean accurateMatch, - JobState jobState) throws AnalysisException { + JobState jobState, String catalogName) throws AnalysisException { LinkedList> loadJobInfos = new LinkedList<>(); if (!Env.getCurrentEnv().getLabelProcessor().existJobs(dbId)) { return loadJobInfos; @@ -356,6 +362,12 @@ public List> getLoadJobInfosByDb(long dbId, String dbName, if (jobState != null && !validState(jobState, loadJob)) { continue; } + // check auth + try { + checkJobAuth(catalogName, dbName, loadJob.getTableNames()); + } catch (AnalysisException e) { + continue; + } // add load job info, convert String list to Comparable list loadJobInfos.add(new ArrayList<>(loadJob.getShowInfo())); } catch (RuntimeException e) { @@ -369,6 +381,27 @@ public List> getLoadJobInfosByDb(long dbId, String dbName, } } + public void checkJobAuth(String ctlName, String dbName, Set tableNames) throws AnalysisException { + if (tableNames.isEmpty()) { + if (!Env.getCurrentEnv().getAccessManager() + .checkDbPriv(ConnectContext.get(), ctlName, dbName, + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, + PrivPredicate.LOAD.getPrivs().toString(), dbName); + } + } else { + for (String tblName : tableNames) { + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), ctlName, dbName, + tblName, PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_ACCESS_DENIED_ERROR, + PrivPredicate.LOAD.getPrivs().toString(), tblName); + return; + } + } + } + } + private static boolean validState(JobState jobState, InsertJob loadJob) { JobStatus status = loadJob.getJobStatus(); switch (status) { @@ -412,6 +445,27 @@ public void cancelLoadJob(CancelLoadStmt cs) } finally { readUnlock(); } + // check auth + if (unfinishedLoadJob.size() > 1 || unfinishedLoadJob.get(0).getTableNames().isEmpty()) { + if (Env.getCurrentEnv().getAccessManager() + .checkDbPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, "LOAD", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), dbName); + } + } else { + for (String tableName : unfinishedLoadJob.get(0).getTableNames()) { + if (Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, dbName, + tableName, + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", + ConnectContext.get().getQualifiedUser(), + ConnectContext.get().getRemoteIP(), dbName + ":" + tableName); + } + } + } for (InsertJob loadJob : unfinishedLoadJob) { try { alterJobStatus(loadJob.getJobId(), JobStatus.STOPPED); 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 4702dd7a9ae92b..afc7ea51984475 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 @@ -26,6 +26,8 @@ import org.apache.doris.common.CaseSensibility; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeConstants; import org.apache.doris.common.LabelAlreadyUsedException; import org.apache.doris.common.PatternMatcher; @@ -123,6 +125,9 @@ public void cancelExportJob(CancelExportStmt stmt) throws DdlException, Analysis if (matchExportJobs.isEmpty()) { throw new DdlException("All export job(s) are at final state (CANCELLED/FINISHED)"); } + + // check auth + checkCancelExportJobAuth(InternalCatalog.INTERNAL_CATALOG_NAME, stmt.getDbName(), matchExportJobs); try { for (ExportJob exportJob : matchExportJobs) { // exportJob.cancel(ExportFailMsg.CancelType.USER_CANCEL, "user cancel"); @@ -134,6 +139,29 @@ public void cancelExportJob(CancelExportStmt stmt) throws DdlException, Analysis } } + public void checkCancelExportJobAuth(String ctlName, String dbName, List jobs) throws AnalysisException { + if (jobs.size() > 1) { + if (Env.getCurrentEnv().getAccessManager() + .checkDbPriv(ConnectContext.get(), ctlName, dbName, + PrivPredicate.SELECT)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, + PrivPredicate.SELECT.getPrivs().toString(), dbName); + } + } else { + TableName tableName = jobs.get(0).getTableName(); + if (tableName == null) { + return; + } + if (Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), ctlName, dbName, + tableName.getTbl(), + PrivPredicate.SELECT)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_ACCESS_DENIED_ERROR, + PrivPredicate.SELECT.getPrivs().toString(), tableName.getTbl()); + } + } + } + public void unprotectAddJob(ExportJob job) { exportIdToJob.put(job.getId(), job); dbTolabelToExportJobId.computeIfAbsent(job.getDbId(), @@ -395,7 +423,7 @@ public void removeOldExportJobs() { ExportJob job = entry.getValue(); if ((currentTimeMs - job.getCreateTimeMs()) / 1000 > Config.history_job_keep_max_second && (job.getState() == ExportJobState.CANCELLED - || job.getState() == ExportJobState.FINISHED)) { + || job.getState() == ExportJobState.FINISHED)) { iter.remove(); Map labelJobs = dbTolabelToExportJobId.get(job.getDbId()); if (labelJobs != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecord.java b/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecord.java index ecbaa2f48eefe2..6ce8be66bbe50b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecord.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecord.java @@ -93,4 +93,12 @@ public String getStatus() { public String getFinishTime() { return this.finishTime; } + + public String getDb() { + return db; + } + + public String getTable() { + return table; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecordMgr.java b/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecordMgr.java index 488e73f3ab43c7..3ceeaa1f38a421 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecordMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecordMgr.java @@ -27,10 +27,13 @@ import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.MasterDaemon; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.plugin.audit.AuditEvent; import org.apache.doris.plugin.audit.AuditEvent.EventType; import org.apache.doris.plugin.audit.StreamLoadAuditEvent; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.system.Backend; import org.apache.doris.thrift.BackendService; import org.apache.doris.thrift.TNetworkAddress; @@ -186,6 +189,13 @@ public List> getStreamLoadRecordByDb( if (state != null && !String.valueOf(state).equalsIgnoreCase(streamLoadRecord.getStatus())) { continue; } + // check auth + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), InternalCatalog.INTERNAL_CATALOG_NAME, + streamLoadRecord.getDb(), streamLoadRecord.getTable(), + PrivPredicate.LOAD)) { + continue; + } streamLoadRecords.add(streamLoadRecord.getStreamLoadInfo()); } catch (Exception e) { continue; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java index 073fec83e12985..0b67ba544c6bbe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java @@ -31,6 +31,8 @@ import org.apache.doris.common.Config; import org.apache.doris.common.DataQualityException; import org.apache.doris.common.DdlException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.LabelAlreadyUsedException; import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; @@ -613,9 +615,16 @@ public List> getLoadJobInfosByDb(long dbId, String labelValue, if (!states.contains(loadJob.getState())) { continue; } + // check auth + try { + checkJobAuth(loadJob.getDb().getCatalog().getName(), loadJob.getDb().getName(), + loadJob.getTableNames()); + } catch (AnalysisException e) { + continue; + } // add load job info loadJobInfos.add(loadJob.getShowInfo()); - } catch (RuntimeException | DdlException e) { + } catch (RuntimeException | DdlException | MetaNotFoundException e) { // ignore this load job LOG.warn("get load job info failed. job id: {}", loadJob.getId(), e); } @@ -626,6 +635,27 @@ public List> getLoadJobInfosByDb(long dbId, String labelValue, } } + public void checkJobAuth(String ctlName, String dbName, Set tableNames) throws AnalysisException { + if (tableNames.isEmpty()) { + if (!Env.getCurrentEnv().getAccessManager() + .checkDbPriv(ConnectContext.get(), ctlName, dbName, + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, + PrivPredicate.LOAD.getPrivs().toString(), dbName); + } + } else { + for (String tblName : tableNames) { + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), ctlName, dbName, + tblName, PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_ACCESS_DENIED_ERROR, + PrivPredicate.LOAD.getPrivs().toString(), tblName); + return; + } + } + } + } + public List> getAllLoadJobInfos() { LinkedList> loadJobInfos = new LinkedList>(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PrivBitSet.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PrivBitSet.java index 1397d3ce477a9a..f2335e63c89268 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PrivBitSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PrivBitSet.java @@ -158,10 +158,15 @@ public String toString() { StringBuilder sb = new StringBuilder(); Privilege.privileges.keySet().forEach(idx -> { if (get(idx)) { - sb.append(Privilege.getPriv(idx)).append(" "); + sb.append(Privilege.getPriv(idx)).append(","); } }); - return sb.toString(); + String res = sb.toString(); + if (res.length() > 0) { + return res.substring(0, res.length() - 1); + } else { + return res; + } } public static PrivBitSet read(DataInput in) throws IOException { 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 f7caec9a657724..6bd829765bc2fd 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 @@ -1306,7 +1306,7 @@ private void handleShowLoad() throws AnalysisException { // add the nerieds load info JobManager loadMgr = env.getJobManager(); loadInfos.addAll(loadMgr.getLoadJobInfosByDb(dbId, db.getFullName(), showStmt.getLabelValue(), - showStmt.isAccurateMatch(), showStmt.getStateV2())); + showStmt.isAccurateMatch(), showStmt.getStateV2(), db.getCatalog().getName())); // order the result of List by orderByPairs in show stmt List orderByPairs = showStmt.getOrderByPairs(); 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 a5cff4fca1aa2a..0b4ecad12e23bd 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 @@ -28,9 +28,9 @@ import org.apache.doris.utframe.TestWithFeService; import com.google.common.collect.Lists; +import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.wildfly.common.Assert; import java.lang.reflect.Method; import java.util.List; @@ -350,4 +350,30 @@ public void testExportMgrCancelJob() throws UserException { exportMgr.cancelExportJob(stmt); Assert.assertTrue(job8.getState() == ExportJobState.CANCELLED); } + + @Test + public void testCancelAuth() { + ExportMgr exportMgr = new ExportMgr(); + List jobs = Lists.newArrayList(); + ExportJob job1 = new ExportJob(); + job1.setTableName(new TableName("ctl1", "db1", "table1")); + jobs.add(job1); + try { + // should check table auth + exportMgr.checkCancelExportJobAuth("ctl1", "db1", jobs); + throw new RuntimeException("should exception"); + } catch (AnalysisException e) { + Assert.assertTrue(e.getMessage().contains("Admin_priv,Select_priv")); + Assert.assertTrue(e.getMessage().contains("table1")); + } + jobs.add(new ExportJob()); + try { + // should check db auth + exportMgr.checkCancelExportJobAuth("ctl1", "db1", jobs); + throw new RuntimeException("should exception"); + } catch (AnalysisException e) { + Assert.assertTrue(e.getMessage().contains("Admin_priv,Select_priv")); + Assert.assertTrue(e.getMessage().contains("db1")); + } + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropMaterializedViewStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropMaterializedViewStmtTest.java index 617f6bf512e769..b43fa6a5ad48d1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DropMaterializedViewStmtTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DropMaterializedViewStmtTest.java @@ -51,7 +51,7 @@ public void testNoPermission(@Injectable TableName tableName) { new Expectations() { { accessManager.checkTblPriv(ConnectContext.get(), tableName.getCtl(), tableName.getDb(), - tableName.getTbl(), PrivPredicate.DROP); + tableName.getTbl(), PrivPredicate.ALTER); result = false; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java index d37058c20cd67d..aee15abd27884b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/RefreshTableTest.java @@ -138,7 +138,7 @@ public void testRefreshPriv() throws Exception { user1.analyze(); ConnectContext user1Ctx = createCtx(user1, "127.0.0.1"); ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, - "Access denied; you need (at least one of) the DROP privilege(s) for this operation", + "Access denied", () -> parseAndAnalyzeStmt("refresh table test1.db1.tbl11", user1Ctx)); ConnectContext.remove(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/job/manager/JobManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/job/manager/JobManagerTest.java new file mode 100644 index 00000000000000..9e3aa386cd06e7 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/job/manager/JobManagerTest.java @@ -0,0 +1,65 @@ +// 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.job.manager; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.utframe.TestWithFeService; + +import com.google.common.collect.Sets; +import mockit.Expectations; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashSet; + +public class JobManagerTest { + @Test + public void testJobAuth() throws IOException, AnalysisException { + UserIdentity user1 = new UserIdentity("testJobAuthUser", "%"); + user1.analyze(); + new Expectations() { + { + ConnectContext.get(); + minTimes = 0; + result = TestWithFeService.createCtx(user1, "%"); + } + }; + JobManager manager = new JobManager(); + HashSet tableNames = Sets.newHashSet(); + try { + // should check db auth + manager.checkJobAuth("ctl1", "db1", tableNames); + throw new RuntimeException("should exception"); + } catch (AnalysisException e) { + Assert.assertTrue(e.getMessage().contains("Admin_priv,Load_priv")); + Assert.assertTrue(e.getMessage().contains("db1")); + } + tableNames.add("table1"); + try { + // should check db auth + manager.checkJobAuth("ctl1", "db1", tableNames); + throw new RuntimeException("should exception"); + } catch (AnalysisException e) { + Assert.assertTrue(e.getMessage().contains("Admin_priv,Load_priv")); + Assert.assertTrue(e.getMessage().contains("table1")); + } + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java index e9b3278cfd08fa..9c09c72bd79a56 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java @@ -21,12 +21,16 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Table; +import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.meta.MetaContext; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.utframe.TestWithFeService; +import com.google.common.collect.Sets; import mockit.Expectations; import mockit.Injectable; import mockit.Mocked; @@ -40,6 +44,8 @@ import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; +import java.io.IOException; +import java.util.HashSet; import java.util.List; import java.util.Map; @@ -195,4 +201,36 @@ private LoadManager deserializeFromFile(File file) throws Exception { loadManager.readFields(dis); return loadManager; } + + @Test + public void testJobAuth() throws IOException, AnalysisException { + UserIdentity user1 = new UserIdentity("testJobAuthUser", "%"); + user1.analyze(); + new Expectations() { + { + ConnectContext.get(); + minTimes = 0; + result = TestWithFeService.createCtx(user1, "%"); + } + }; + LoadManager manager = new LoadManager(new LoadJobScheduler()); + HashSet tableNames = Sets.newHashSet(); + try { + // should check db auth + manager.checkJobAuth("ctl1", "db1", tableNames); + throw new RuntimeException("should exception"); + } catch (AnalysisException e) { + Assert.assertTrue(e.getMessage().contains("Admin_priv,Load_priv")); + Assert.assertTrue(e.getMessage().contains("db1")); + } + tableNames.add("table1"); + try { + // should check db auth + manager.checkJobAuth("ctl1", "db1", tableNames); + throw new RuntimeException("should exception"); + } catch (AnalysisException e) { + Assert.assertTrue(e.getMessage().contains("Admin_priv,Load_priv")); + Assert.assertTrue(e.getMessage().contains("table1")); + } + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java index 41f21110330478..de9e828bacb90c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java @@ -497,7 +497,7 @@ public void testUpdateUnique() throws Exception { QueryState state = connectContext.getState(); Assertions.assertEquals(MysqlStateType.ERR, state.getStateType()); Assertions.assertTrue(state.getErrorMessage() - .contains("you need (at least one of) the LOAD privilege(s) for this operation")); + .contains("you need (at least one of) the (LOAD) privilege(s) for this operation")); // set to admin user connectContext.setCurrentUserIdentity(UserIdentity.ADMIN); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java index 8b06b7a2633a2c..37bc5f431f8257 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java @@ -278,7 +278,7 @@ public StatementBase analyzeAndGetStmtByNereids(String sql, ConnectContext ctx) return adapter; } - protected static ConnectContext createCtx(UserIdentity user, String host) throws IOException { + public static ConnectContext createCtx(UserIdentity user, String host) throws IOException { ConnectContext ctx = new ConnectContext(); ctx.setCurrentUserIdentity(user); ctx.setQualifiedUser(user.getQualifiedUser()); diff --git a/regression-test/data/auth_p0/test_strict_mode.csv b/regression-test/data/auth_p0/test_strict_mode.csv new file mode 100644 index 00000000000000..fbad74eb481183 --- /dev/null +++ b/regression-test/data/auth_p0/test_strict_mode.csv @@ -0,0 +1,2 @@ +1,1 +1,2 diff --git a/regression-test/suites/auth_p0/test_admin_copy_tablet_auth.groovy b/regression-test/suites/auth_p0/test_admin_copy_tablet_auth.groovy new file mode 100644 index 00000000000000..c040dfb8e956c3 --- /dev/null +++ b/regression-test/suites/auth_p0/test_admin_copy_tablet_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_admin_copy_tablet_auth","p0,auth") { + String user = 'test_admin_copy_tablet_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "ADMIN COPY TABLET 10010 PROPERTIES('backend_id' = '10001');" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_alter_policy_auth.groovy b/regression-test/suites/auth_p0/test_alter_policy_auth.groovy new file mode 100644 index 00000000000000..3b1e9af6e2c609 --- /dev/null +++ b/regression-test/suites/auth_p0/test_alter_policy_auth.groovy @@ -0,0 +1,37 @@ +// 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("test_alter_policy_auth","p0,auth") { + String user = 'test_alter_policy_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql """ + ALTER STORAGE POLICY has_test_policy_to_alter PROPERTIES("cooldown_datetime" = "2023-06-08 00:00:00"); + """ + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_alter_view_auth.groovy b/regression-test/suites/auth_p0/test_alter_view_auth.groovy new file mode 100644 index 00000000000000..52bf8ac75c7217 --- /dev/null +++ b/regression-test/suites/auth_p0/test_alter_view_auth.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. + +import org.junit.Assert; + +suite("test_alter_view_auth","p0,auth") { + String user = 'test_alter_view_auth_user' + String pwd = 'C123_567p' + String dbName = 'test_alter_view_auth_db' + String tableName = 'test_alter_view_auth_table' + String viewName = 'test_alter_view_auth_view' + try_sql("DROP USER ${user}") + try_sql """drop table if exists ${dbName}.${tableName}""" + try_sql """drop view if exists ${dbName}.${viewName}""" + sql """drop database if exists ${dbName}""" + + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """create database ${dbName}""" + sql """ + CREATE TABLE IF NOT EXISTS ${dbName}.`${tableName}` ( + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """grant select_priv on regression_test to ${user}""" + sql """create view ${dbName}.${viewName} as select * from ${dbName}.${tableName};""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "alter view ${dbName}.${viewName} as select * from ${dbName}.${tableName};" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv,Alter_priv")) + } + } + sql """grant Alter_priv on ${dbName}.${viewName} to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "alter view ${dbName}.${viewName} as select * from ${dbName}.${tableName};" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv,Select_priv")) + } + } + try_sql """drop table if exists ${dbName}.${tableName}""" + try_sql """drop view if exists ${dbName}.${viewName}""" + sql """drop database if exists ${dbName}""" + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_cancel_alter_system_auth.groovy b/regression-test/suites/auth_p0/test_cancel_alter_system_auth.groovy new file mode 100644 index 00000000000000..0e630fdae8e9cc --- /dev/null +++ b/regression-test/suites/auth_p0/test_cancel_alter_system_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_cancel_alter_system_auth","p0,auth") { + String user = 'test_cancel_alter_system_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "CANCEL DECOMMISSION BACKEND 'id1';" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Node_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_create_policy_auth.groovy b/regression-test/suites/auth_p0/test_create_policy_auth.groovy new file mode 100644 index 00000000000000..8debfaedb273a7 --- /dev/null +++ b/regression-test/suites/auth_p0/test_create_policy_auth.groovy @@ -0,0 +1,47 @@ +// 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("test_create_policy_auth","p0,auth") { + String user = 'test_create_policy_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "CREATE ROW POLICY test_create_policy_auth ON test.table1 AS RESTRICTIVE TO test USING (c1 = 'a');" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv,Grant_priv")) + } + try { + sql """ + CREATE STORAGE POLICY testPolicy + PROPERTIES( + "storage_resource" = "s3", + "cooldown_datetime" = "2022-06-08 00:00:00" + ); + """ + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_create_view_auth.groovy b/regression-test/suites/auth_p0/test_create_view_auth.groovy new file mode 100644 index 00000000000000..124d3f96014073 --- /dev/null +++ b/regression-test/suites/auth_p0/test_create_view_auth.groovy @@ -0,0 +1,62 @@ +// 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("test_create_view_auth","p0,auth") { + String user = 'test_create_view_auth_user' + String pwd = 'C123_567p' + String dbName = 'test_create_view_auth_db' + String tableName = 'test_create_view_auth_table' + try_sql("DROP USER ${user}") + try_sql """drop table if exists ${dbName}.${tableName}""" + sql """drop database if exists ${dbName}""" + + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """create database ${dbName}""" + sql """ + CREATE TABLE IF NOT EXISTS ${dbName}.`${tableName}` ( + id BIGINT, + username VARCHAR(20) + ) + DISTRIBUTED BY HASH(id) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "create view ${dbName}.v1 as select * from ${dbName}.t1;" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv,Create_priv")) + } + } + sql """grant create_priv on ${dbName}.v1 to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "create view ${dbName}.v1 as select * from ${dbName}.${tableName};" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv,Select_priv")) + } + } + sql """drop table if exists ${dbName}.${tableName}""" + sql """drop database if exists ${dbName}""" + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_drop_materialized_view_auth.groovy b/regression-test/suites/auth_p0/test_drop_materialized_view_auth.groovy new file mode 100644 index 00000000000000..4d55b1482cff12 --- /dev/null +++ b/regression-test/suites/auth_p0/test_drop_materialized_view_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_drop_materialized_view_auth","p0,auth") { + String user = 'test_drop_materialized_view_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "DROP MATERIALIZED VIEW mv_name ON table_name;" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv,Alter_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_drop_policy_auth.groovy b/regression-test/suites/auth_p0/test_drop_policy_auth.groovy new file mode 100644 index 00000000000000..0d6a9e103d173f --- /dev/null +++ b/regression-test/suites/auth_p0/test_drop_policy_auth.groovy @@ -0,0 +1,43 @@ +// 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("test_drop_policy_auth","p0,auth") { + String user = 'test_drop_policy_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "DROP ROW POLICY test_row_policy_1 on table1;" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv,Grant_priv")) + } + try { + sql """ + DROP STORAGE POLICY policy_name1 + """ + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_set_ldap_admin_password_auth.groovy b/regression-test/suites/auth_p0/test_set_ldap_admin_password_auth.groovy new file mode 100644 index 00000000000000..6c5032f7ef9867 --- /dev/null +++ b/regression-test/suites/auth_p0/test_set_ldap_admin_password_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_set_ldap_admin_password_auth","p0,auth") { + String user = 'test_set_ldap_admin_password_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SET LDAP_ADMIN_PASSWORD = PASSWORD('plain password')" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_catalog_recycle_bin_auth.groovy b/regression-test/suites/auth_p0/test_show_catalog_recycle_bin_auth.groovy new file mode 100644 index 00000000000000..aa3ba7f8e2bfa2 --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_catalog_recycle_bin_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_catalog_recycle_bin_auth","p0,auth") { + String user = 'test_show_catalog_recycle_bin_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW CATALOG RECYCLE BIN WHERE NAME = 'test'" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_create_database_auth.groovy b/regression-test/suites/auth_p0/test_show_create_database_auth.groovy new file mode 100644 index 00000000000000..c4efff7d662478 --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_create_database_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_create_database_auth","p0,auth") { + String user = 'test_show_create_database_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW CREATE DATABASE db_name" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv,Select_priv,Load_priv,Alter_priv,Create_priv,Drop_priv,Show_view_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_create_repository_auth.groovy b/regression-test/suites/auth_p0/test_show_create_repository_auth.groovy new file mode 100644 index 00000000000000..5e787aead258bf --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_create_repository_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_create_repository_auth","p0,auth") { + String user = 'test_show_create_repository_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW CREATE REPOSITORY for repository_name" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_data_auth.groovy b/regression-test/suites/auth_p0/test_show_data_auth.groovy new file mode 100644 index 00000000000000..fdd42d2759664f --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_data_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_data_auth","p0,auth") { + String user = 'test_show_data_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW DATA" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_encryptkeys_auth.groovy b/regression-test/suites/auth_p0/test_show_encryptkeys_auth.groovy new file mode 100644 index 00000000000000..1e2cf4c27fd212 --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_encryptkeys_auth.groovy @@ -0,0 +1,36 @@ +// 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("test_show_encryptkeys_auth","p0,auth") { + String user = 'test_show_encryptkeys_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "use regression_test" + sql "SHOW ENCRYPTKEYS" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_plugins_auth.groovy b/regression-test/suites/auth_p0/test_show_plugins_auth.groovy new file mode 100644 index 00000000000000..b5b4de84265162 --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_plugins_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_plugins_auth","p0,auth") { + String user = 'test_show_plugins_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW PLUGINS" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_repositories_auth.groovy b/regression-test/suites/auth_p0/test_show_repositories_auth.groovy new file mode 100644 index 00000000000000..507d8768669973 --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_repositories_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_repositories_auth","p0,auth") { + String user = 'test_show_repositories_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW REPOSITORIES" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_snapshot_auth.groovy b/regression-test/suites/auth_p0/test_show_snapshot_auth.groovy new file mode 100644 index 00000000000000..f90e1fdc57aaab --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_snapshot_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_snapshot_auth","p0,auth") { + String user = 'test_show_snapshot_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW SNAPSHOT ON example_repo" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_stream_load_auth.groovy b/regression-test/suites/auth_p0/test_show_stream_load_auth.groovy new file mode 100644 index 00000000000000..cb2446f04405f3 --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_stream_load_auth.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. + +import org.junit.Assert; + +suite("test_show_stream_load_auth","p0,auth") { + String tableName = "test_show_stream_load_auth_table" + String label = "test_show_stream_load_auth_label" + System.currentTimeMillis(); + String user = 'test_show_stream_load_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """ DROP TABLE IF EXISTS ${tableName} """ + + + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + `k1` bigint(20) NULL, + `k2` bigint(20) NULL + ) ENGINE=OLAP + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 2 + PROPERTIES ("replication_allocation" = "tag.location.default: 1"); + """ + + streamLoad { + table "${tableName}" + + set 'column_separator', ',' + set 'columns', 'k1, k2' + set 'label', label + set 'strict_mode', 'true' + + file 'test_strict_mode.csv' + time 10000 // limit inflight 10s + } + + Thread.sleep(60000); + def res = sql "SHOW STREAM LOAD from regression_test_auth_p0 where label = '${label}'" + log.info(res.toString()) + if(res.size() == 0) { + // `show stream load` has some delay, and need be config `enable_stream_load_record=true` + // we not sure when can has result, so if `admin` can not get res, ignore this case. + return; + } + + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + res = sql "SHOW STREAM LOAD from regression_test_auth_p0 where label = '${label}'" + log.info(res.toString()) + assertFalse(res.toString().contains("${label}")) + } + sql """grant load_priv on regression_test_auth_p0.${tableName} to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + res = sql "SHOW STREAM LOAD from regression_test_auth_p0 where label = '${label}'" + log.info(res.toString()) + assertTrue(res.toString().contains("${label}")) + } + try_sql("DROP USER ${user}") + sql """ DROP TABLE IF EXISTS ${tableName} """ +} diff --git a/regression-test/suites/auth_p0/test_show_tablets_belong_auth.groovy b/regression-test/suites/auth_p0/test_show_tablets_belong_auth.groovy new file mode 100644 index 00000000000000..b7d66af9348f0d --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_tablets_belong_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_tablets_belong_auth","p0,auth") { + String user = 'test_show_tablets_belong_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW TABLETS BELONG 27028" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} diff --git a/regression-test/suites/auth_p0/test_show_transaction_auth.groovy b/regression-test/suites/auth_p0/test_show_transaction_auth.groovy new file mode 100644 index 00000000000000..f4e564c52889a0 --- /dev/null +++ b/regression-test/suites/auth_p0/test_show_transaction_auth.groovy @@ -0,0 +1,35 @@ +// 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("test_show_transaction_auth","p0,auth") { + String user = 'test_show_transaction_auth_user' + String pwd = 'C123_567p' + try_sql("DROP USER ${user}") + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + sql """grant select_priv on regression_test to ${user}""" + connect(user=user, password="${pwd}", url=context.config.jdbcUrl) { + try { + sql "SHOW TRANSACTION WHERE ID=4005;" + } catch (Exception e) { + log.info(e.getMessage()) + assertTrue(e.getMessage().contains("Admin_priv")) + } + } + try_sql("DROP USER ${user}") +} From a393cec5caf3b81a1ab171fb67cecbe2b05ce66a Mon Sep 17 00:00:00 2001 From: Liqf <109049295+LemonLiTree@users.noreply.github.com> Date: Mon, 15 Apr 2024 11:25:21 +0800 Subject: [PATCH 65/71] [docker](script)add --grace to be_prestop.sh (#33599) --- docker/runtime/be/resource/be_prestop.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/runtime/be/resource/be_prestop.sh b/docker/runtime/be/resource/be_prestop.sh index af9ab1001c604f..798cf28944fb37 100755 --- a/docker/runtime/be/resource/be_prestop.sh +++ b/docker/runtime/be/resource/be_prestop.sh @@ -18,4 +18,4 @@ DORIS_ROOT=${DORIS_ROOT:-"/opt/apache-doris"} DORIS_HOME=${DORIS_ROOT}/be -$DORIS_HOME/bin/stop_be.sh +$DORIS_HOME/bin/stop_be.sh --grace From debb83f35d37cac9089b2e0282e2d555ad9c1d62 Mon Sep 17 00:00:00 2001 From: starocean999 <40539150+starocean999@users.noreply.github.com> Date: Mon, 15 Apr 2024 11:39:30 +0800 Subject: [PATCH 66/71] [fix](nereids) ExtractAndNormalizeWindowExpression should only normalize alias in output (#33527) --- .../rules/rewrite/ExtractAndNormalizeWindowExpression.java | 5 ++++- .../suites/nereids_p0/aggregate/agg_window_project.groovy | 2 ++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java index 5ec0f0cd698d5e..6f067545cee0cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpression.java @@ -123,8 +123,11 @@ private Plan normalize(LogicalProject project) { .collect(Collectors.toMap(expr -> ((Alias) expr).child(), expr -> ((Alias) expr).toSlot(), (oldExpr, newExpr) -> oldExpr)); + // customNormalizeMap is only for alias, so we just normalize alias in outputs too List normalizedOutputs = context.normalizeToUseSlotRef(outputs, - (ctx, expr) -> customNormalizeMap.getOrDefault(expr, null)); + (ctx, expr) -> expr instanceof Alias ? customNormalizeMap.getOrDefault(expr, null) : null); + // replace child exprs in normalizedOutputs by customNormalizeMap + normalizedOutputs = ExpressionUtils.replaceNamedExpressions(normalizedOutputs, customNormalizeMap); Set normalizedWindows = ExpressionUtils.collect(normalizedOutputs, WindowExpression.class::isInstance); diff --git a/regression-test/suites/nereids_p0/aggregate/agg_window_project.groovy b/regression-test/suites/nereids_p0/aggregate/agg_window_project.groovy index 48d342b4b02b41..ace63dce3b4b10 100644 --- a/regression-test/suites/nereids_p0/aggregate/agg_window_project.groovy +++ b/regression-test/suites/nereids_p0/aggregate/agg_window_project.groovy @@ -103,5 +103,7 @@ suite("agg_window_project") { contains "ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW" } + sql """select a, a aa, row_number() over (partition by b) from test_window_table2;""" + sql "DROP TABLE IF EXISTS test_window_table2;" } From 65ccd32babc586ee4af16563b27ace6cee56a4f0 Mon Sep 17 00:00:00 2001 From: starocean999 <40539150+starocean999@users.noreply.github.com> Date: Mon, 15 Apr 2024 14:59:09 +0800 Subject: [PATCH 67/71] [fix](planner) fix bug of InlineViewRef's tableNameToSql method (#33575) --- .../apache/doris/analysis/InlineViewRef.java | 4 +-- .../apache/doris/qe/OlapQueryCacheTest.java | 10 +++--- .../suites/ddl_p0/test_create_view.groovy | 32 ++++++++++++++++++- 3 files changed, 37 insertions(+), 9 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/InlineViewRef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/InlineViewRef.java index 86e93c420617f7..a6418abf3b0cb0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/InlineViewRef.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/InlineViewRef.java @@ -514,9 +514,7 @@ public String tableNameToSql() { aliasSql = ToSqlUtils.getIdentSql(alias); } StringBuilder sb = new StringBuilder(); - sb.append("(").append(queryStmt.toSql()).append(") ") - .append(aliasSql); - + sb.append("(").append(queryStmt.toSqlWithSelectList()).append(") ").append(aliasSql); return sb.toString(); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java index 9ff7042e76e027..5767f1c4b78080 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java @@ -975,7 +975,7 @@ public void testSubSelect() throws Exception { LOG.warn("Sub nokey={}", cache.getNokeyStmt().toSql()); Assert.assertEquals(cache.getNokeyStmt().toSql(), "SELECT `eventdate` AS `eventdate`, sum(`pv`) AS `sum(``pv``)` " - + "FROM (SELECT `eventdate` AS `eventdate`, count(`userid`) AS `pv` " + + "FROM (SELECT `eventdate`, count(`userid`) `pv` " + "FROM `testDb`.`appevent` WHERE (`eventid` = 1) GROUP BY `eventdate`) tbl " + "GROUP BY `eventdate`"); @@ -997,7 +997,7 @@ public void testSubSelect() throws Exception { LOG.warn("Sub rewrite={}", sql); Assert.assertEquals(sql, "SELECT `eventdate` AS `eventdate`, sum(`pv`) AS `sum(``pv``)` " - + "FROM (SELECT `eventdate` AS `eventdate`, count(`userid`) AS `pv` " + + "FROM (SELECT `eventdate`, count(`userid`) `pv` " + "FROM `testDb`.`appevent` WHERE (`eventdate` > '2020-01-13') " + "AND (`eventdate` < '2020-01-16') AND (`eventid` = 1) GROUP BY `eventdate`) tbl " + "GROUP BY `eventdate`"); @@ -1133,8 +1133,8 @@ public void testSqlCacheKeyWithSubSelectView() { SqlCache sqlCache = (SqlCache) ca.getCache(); String cacheKey = sqlCache.getSqlWithViewStmt(); Assert.assertEquals(cacheKey, "SELECT `origin`.`eventdate` AS `eventdate`, " - + "`origin`.`userid` AS `userid` FROM (SELECT `view2`.`eventdate` AS `eventdate`, " - + "`view2`.`userid` AS `userid` FROM `testDb`.`view2` view2 " + + "`origin`.`userid` AS `userid` FROM (SELECT `view2`.`eventdate` `eventdate`, " + + "`view2`.`userid` `userid` FROM `testDb`.`view2` view2 " + "WHERE (`view2`.`eventdate` >= '2020-01-12') AND (`view2`.`eventdate` <= '2020-01-14')) origin|" + "SELECT `eventdate` AS `eventdate`, `userid` AS `userid` FROM `testDb`.`appevent`"); Assert.assertEquals(selectedPartitionIds.size(), sqlCache.getSumOfPartitionNum()); @@ -1219,7 +1219,7 @@ public void testPartitionCacheKeyWithSubSelectView() { Assert.assertEquals(cache.getNokeyStmt().getWhereClause(), null); Assert.assertEquals(cache.getSqlWithViewStmt(), "SELECT `origin`.`eventdate` AS `eventdate`, `origin`.`cnt` AS `cnt` " - + "FROM (SELECT `eventdate` AS `eventdate`, count(`userid`) AS `cnt` " + + "FROM (SELECT `eventdate`, count(`userid`) `cnt` " + "FROM `testDb`.`view2` GROUP BY `eventdate`) origin|SELECT `eventdate` " + "AS `eventdate`, `userid` AS `userid` FROM `testDb`.`appevent`"); } catch (Exception e) { diff --git a/regression-test/suites/ddl_p0/test_create_view.groovy b/regression-test/suites/ddl_p0/test_create_view.groovy index e66798e50bbb19..e5d6f41d319169 100644 --- a/regression-test/suites/ddl_p0/test_create_view.groovy +++ b/regression-test/suites/ddl_p0/test_create_view.groovy @@ -204,4 +204,34 @@ suite("test_create_view") { view_column_name_test """ qt_test_view_6 """ SHOW VIEW FROM view_column_name_test;""" -} \ No newline at end of file + + sql """ drop view if exists xxx;""" + sql """CREATE VIEW + `xxx` COMMENT 'VIEW' AS + WITH + CHENGBENJIA AS ( + SELECT + RN + FROM + ( + SELECT + row_number() OVER ( + PARTITION BY `A`.`timestamp`, + `A`.`type` + ORDER BY + CAST( + concat( + CAST(`A`.`error_msg` AS VARCHAR(*)), + CAST(`A`.`remark` AS VARCHAR(*)) + ) AS INT + ) DESC NULLS LAST + ) AS `RN` + FROM + view_column_name_test A + ) A + ) + SELECT + * from CHENGBENJIA;""" + sql """select * from xxx;""" + sql """ drop view if exists xxx;""" +} From 7c0669e2c252ce51173b7d051153f5b37120f6ec Mon Sep 17 00:00:00 2001 From: Mryange <59914473+Mryange@users.noreply.github.com> Date: Mon, 15 Apr 2024 15:00:09 +0800 Subject: [PATCH 68/71] [refine](Operator) When _stop_emplace_flag is not set to true, perform batch processing on the block. (#33173) --- ...istinct_streaming_aggregation_operator.cpp | 112 ++++++++++++------ .../distinct_streaming_aggregation_operator.h | 15 ++- be/src/vec/columns/column.h | 6 + be/src/vec/columns/column_array.h | 4 + be/src/vec/columns/column_complex.h | 5 + be/src/vec/columns/column_const.h | 4 + be/src/vec/columns/column_decimal.h | 4 + be/src/vec/columns/column_dictionary.h | 5 + be/src/vec/columns/column_dummy.h | 14 +++ .../vec/columns/column_fixed_length_object.h | 5 + be/src/vec/columns/column_impl.h | 10 +- be/src/vec/columns/column_map.h | 4 + be/src/vec/columns/column_nullable.h | 5 + be/src/vec/columns/column_object.cpp | 5 - be/src/vec/columns/column_object.h | 9 +- be/src/vec/columns/column_string.h | 4 + be/src/vec/columns/column_struct.h | 4 + be/src/vec/columns/column_vector.h | 4 + be/src/vec/columns/predicate_column.h | 4 + 19 files changed, 178 insertions(+), 45 deletions(-) diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp index 7983b269488eca..9151f4a29d5d7a 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.cpp @@ -46,7 +46,7 @@ static constexpr StreamingHtMinReductionEntry STREAMING_HT_MIN_REDUCTION[] = { {0, 0.0}, // Expand into L3 cache if we look like we're getting some reduction. // At present, The L2 cache is generally 1024k or more - {1024 * 1024, 1.1}, + {1024 * 1024, 0.0}, // Expand into main memory if we're getting a significant reduction. // The L3 cache is generally 16MB or more {16 * 1024 * 1024, 2.0}, @@ -59,6 +59,7 @@ DistinctStreamingAggLocalState::DistinctStreamingAggLocalState(RuntimeState* sta OperatorXBase* parent) : PipelineXLocalState(state, parent), dummy_mapped_data(std::make_shared('A')), + batch_size(state->batch_size()), _agg_arena_pool(std::make_unique()), _agg_data(std::make_unique()), _agg_profile_arena(std::make_unique()), @@ -83,6 +84,8 @@ Status DistinctStreamingAggLocalState::init(RuntimeState* state, LocalStateInfo& _hash_table_compute_timer = ADD_TIMER(Base::profile(), "HashTableComputeTime"); _hash_table_emplace_timer = ADD_TIMER(Base::profile(), "HashTableEmplaceTime"); _hash_table_input_counter = ADD_COUNTER(Base::profile(), "HashTableInputCount", TUnit::UNIT); + _hash_table_size_counter = ADD_COUNTER(profile(), "HashTableSize", TUnit::UNIT); + _insert_keys_to_column_timer = ADD_TIMER(profile(), "InsertKeysToColumnTime"); if (_probe_expr_ctxs.empty()) { _agg_data->without_key = reinterpret_cast( @@ -120,7 +123,7 @@ bool DistinctStreamingAggLocalState::_should_expand_preagg_hash_tables() { // were aggregated into it. Exclude passed through rows from this calculation since // they were not in hash tables. const int64_t input_rows = _input_num_rows; - const int64_t aggregated_input_rows = input_rows - _cur_num_rows_returned; + const int64_t aggregated_input_rows = input_rows - _num_rows_returned; // TODO chenhao // const int64_t expected_input_rows = estimated_input_cardinality_ - num_rows_returned_; double current_reduction = static_cast(aggregated_input_rows) / ht_rows; @@ -245,22 +248,48 @@ Status DistinctStreamingAggLocalState::_distinct_pre_agg_with_serialized_key( _distinct_row.clear(); _distinct_row.reserve(rows); - RETURN_IF_CATCH_EXCEPTION( - _emplace_into_hash_table_to_distinct(_distinct_row, key_columns, rows)); - // need use _cur_num_rows_returned to decide whether to do continue emplace into hash table - _cur_num_rows_returned += _distinct_row.size(); + if (!_stop_emplace_flag) { + RETURN_IF_CATCH_EXCEPTION( + _emplace_into_hash_table_to_distinct(_distinct_row, key_columns, rows)); + } bool mem_reuse = _parent->cast()._make_nullable_keys.empty() && out_block->mem_reuse(); + SCOPED_TIMER(_insert_keys_to_column_timer); if (mem_reuse) { - for (int i = 0; i < key_size; ++i) { - auto output_column = out_block->get_by_position(i).column; - if (_stop_emplace_flag) { // swap the column directly, to solve Check failed: d.column->use_count() == 1 (2 vs. 1) + if (_stop_emplace_flag && !out_block->empty()) { + // when out_block row >= batch_size, push it to data_queue, so when _stop_emplace_flag = true, maybe have some data in block + // need output those data firstly + DCHECK(_distinct_row.empty()); + _distinct_row.resize(rows); + std::iota(_distinct_row.begin(), _distinct_row.end(), 0); + } + DCHECK_EQ(out_block->columns(), key_size); + if (_stop_emplace_flag && _distinct_row.empty()) { + // swap the column directly, to solve Check failed: d.column->use_count() == 1 (2 vs. 1) + for (int i = 0; i < key_size; ++i) { + auto output_column = out_block->get_by_position(i).column; out_block->replace_by_position(i, key_columns[i]->assume_mutable()); in_block->replace_by_position(result_idxs[i], output_column); + } + } else { + DCHECK_EQ(_cache_block.rows(), 0); + if (out_block->rows() + _distinct_row.size() > batch_size) { + size_t split_size = batch_size - out_block->rows(); + for (int i = 0; i < key_size; ++i) { + auto output_dst = out_block->get_by_position(i).column->assume_mutable(); + key_columns[i]->append_data_by_selector(output_dst, _distinct_row, 0, + split_size); + auto cache_dst = _cache_block.get_by_position(i).column->assume_mutable(); + key_columns[i]->append_data_by_selector(cache_dst, _distinct_row, split_size, + _distinct_row.size()); + } } else { - auto dst = output_column->assume_mutable(); - key_columns[i]->append_data_by_selector(dst, _distinct_row); + for (int i = 0; i < key_size; ++i) { + auto output_column = out_block->get_by_position(i).column; + auto dst = output_column->assume_mutable(); + key_columns[i]->append_data_by_selector(dst, _distinct_row); + } } } } else { @@ -279,6 +308,7 @@ Status DistinctStreamingAggLocalState::_distinct_pre_agg_with_serialized_key( } } out_block->swap(vectorized::Block(columns_with_schema)); + _cache_block = out_block->clone_empty(); if (_stop_emplace_flag) { in_block->clear(); // clear the column ref with stop_emplace_flag = true } @@ -443,26 +473,18 @@ Status DistinctStreamingAggOperatorX::push(RuntimeState* state, vectorized::Bloc bool eos) const { auto& local_state = get_local_state(state); local_state._input_num_rows += in_block->rows(); - Status ret = Status::OK(); - if (in_block->rows() > 0) { - RETURN_IF_ERROR(local_state._distinct_pre_agg_with_serialized_key( - in_block, local_state._aggregated_block.get())); - - // get enough data or reached limit rows, need push block to queue - if (!local_state._stop_emplace_flag && _limit != -1 && - (local_state._aggregated_block->rows() + local_state._output_distinct_rows) >= _limit) { - auto limit_rows = _limit - local_state._output_distinct_rows; - local_state._aggregated_block->set_num_rows(limit_rows); - local_state._output_distinct_rows += limit_rows; - } else if (!local_state._stop_emplace_flag) { - local_state._output_distinct_rows += local_state._aggregated_block->rows(); - } + if (in_block->rows() == 0) { + return Status::OK(); } - // reach limit or source finish - if ((UNLIKELY(eos)) || (_limit != -1 && local_state._output_distinct_rows >= _limit)) { - local_state._output_distinct_rows += local_state._aggregated_block->rows(); - return Status::OK(); // need given finish signal + RETURN_IF_ERROR(local_state._distinct_pre_agg_with_serialized_key( + in_block, local_state._aggregated_block.get())); + // set limit and reach limit + if (_limit != -1 && + (local_state._num_rows_returned + local_state._aggregated_block->rows()) > _limit) { + auto limit_rows = _limit - local_state._num_rows_returned; + local_state._aggregated_block->set_num_rows(limit_rows); + local_state._reach_limit = true; } return Status::OK(); } @@ -473,23 +495,34 @@ Status DistinctStreamingAggOperatorX::pull(RuntimeState* state, vectorized::Bloc if (!local_state._aggregated_block->empty()) { block->swap(*local_state._aggregated_block); local_state._aggregated_block->clear_column_data(block->columns()); + // The cache block may have additional data due to exceeding the batch size. + if (!local_state._cache_block.empty()) { + local_state._swap_cache_block(local_state._aggregated_block.get()); + } } local_state._make_nullable_output_key(block); - if (_is_streaming_preagg == false) { + if (!_is_streaming_preagg) { // dispose the having clause, should not be execute in prestreaming agg RETURN_IF_ERROR( vectorized::VExprContext::filter_block(_conjuncts, block, block->columns())); } local_state.add_num_rows_returned(block->rows()); - *eos = local_state._child_eos || (_limit != -1 && local_state._output_distinct_rows >= _limit); + COUNTER_UPDATE(local_state.blocks_returned_counter(), 1); + // If the limit is not reached, it is important to ensure that _aggregated_block is empty + // because it may still contain data. + // However, if the limit is reached, there is no need to output data even if some exists. + *eos = (local_state._child_eos && local_state._aggregated_block->empty()) || + (local_state._reach_limit); return Status::OK(); } bool DistinctStreamingAggOperatorX::need_more_input_data(RuntimeState* state) const { auto& local_state = get_local_state(state); - return local_state._aggregated_block->empty() && !local_state._child_eos && - (_limit == -1 || local_state._output_distinct_rows < _limit); + const bool need_batch = local_state._stop_emplace_flag + ? local_state._aggregated_block->empty() + : local_state._aggregated_block->rows() < state->batch_size(); + return need_batch && !(local_state._child_eos || local_state._reach_limit); } Status DistinctStreamingAggLocalState::close(RuntimeState* state) { @@ -498,10 +531,23 @@ Status DistinctStreamingAggLocalState::close(RuntimeState* state) { } SCOPED_TIMER(Base::exec_time_counter()); SCOPED_TIMER(Base::_close_timer); + /// _hash_table_size_counter may be null if prepare failed. + if (_hash_table_size_counter) { + std::visit( + [&](auto&& agg_method) { + COUNTER_SET(_hash_table_size_counter, int64_t(agg_method.hash_table->size())); + }, + _agg_data->method_variant); + } if (Base::_closed) { return Status::OK(); } _aggregated_block->clear(); + // If the limit is reached, there may still be remaining data in the cache block. + // If the limit is not reached, the cache block must be empty. + DCHECK(_reach_limit || _aggregated_block->empty()); + DCHECK(_reach_limit || _cache_block.empty()); + _cache_block.clear(); return Base::close(state); } diff --git a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h index 4c15194362e193..125f176375b862 100644 --- a/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h +++ b/be/src/pipeline/exec/distinct_streaming_aggregation_operator.h @@ -58,15 +58,19 @@ class DistinctStreamingAggLocalState final : public PipelineXLocalStateswap(_cache_block); + _cache_block = block->clone_empty(); + } + std::shared_ptr dummy_mapped_data; vectorized::IColumn::Selector _distinct_row; vectorized::Arena _arena; - int64_t _output_distinct_rows = 0; size_t _input_num_rows = 0; bool _should_expand_hash_table = true; - int64_t _cur_num_rows_returned = 0; bool _stop_emplace_flag = false; - + const int batch_size; std::unique_ptr _agg_arena_pool = nullptr; vectorized::AggregatedDataVariantsUPtr _agg_data = nullptr; std::vector _aggregate_evaluators; @@ -75,13 +79,16 @@ class DistinctStreamingAggLocalState final : public PipelineXLocalState _agg_profile_arena = nullptr; std::unique_ptr _child_block = nullptr; bool _child_eos = false; + bool _reach_limit = false; std::unique_ptr _aggregated_block = nullptr; - + vectorized::Block _cache_block; RuntimeProfile::Counter* _build_timer = nullptr; RuntimeProfile::Counter* _expr_timer = nullptr; RuntimeProfile::Counter* _hash_table_compute_timer = nullptr; RuntimeProfile::Counter* _hash_table_emplace_timer = nullptr; RuntimeProfile::Counter* _hash_table_input_counter = nullptr; + RuntimeProfile::Counter* _hash_table_size_counter = nullptr; + RuntimeProfile::Counter* _insert_keys_to_column_timer = nullptr; }; class DistinctStreamingAggOperatorX final diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index f1bcee1ad7d5b4..a6d48a41fca619 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -499,6 +499,9 @@ class IColumn : public COW { virtual void append_data_by_selector(MutablePtr& res, const Selector& selector) const = 0; + virtual void append_data_by_selector(MutablePtr& res, const Selector& selector, size_t begin, + size_t end) const = 0; + /// Insert data from several other columns according to source mask (used in vertical merge). /// For now it is a helper to de-virtualize calls to insert*() functions inside gather loop /// (descendants should call gatherer_stream.gather(*this) to implement this function.) @@ -695,6 +698,9 @@ class IColumn : public COW { protected: template void append_data_by_selector_impl(MutablePtr& res, const Selector& selector) const; + template + void append_data_by_selector_impl(MutablePtr& res, const Selector& selector, size_t begin, + size_t end) const; }; using ColumnPtr = IColumn::Ptr; diff --git a/be/src/vec/columns/column_array.h b/be/src/vec/columns/column_array.h index 118e7ab05c6428..7d619c14efffb7 100644 --- a/be/src/vec/columns/column_array.h +++ b/be/src/vec/columns/column_array.h @@ -207,6 +207,10 @@ class ColumnArray final : public COWHelper { const IColumn::Selector& selector) const override { return append_data_by_selector_impl(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + return append_data_by_selector_impl(res, selector, begin, end); + } void for_each_subcolumn(ColumnCallback callback) override { callback(offsets); diff --git a/be/src/vec/columns/column_complex.h b/be/src/vec/columns/column_complex.h index d983fc9175ea1f..54b0650e8003de 100644 --- a/be/src/vec/columns/column_complex.h +++ b/be/src/vec/columns/column_complex.h @@ -262,6 +262,11 @@ class ColumnComplexType final : public COWHelper> const IColumn::Selector& selector) const override { this->template append_data_by_selector_impl>(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + this->template append_data_by_selector_impl>(res, selector, begin, + end); + } void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { DCHECK(size() > self_row); diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index 66db2ed54f0ca1..e1c3c52949bd04 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -253,6 +253,10 @@ class ColumnConst final : public COWHelper { const IColumn::Selector& selector) const override { assert_cast(*res).resize(selector.size()); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + assert_cast(*res).resize(end - begin); + } void for_each_subcolumn(ColumnCallback callback) override { callback(data); } diff --git a/be/src/vec/columns/column_decimal.h b/be/src/vec/columns/column_decimal.h index 152d4165416cf0..7f286699ab8449 100644 --- a/be/src/vec/columns/column_decimal.h +++ b/be/src/vec/columns/column_decimal.h @@ -231,6 +231,10 @@ class ColumnDecimal final : public COWHelper> { const IColumn::Selector& selector) const override { this->template append_data_by_selector_impl(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + this->template append_data_by_selector_impl(res, selector, begin, end); + } // void gather(ColumnGathererStream & gatherer_stream) override; diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h index 175912f96682dc..3b1537b83d3a8a 100644 --- a/be/src/vec/columns/column_dictionary.h +++ b/be/src/vec/columns/column_dictionary.h @@ -195,6 +195,11 @@ class ColumnDictionary final : public COWHelper> { LOG(FATAL) << "append_data_by_selector is not supported in ColumnDictionary!"; } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + LOG(FATAL) << "append_data_by_selector is not supported in ColumnDictionary!"; + } + [[noreturn]] ColumnPtr index(const IColumn& indexes, size_t limit) const override { LOG(FATAL) << "index not implemented"; __builtin_unreachable(); diff --git a/be/src/vec/columns/column_dummy.h b/be/src/vec/columns/column_dummy.h index f330a60642a96c..a28133c69a3d80 100644 --- a/be/src/vec/columns/column_dummy.h +++ b/be/src/vec/columns/column_dummy.h @@ -130,6 +130,20 @@ class IColumnDummy : public IColumn { for (size_t i = 0; i < selector.size(); ++i) res->insert_from(*this, selector[i]); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + size_t num_rows = size(); + + if (num_rows < selector.size()) { + LOG(FATAL) << fmt::format("Size of selector: {}, is larger than size of column:{}", + selector.size(), num_rows); + } + + res->reserve(num_rows); + + for (size_t i = begin; i < end; ++i) res->insert_from(*this, selector[i]); + } + void addSize(size_t delta) { s += delta; } void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { diff --git a/be/src/vec/columns/column_fixed_length_object.h b/be/src/vec/columns/column_fixed_length_object.h index 5875b8f6c30cbe..a10204ed48b0e3 100644 --- a/be/src/vec/columns/column_fixed_length_object.h +++ b/be/src/vec/columns/column_fixed_length_object.h @@ -238,6 +238,11 @@ class ColumnFixedLengthObject final : public COWHelpertemplate append_data_by_selector_impl(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + this->template append_data_by_selector_impl(res, selector, begin, end); + } + size_t byte_size() const override { return _data.size(); } size_t item_size() const { return _item_size; } diff --git a/be/src/vec/columns/column_impl.h b/be/src/vec/columns/column_impl.h index 20292ad23518dc..4a66720c1168cb 100644 --- a/be/src/vec/columns/column_impl.h +++ b/be/src/vec/columns/column_impl.h @@ -33,7 +33,8 @@ namespace doris::vectorized { template -void IColumn::append_data_by_selector_impl(MutablePtr& res, const Selector& selector) const { +void IColumn::append_data_by_selector_impl(MutablePtr& res, const Selector& selector, size_t begin, + size_t end) const { size_t num_rows = size(); if (num_rows < selector.size()) { @@ -43,8 +44,13 @@ void IColumn::append_data_by_selector_impl(MutablePtr& res, const Selector& sele res->reserve(num_rows); - for (size_t i = 0; i < selector.size(); ++i) + for (size_t i = begin; i < end; ++i) { static_cast(*res).insert_from(*this, selector[i]); + } +} +template +void IColumn::append_data_by_selector_impl(MutablePtr& res, const Selector& selector) const { + append_data_by_selector_impl(res, selector, 0, selector.size()); } template diff --git a/be/src/vec/columns/column_map.h b/be/src/vec/columns/column_map.h index 2cdfcae8c73f70..e0bc7e72d7826e 100644 --- a/be/src/vec/columns/column_map.h +++ b/be/src/vec/columns/column_map.h @@ -133,6 +133,10 @@ class ColumnMap final : public COWHelper { const IColumn::Selector& selector) const override { return append_data_by_selector_impl(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + return append_data_by_selector_impl(res, selector, begin, end); + } void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { LOG(FATAL) << "Method replace_column_data is not supported for " << get_name(); diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 56dbc619a39212..0b78330949974a 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -232,6 +232,11 @@ class ColumnNullable final : public COWHelper { append_data_by_selector_impl(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + append_data_by_selector_impl(res, selector, begin, end); + } + // void gather(ColumnGathererStream & gatherer_stream) override; void for_each_subcolumn(ColumnCallback callback) override { diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp index c596717194fc23..6d7104647c7c11 100644 --- a/be/src/vec/columns/column_object.cpp +++ b/be/src/vec/columns/column_object.cpp @@ -1435,11 +1435,6 @@ Status ColumnObject::extract_root(const PathInData& path, MutableColumnPtr& dst) return Status::OK(); } -void ColumnObject::append_data_by_selector(MutableColumnPtr& res, - const IColumn::Selector& selector) const { - return append_data_by_selector_impl(res, selector); -} - void ColumnObject::insert_indices_from(const IColumn& src, const uint32_t* indices_begin, const uint32_t* indices_end) { for (const auto* x = indices_begin; x != indices_end; ++x) { diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h index 26b2c66a7551b7..02edde29cf55ca 100644 --- a/be/src/vec/columns/column_object.h +++ b/be/src/vec/columns/column_object.h @@ -385,7 +385,14 @@ class ColumnObject final : public COWHelper { void insert(const Field& field) override { try_insert(field); } void append_data_by_selector(MutableColumnPtr& res, - const IColumn::Selector& selector) const override; + const IColumn::Selector& selector) const override { + append_data_by_selector_impl(res, selector); + } + + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + append_data_by_selector_impl(res, selector, begin, end); + } void insert_indices_from(const IColumn& src, const uint32_t* indices_begin, const uint32_t* indices_end) override; diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index 405ada3b48d361..c2eedfbc7911d4 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -521,6 +521,10 @@ class ColumnString final : public COWHelper { append_data_by_selector_impl(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + append_data_by_selector_impl(res, selector, begin, end); + } // void gather(ColumnGathererStream & gatherer_stream) override; void reserve(size_t n) override; diff --git a/be/src/vec/columns/column_struct.h b/be/src/vec/columns/column_struct.h index 2ca4fdec0153b4..5157b1ad6b09e5 100644 --- a/be/src/vec/columns/column_struct.h +++ b/be/src/vec/columns/column_struct.h @@ -132,6 +132,10 @@ class ColumnStruct final : public COWHelper { void append_data_by_selector(MutableColumnPtr& res, const Selector& selector) const override { return append_data_by_selector_impl(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const Selector& selector, size_t begin, + size_t end) const override { + return append_data_by_selector_impl(res, selector, begin, end); + } void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { LOG(FATAL) << "Method replace_column_data is not supported for " << get_name(); } diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index 7e035b8b47a68e..0b7d8350c5f44f 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -412,6 +412,10 @@ class ColumnVector final : public COWHelper> { const IColumn::Selector& selector) const override { this->template append_data_by_selector_impl(res, selector); } + void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, + size_t begin, size_t end) const override { + this->template append_data_by_selector_impl(res, selector, begin, end); + } bool is_fixed_and_contiguous() const override { return true; } size_t size_of_value_if_fixed() const override { return sizeof(T); } diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h index 2a390d39d294fe..0fdca54dc388f4 100644 --- a/be/src/vec/columns/predicate_column.h +++ b/be/src/vec/columns/predicate_column.h @@ -433,6 +433,10 @@ class PredicateColumnType final : public COWHelper(col_ptr); From f5ebf48a66e785ff4dc1bfcbd1bb11f2bd090769 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Mon, 15 Apr 2024 15:00:36 +0800 Subject: [PATCH 69/71] [opt](Nereids) prefer slot type to support delete task better (#33559) --- .../doris/nereids/types/DecimalV2Type.java | 4 + .../doris/nereids/util/TypeCoercionUtils.java | 96 +++++++++++++++++++ .../nereids/util/TypeCoercionUtilsTest.java | 67 +++++++++++++ 3 files changed, 167 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV2Type.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV2Type.java index 8a5a199a8d4ead..c0f023ed73933e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV2Type.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DecimalV2Type.java @@ -173,6 +173,10 @@ public int getScale() { return scale; } + public int getRange() { + return precision - scale; + } + @Override public DataType conversion() { if (Config.enable_decimal_conversion && shouldConversion) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java index afcdb30f2dcf03..214fbc180497fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/TypeCoercionUtils.java @@ -78,6 +78,7 @@ import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.DateTimeType; import org.apache.doris.nereids.types.DateTimeV2Type; +import org.apache.doris.nereids.types.DateType; import org.apache.doris.nereids.types.DateV2Type; import org.apache.doris.nereids.types.DecimalV2Type; import org.apache.doris.nereids.types.DecimalV3Type; @@ -120,6 +121,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -933,6 +935,18 @@ public static Expression processComparisonPredicate(ComparisonPredicate comparis Optional commonType = findWiderTypeForTwoForComparison( left.getDataType(), right.getDataType(), false); + + if (commonType.isPresent()) { + commonType = Optional.of(downgradeDecimalAndDateLikeType( + commonType.get(), + left, + right)); + commonType = Optional.of(downgradeDecimalAndDateLikeType( + commonType.get(), + right, + left)); + } + if (commonType.isPresent()) { if (!supportCompare(commonType.get())) { throw new AnalysisException("data type " + commonType.get() @@ -982,6 +996,13 @@ public static Expression processInPredicate(InPredicate inPredicate) { + " could not used in InPredicate " + inPredicate.toSql()); } + if (optionalCommonType.isPresent()) { + optionalCommonType = Optional.of(downgradeDecimalAndDateLikeType( + optionalCommonType.get(), + inPredicate.getCompareExpr(), + inPredicate.getOptions().toArray(new Expression[0]))); + } + return optionalCommonType .map(commonType -> { List newChildren = inPredicate.children().stream() @@ -992,6 +1013,81 @@ public static Expression processInPredicate(InPredicate inPredicate) { .orElse(inPredicate); } + /** + * if the expression like slot vs literal, then we prefer cast literal to slot type to ensure delete task could run. + * currently process decimalv2 vs decimalv3, datev1 vs datev2, datetimev1 vs datetimev2. + */ + private static DataType downgradeDecimalAndDateLikeType( + DataType commonType, + Expression target, + Expression... compareExpressions) { + if (shouldDowngrade(DecimalV3Type.class, DecimalV2Type.class, + commonType, target, + d -> ((DecimalV3Type) d).getRange() <= DecimalV2Type.MAX_PRECISION - DecimalV2Type.MAX_SCALE + && ((DecimalV3Type) d).getScale() <= DecimalV2Type.MAX_SCALE, + o -> o.isLiteral() + && (o.getDataType().isDecimalV2Type() || o.getDataType().isDecimalV3Type()), + compareExpressions)) { + DecimalV3Type decimalV3Type = (DecimalV3Type) commonType; + return DecimalV2Type.createDecimalV2Type(decimalV3Type.getPrecision(), decimalV3Type.getScale()); + } + // cast to datev1 for datev1 slot in (datev1 or datev2 literal) + if (shouldDowngrade(DateV2Type.class, DateType.class, + commonType, target, + d -> true, + o -> o.isLiteral() + && (o.getDataType().isDateType() || o.getDataType().isDateV2Type()), + compareExpressions)) { + return DateType.INSTANCE; + } + // cast to datetimev1 for datetimev1 slot in (date like literal) if scale is 0 + if (shouldDowngrade(DateTimeV2Type.class, DateTimeType.class, + commonType, target, + d -> ((DateTimeV2Type) d).getScale() == 0, + o -> o.isLiteral() && o.getDataType().isDateLikeType(), + compareExpressions)) { + return DateTimeType.INSTANCE; + } + return commonType; + } + + /** + * check should downgrade from commonTypeClazz to targetTypeClazz. + * @param commonTypeClazz before downgrade type + * @param targetTypeClazz try to downgrade to type + * @param commonType original common type + * @param target target expression aka slot + * @param commonTypePredicate constraint for original type + * @param otherPredicate constraint for other expressions aka literals + * @param others literals + * + * @return true for should downgrade + */ + private static boolean shouldDowngrade( + Class commonTypeClazz, + Class targetTypeClazz, + DataType commonType, + Expression target, + Function commonTypePredicate, + Function otherPredicate, + Expression... others) { + if (!commonTypeClazz.isInstance(commonType)) { + return false; + } + if (!targetTypeClazz.isInstance(target.getDataType())) { + return false; + } + if (!commonTypePredicate.apply(commonType)) { + return false; + } + for (Expression other : others) { + if (!otherPredicate.apply(other)) { + return false; + } + } + return true; + } + /** * process case when type coercion. */ diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java index d9a2946ba883c5..be465d9c371e4a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/TypeCoercionUtilsTest.java @@ -20,10 +20,17 @@ import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Divide; +import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.InPredicate; import org.apache.doris.nereids.trees.expressions.Multiply; +import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.Subtract; import org.apache.doris.nereids.trees.expressions.literal.CharLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DateLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral; +import org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal; +import org.apache.doris.nereids.trees.expressions.literal.DateV2Literal; import org.apache.doris.nereids.trees.expressions.literal.DecimalLiteral; import org.apache.doris.nereids.trees.expressions.literal.DecimalV3Literal; import org.apache.doris.nereids.trees.expressions.literal.DoubleLiteral; @@ -56,6 +63,7 @@ import org.apache.doris.nereids.types.VarcharType; import org.apache.doris.nereids.types.coercion.IntegralType; +import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -735,4 +743,63 @@ public void testDecimalArithmetic() { Assertions.assertEquals(expression.child(0), new Cast(multiply.child(0), DecimalV3Type.createDecimalV3Type(10, 3))); } + + @Test + public void testProcessInDowngrade() { + // DecimalV2 slot vs DecimalV3 literal + InPredicate decimalDowngrade = new InPredicate( + new SlotReference("c1", DecimalV2Type.createDecimalV2Type(15, 6)), + ImmutableList.of( + new DecimalV3Literal(BigDecimal.valueOf(12345.1234567)), + new DecimalLiteral(BigDecimal.valueOf(12345.1234)))); + decimalDowngrade = (InPredicate) TypeCoercionUtils.processInPredicate(decimalDowngrade); + Assertions.assertEquals(DecimalV2Type.createDecimalV2Type(16, 7), decimalDowngrade.getCompareExpr().getDataType()); + + // DateV1 slot vs DateV2 literal + InPredicate dateDowngrade = new InPredicate( + new SlotReference("c1", DateType.INSTANCE), + ImmutableList.of( + new DateLiteral(2024, 4, 12), + new DateV2Literal(2024, 4, 12))); + dateDowngrade = (InPredicate) TypeCoercionUtils.processInPredicate(dateDowngrade); + Assertions.assertEquals(DateType.INSTANCE, dateDowngrade.getCompareExpr().getDataType()); + + // DatetimeV1 slot vs DateLike literal + InPredicate datetimeDowngrade = new InPredicate( + new SlotReference("c1", DateTimeType.INSTANCE), + ImmutableList.of( + new DateLiteral(2024, 4, 12), + new DateV2Literal(2024, 4, 12), + new DateTimeLiteral(2024, 4, 12, 18, 25, 30), + new DateTimeV2Literal(2024, 4, 12, 18, 25, 30, 0))); + datetimeDowngrade = (InPredicate) TypeCoercionUtils.processInPredicate(datetimeDowngrade); + Assertions.assertEquals(DateTimeType.INSTANCE, datetimeDowngrade.getCompareExpr().getDataType()); + } + + @Test + public void testProcessComparisonPredicateDowngrade() { + // DecimalV2 slot vs DecimalV3 literal + EqualTo decimalDowngrade = new EqualTo( + new SlotReference("c1", DecimalV2Type.createDecimalV2Type(15, 6)), + new DecimalV3Literal(BigDecimal.valueOf(12345.1234567)) + ); + decimalDowngrade = (EqualTo) TypeCoercionUtils.processComparisonPredicate(decimalDowngrade); + Assertions.assertEquals(DecimalV2Type.createDecimalV2Type(16, 7), decimalDowngrade.left().getDataType()); + + // DateV1 slot vs DateV2 literal (this case cover right slot vs left literal) + EqualTo dateDowngrade = new EqualTo( + new DateV2Literal(2024, 4, 12), + new SlotReference("c1", DateType.INSTANCE) + ); + dateDowngrade = (EqualTo) TypeCoercionUtils.processComparisonPredicate(dateDowngrade); + Assertions.assertEquals(DateType.INSTANCE, dateDowngrade.left().getDataType()); + + // DatetimeV1 slot vs DateLike literal + EqualTo datetimeDowngrade = new EqualTo( + new SlotReference("c1", DateTimeType.INSTANCE), + new DateTimeV2Literal(2024, 4, 12, 18, 25, 30, 0) + ); + datetimeDowngrade = (EqualTo) TypeCoercionUtils.processComparisonPredicate(datetimeDowngrade); + Assertions.assertEquals(DateTimeType.INSTANCE, datetimeDowngrade.left().getDataType()); + } } From cac7ac971dd13d4992c83ef99aea5dcee3927dcd Mon Sep 17 00:00:00 2001 From: zy-kkk Date: Mon, 15 Apr 2024 15:26:19 +0800 Subject: [PATCH 70/71] [fix](catalog) Remove unexpected cleanup when reading jdbc data (#33529) --- .../main/java/org/apache/doris/jdbc/DefaultJdbcExecutor.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/DefaultJdbcExecutor.java b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/DefaultJdbcExecutor.java index fd10ea74a708ed..ef29ad17b2e645 100644 --- a/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/DefaultJdbcExecutor.java +++ b/fe/be-java-extensions/jdbc-scanner/src/main/java/org/apache/doris/jdbc/DefaultJdbcExecutor.java @@ -276,8 +276,6 @@ public long getBlockAddress(int batchSize, Map outputParams) } catch (Exception e) { LOG.warn("jdbc get block address exception: ", e); throw new UdfRuntimeException("jdbc get block address: ", e); - } finally { - block.clear(); } return outputTable.getMetaAddress(); } From 51373428cde6f9e2404209aa2f59a7ac7f10f575 Mon Sep 17 00:00:00 2001 From: zzzxl <33418555+zzzxl1993@users.noreply.github.com> Date: Mon, 15 Apr 2024 15:35:31 +0800 Subject: [PATCH 71/71] [feature](inverted index) add slop functionality to match_phrase (#33225) https://github.com/apache/doris-website/pull/553 doc --- .../inverted_index/query/phrase_query.cpp | 67 ++++++++-- .../inverted_index/query/phrase_query.h | 10 ++ .../segment_v2/inverted_index/query/query.h | 11 ++ .../segment_v2/inverted_index_reader.cpp | 36 +++--- .../rowset/segment_v2/inverted_index_reader.h | 5 +- .../test_index_match_phrase_slop.out | 75 +++++++++++ .../test_index_match_phrase_slop.groovy | 122 ++++++++++++++++++ 7 files changed, 300 insertions(+), 26 deletions(-) create mode 100644 regression-test/data/inverted_index_p0/test_index_match_phrase_slop.out create mode 100644 regression-test/suites/inverted_index_p0/test_index_match_phrase_slop.groovy diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.cpp b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.cpp index a4b7f7502d158f..9d242bce68a528 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.cpp @@ -17,11 +17,13 @@ #include "phrase_query.h" +#include + namespace doris::segment_v2 { PhraseQuery::PhraseQuery(const std::shared_ptr& searcher, const TQueryOptions& query_options) - : _searcher(searcher) {} + : _searcher(searcher), _query(std::make_unique()) {} PhraseQuery::~PhraseQuery() { for (auto& term_doc : _term_docs) { @@ -36,6 +38,25 @@ PhraseQuery::~PhraseQuery() { } } +void PhraseQuery::add(const InvertedIndexQueryInfo& query_info) { + if (query_info.terms.empty()) { + _CLTHROWA(CL_ERR_IllegalArgument, "PhraseQuery::add: terms empty"); + } + + _slop = query_info.slop; + if (_slop <= 0) { + add(query_info.field_name, query_info.terms); + } else { + for (const auto& term : query_info.terms) { + std::wstring ws_term = StringUtil::string_to_wstring(term); + auto* t = _CLNEW lucene::index::Term(query_info.field_name.c_str(), ws_term.c_str()); + _query->add(t); + _CLDECDELETE(t); + } + _query->setSlop(_slop); + } +} + void PhraseQuery::add(const std::wstring& field_name, const std::vector& terms) { if (terms.empty()) { _CLTHROWA(CL_ERR_IllegalArgument, "PhraseQuery::add: terms empty"); @@ -74,14 +95,20 @@ void PhraseQuery::add(const std::wstring& field_name, const std::vector_search(_query.get(), [&roaring](const int32_t docid, const float_t /*score*/) { + roaring.add(docid); + }); } - search_by_skiplist(roaring); } void PhraseQuery::search_by_bitmap(roaring::Roaring& roaring) { @@ -202,4 +229,28 @@ void PhraseQuery::reset() { } } +Status PhraseQuery::parser_slop(std::string& query, InvertedIndexQueryInfo& query_info) { + auto is_digits = [](const std::string_view& str) { + return std::all_of(str.begin(), str.end(), [](unsigned char c) { return std::isdigit(c); }); + }; + + size_t last_space_pos = query.find_last_of(' '); + if (last_space_pos != std::string::npos) { + size_t tilde_pos = last_space_pos + 1; + if (tilde_pos < query.size() - 1 && query[tilde_pos] == '~') { + size_t slop_pos = tilde_pos + 1; + std::string_view slop_str(query.data() + slop_pos, query.size() - slop_pos); + if (is_digits(slop_str)) { + auto result = std::from_chars(slop_str.begin(), slop_str.end(), query_info.slop); + if (result.ec != std::errc()) { + return Status::Error( + "PhraseQuery parser failed: {}", query); + } + query = query.substr(0, last_space_pos); + } + } + } + return Status::OK(); +} + } // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.h index 8f62989d86b619..41b5f2d2e9706f 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/phrase_query.h @@ -17,7 +17,10 @@ #pragma once +// clang-format off #include "olap/rowset/segment_v2/inverted_index/query/query.h" +#include "CLucene/search/PhraseQuery.h" +// clang-format on CL_NS_USE(index) CL_NS_USE(search) @@ -30,6 +33,7 @@ class PhraseQuery : public Query { const TQueryOptions& query_options); ~PhraseQuery() override; + void add(const InvertedIndexQueryInfo& query_info) override; void add(const std::wstring& field_name, const std::vector& terms) override; void search(roaring::Roaring& roaring) override; @@ -54,6 +58,9 @@ class PhraseQuery : public Query { bool advance_position(PostingsAndPosition& posting, int32_t target); void reset(); +public: + static Status parser_slop(std::string& query, InvertedIndexQueryInfo& query_info); + private: std::shared_ptr _searcher; @@ -65,6 +72,9 @@ class PhraseQuery : public Query { std::vector _terms; std::vector _term_docs; + + std::unique_ptr _query; + int32_t _slop = 0; }; } // namespace doris::segment_v2 \ No newline at end of file diff --git a/be/src/olap/rowset/segment_v2/inverted_index/query/query.h b/be/src/olap/rowset/segment_v2/inverted_index/query/query.h index 091ba7d395804c..011229aa667615 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index/query/query.h +++ b/be/src/olap/rowset/segment_v2/inverted_index/query/query.h @@ -26,6 +26,7 @@ #include +#include "common/status.h" #include "roaring/roaring.hh" CL_NS_USE(index) @@ -34,10 +35,20 @@ CL_NS_USE(util) namespace doris::segment_v2 { +struct InvertedIndexQueryInfo { + std::wstring field_name; + std::vector terms; + int32_t slop = 0; +}; + class Query { public: virtual ~Query() = default; + virtual void add(const InvertedIndexQueryInfo& query_info) { + add(query_info.field_name, query_info.terms); + } + // a unified data preparation interface that provides the field names to be queried and the terms for the query. // @param field_name The name of the field within the data source to search against. // @param terms a vector of tokenized strings that represent the search terms. diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp index 11c53bbabc03c2..068d36bc8d0ed4 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp @@ -38,8 +38,11 @@ #include #include #include +#include +#include "gutil/integral_types.h" #include "inverted_index_query_type.h" +#include "olap/rowset/segment_v2/inverted_index/query/phrase_query.h" #ifdef __clang__ #pragma clang diagnostic push @@ -261,14 +264,18 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run << search_str << "]"; try { - std::vector analyse_result; + InvertedIndexQueryInfo query_info; InvertedIndexQueryCache::CacheKey cache_key; auto index_file_key = _inverted_index_file_reader->get_index_file_key(&_index_meta); if (query_type == InvertedIndexQueryType::MATCH_REGEXP_QUERY) { cache_key = {index_file_key, column_name, query_type, search_str}; - analyse_result.emplace_back(search_str); + query_info.terms.emplace_back(search_str); } else { + if (query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY) { + RETURN_IF_ERROR(PhraseQuery::parser_slop(search_str, query_info)); + } + InvertedIndexCtxSPtr inverted_index_ctx = std::make_shared( get_inverted_index_parser_type_from_string( get_parser_string_from_properties(_index_meta.properties())), @@ -283,10 +290,10 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run } inverted_index_ctx->analyzer = analyzer.get(); auto reader = create_reader(inverted_index_ctx.get(), search_str); - get_analyse_result(analyse_result, reader.get(), analyzer.get(), column_name, + get_analyse_result(query_info.terms, reader.get(), analyzer.get(), column_name, query_type); } - if (analyse_result.empty()) { + if (query_info.terms.empty()) { auto msg = fmt::format( "token parser result is empty for query, " "please check your query: '{}' and index parser: '{}'", @@ -300,7 +307,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run } std::unique_ptr query; - std::wstring field_ws = std::wstring(column_name.begin(), column_name.end()); + query_info.field_name = std::wstring(column_name.begin(), column_name.end()); if (query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY || query_type == InvertedIndexQueryType::MATCH_PHRASE_PREFIX_QUERY || @@ -308,8 +315,10 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run query_type == InvertedIndexQueryType::MATCH_ALL_QUERY || query_type == InvertedIndexQueryType::EQUAL_QUERY || query_type == InvertedIndexQueryType::MATCH_ANY_QUERY) { - std::string str_tokens = join(analyse_result, " "); - + std::string str_tokens = join(query_info.terms, " "); + if (query_type == InvertedIndexQueryType::MATCH_PHRASE_QUERY) { + str_tokens += " " + std::to_string(query_info.slop); + } cache_key = {index_file_key, column_name, query_type, str_tokens}; } auto* cache = InvertedIndexQueryCache::instance(); @@ -329,12 +338,8 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run searcher_ptr = std::get_if(&searcher_variant); if (searcher_ptr != nullptr) { term_match_bitmap = std::make_shared(); - - Status res = match_index_search(stats, runtime_state, query_type, field_ws, - analyse_result, *searcher_ptr, term_match_bitmap); - if (!res.ok()) { - return res; - } + RETURN_IF_ERROR(match_index_search(stats, runtime_state, query_type, query_info, + *searcher_ptr, term_match_bitmap)); term_match_bitmap->runOptimize(); cache->insert(cache_key, term_match_bitmap, &cache_handler); bit_map = term_match_bitmap; @@ -348,8 +353,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, RuntimeState* run Status FullTextIndexReader::match_index_search( OlapReaderStatistics* stats, RuntimeState* runtime_state, InvertedIndexQueryType query_type, - const std::wstring& field_ws, const std::vector& analyse_result, - const FulltextIndexSearcherPtr& index_searcher, + const InvertedIndexQueryInfo& query_info, const FulltextIndexSearcherPtr& index_searcher, const std::shared_ptr& term_match_bitmap) { TQueryOptions queryOptions = runtime_state->query_options(); try { @@ -359,7 +363,7 @@ Status FullTextIndexReader::match_index_search( return Status::Error( "query type " + query_type_to_string(query_type) + ", query is nullptr"); } - query->add(field_ws, analyse_result); + query->add(query_info); query->search(*term_match_bitmap); } catch (const CLuceneError& e) { return Status::Error("CLuceneError occured: {}", diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.h b/be/src/olap/rowset/segment_v2/inverted_index_reader.h index 63002da5c92c0e..ffc8c7c75fd4df 100644 --- a/be/src/olap/rowset/segment_v2/inverted_index_reader.h +++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.h @@ -71,6 +71,7 @@ namespace segment_v2 { class InvertedIndexIterator; class InvertedIndexQueryCacheHandle; class InvertedIndexFileReader; +struct InvertedIndexQueryInfo; class InvertedIndexReader : public std::enable_shared_from_this { public: @@ -173,8 +174,8 @@ class FullTextIndexReader : public InvertedIndexReader { private: Status match_index_search(OlapReaderStatistics* stats, RuntimeState* runtime_state, - InvertedIndexQueryType query_type, const std::wstring& field_ws, - const std::vector& analyse_result, + InvertedIndexQueryType query_type, + const InvertedIndexQueryInfo& query_info, const FulltextIndexSearcherPtr& index_searcher, const std::shared_ptr& term_match_bitmap); }; diff --git a/regression-test/data/inverted_index_p0/test_index_match_phrase_slop.out b/regression-test/data/inverted_index_p0/test_index_match_phrase_slop.out new file mode 100644 index 00000000000000..404921dd401c08 --- /dev/null +++ b/regression-test/data/inverted_index_p0/test_index_match_phrase_slop.out @@ -0,0 +1,75 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +0 + +-- !sql -- +21 + +-- !sql -- +21 + +-- !sql -- +1 + +-- !sql -- +125 + +-- !sql -- +125 + +-- !sql -- +0 + +-- !sql -- +137 + +-- !sql -- +137 + +-- !sql -- +0 + +-- !sql -- +80 + +-- !sql -- +80 + +-- !sql -- +12 + +-- !sql -- +823 + +-- !sql -- +823 + +-- !sql -- +1 127.0.0.1 I'm glad I kept my fingers crossed ~4 1 1 + +-- !sql -- +1 127.0.0.1 I'm glad I kept my fingers crossed ~4 1 1 + +-- !sql -- +1 127.0.0.1 I'm glad I kept my fingers crossed ~4 1 1 + +-- !sql -- +1 127.0.0.1 I'm glad I kept my fingers crossed ~4 1 1 + +-- !sql -- + +-- !sql -- + +-- !sql -- + +-- !sql -- + +-- !sql -- + +-- !sql -- + +-- !sql -- + +-- !sql -- +1 127.0.0.1 I'm glad I kept my fingers crossed ~4 1 1 + diff --git a/regression-test/suites/inverted_index_p0/test_index_match_phrase_slop.groovy b/regression-test/suites/inverted_index_p0/test_index_match_phrase_slop.groovy new file mode 100644 index 00000000000000..a8454878e61993 --- /dev/null +++ b/regression-test/suites/inverted_index_p0/test_index_match_phrase_slop.groovy @@ -0,0 +1,122 @@ +// 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_index_match_phrase_slop", "p0"){ + def indexTbName1 = "test_index_match_phrase_slop" + + sql "DROP TABLE IF EXISTS ${indexTbName1}" + + sql """ + CREATE TABLE ${indexTbName1} ( + `@timestamp` int(11) NULL COMMENT "", + `clientip` varchar(20) NULL COMMENT "", + `request` text NULL COMMENT "", + `status` int(11) NULL COMMENT "", + `size` int(11) NULL COMMENT "", + INDEX request_idx (`request`) USING INVERTED PROPERTIES("parser" = "english", "support_phrase" = "true") COMMENT '' + ) ENGINE=OLAP + DUPLICATE KEY(`@timestamp`) + COMMENT "OLAP" + DISTRIBUTED BY RANDOM BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "disable_auto_compaction" = "true" + ); + """ + + def load_httplogs_data = {table_name, label, read_flag, format_flag, file_name, ignore_failure=false, + expected_succ_rows = -1, load_to_single_tablet = 'true' -> + + // load the json data + streamLoad { + table "${table_name}" + + // set http request header params + set 'label', label + "_" + UUID.randomUUID().toString() + set 'read_json_by_line', read_flag + set 'format', format_flag + file file_name // import json file + time 10000 // limit inflight 10s + if (expected_succ_rows >= 0) { + set 'max_filter_ratio', '1' + } + + // if declared a check callback, the default check condition will ignore. + // So you must check all condition + check { result, exception, startTime, endTime -> + if (ignore_failure && expected_succ_rows < 0) { return } + if (exception != null) { + throw exception + } + log.info("Stream load result: ${result}".toString()) + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + if (expected_succ_rows >= 0) { + assertEquals(json.NumberLoadedRows, expected_succ_rows) + } else { + assertEquals(json.NumberTotalRows, json.NumberLoadedRows + json.NumberUnselectedRows) + assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0) + } + } + } + } + + try { + load_httplogs_data.call(indexTbName1, 'test_index_match_phrase_slop', 'true', 'json', 'documents-1000.json') + + sql """ INSERT INTO ${indexTbName1} VALUES (1, "127.0.0.1", "I'm glad I kept my fingers crossed ~4", 1, 1); """ + + sql "sync" + + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'get jpg'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'get jpg ~2'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'get jpg ~2'; """ + + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'images bg'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'images bg ~1'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'images bg ~1'; """ + + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'images jpg'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'images jpg ~2'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'images jpg ~2'; """ + + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'french gif'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'french gif ~4'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'french gif ~4'; """ + + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'get http'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'get http ~6'; """ + qt_sql """ select count() from ${indexTbName1} where request match_phrase 'get http ~6'; """ + + qt_sql """ select * from ${indexTbName1} where request match_phrase 'crossed~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'crossed ~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad crossed ~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad crossed ~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad crossed ~4 '; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad crossed ~4.'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad crossed~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad~4crossed~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad ~4 crossed~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad ~4 crossed ~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad crossed \\\\~4'; """ + qt_sql """ select * from ${indexTbName1} where request match_phrase 'glad crossed \\~4'; """ + + } finally { + //try_sql("DROP TABLE IF EXISTS ${testTable}") + } +} \ No newline at end of file